Split AggregatorFactory into AggregatorBuilder and AggregatorFactory

This commit is contained in:
Colin Goodheart-Smithe 2016-01-27 17:38:20 +00:00
parent 19d9c588e1
commit a0b004acb5
202 changed files with 5286 additions and 4782 deletions

View File

@ -26,7 +26,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.SortBuilder;
@ -154,9 +154,9 @@ public class PercolateRequestBuilder extends BroadcastOperationRequestBuilder<Pe
/**
* Delegates to
* {@link PercolateSourceBuilder#addAggregation(AggregatorFactory)}
* {@link PercolateSourceBuilder#addAggregation(AggregatorBuilder)}
*/
public PercolateRequestBuilder addAggregation(AggregatorFactory<?> aggregationBuilder) {
public PercolateRequestBuilder addAggregation(AggregatorBuilder<?> aggregationBuilder) {
sourceBuilder().addAggregation(aggregationBuilder);
return this;
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
@ -52,7 +52,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
private List<SortBuilder> sorts;
private Boolean trackScores;
private HighlightBuilder highlightBuilder;
private List<AggregatorFactory<?>> aggregationFactorys;
private List<AggregatorBuilder<?>> aggregationFactorys;
private List<PipelineAggregatorFactory> pipelineAggregationFactorys;
/**
@ -125,7 +125,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
/**
* Add an aggregation definition.
*/
public PercolateSourceBuilder addAggregation(AggregatorFactory<?> aggregationBuilder) {
public PercolateSourceBuilder addAggregation(AggregatorBuilder<?> aggregationBuilder) {
if (aggregationFactorys == null) {
aggregationFactorys = new ArrayList<>();
}
@ -176,7 +176,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
builder.field("aggregations");
builder.startObject();
if (aggregationFactorys != null) {
for (AggregatorFactory<?> aggregation : aggregationFactorys) {
for (AggregatorBuilder<?> aggregation : aggregationFactorys) {
aggregation.toXContent(builder, params);
}
}

View File

@ -28,7 +28,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Template;
import org.elasticsearch.search.Scroll;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder;
@ -374,7 +374,7 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
/**
* Adds an aggregation to the search operation.
*/
public SearchRequestBuilder addAggregation(AggregatorFactory<?> aggregation) {
public SearchRequestBuilder addAggregation(AggregatorBuilder<?> aggregation) {
sourceBuilder().aggregation(aggregation);
return this;
}

View File

@ -39,7 +39,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
@ -666,10 +666,10 @@ public abstract class StreamInput extends InputStream {
}
/**
* Reads a {@link AggregatorFactory} from the current stream
* Reads a {@link AggregatorBuilder} from the current stream
*/
public AggregatorFactory readAggregatorFactory() throws IOException {
return readNamedWriteable(AggregatorFactory.class);
public AggregatorBuilder readAggregatorFactory() throws IOException {
return readNamedWriteable(AggregatorBuilder.class);
}
/**

View File

@ -38,7 +38,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.joda.time.ReadableInstant;
@ -643,9 +643,9 @@ public abstract class StreamOutput extends OutputStream {
}
/**
* Writes a {@link AggregatorFactory} to the current stream
* Writes a {@link AggregatorBuilder} to the current stream
*/
public void writeAggregatorFactory(AggregatorFactory factory) throws IOException {
public void writeAggregatorFactory(AggregatorBuilder factory) throws IOException {
writeNamedWriteable(factory);
}

View File

@ -22,64 +22,65 @@ import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.bucket.children.Children;
import org.elasticsearch.search.aggregations.bucket.children.ParentToChildrenAggregator;
import org.elasticsearch.search.aggregations.bucket.children.ParentToChildrenAggregator.ChildrenAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregator;
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregator.FilterAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.filters.Filters;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.FiltersAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridParser;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridParser.GeoGridAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator.GlobalAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramFactory;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.HistogramAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.missing.Missing;
import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregator;
import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregator.MissingAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.nested.Nested;
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregator;
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregator.NestedAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNestedAggregator;
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNestedAggregator.ReverseNestedAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.date.DateRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser.GeoDistanceFactory;
import org.elasticsearch.search.aggregations.bucket.range.ipv4.IPv4RangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.RangeAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.date.DateRangeAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser.GeoDistanceAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.ipv4.IPv4RangeAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.sampler.Sampler;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.DiversifiedAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.SamplerAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms;
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTermsAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTermsAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator;
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator.AvgAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregator;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregator.GeoBoundsAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroid;
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator;
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator.GeoCentroidAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.max.Max;
import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator;
import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator.MaxAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.min.Min;
import org.elasticsearch.search.aggregations.metrics.min.MinAggregator;
import org.elasticsearch.search.aggregations.metrics.min.MinAggregator.MinAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetric;
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetricAggregator;
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
import org.elasticsearch.search.aggregations.metrics.stats.StatsAggregator;
import org.elasticsearch.search.aggregations.metrics.stats.StatsAggregator.StatsAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregator;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregator;
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregator.SumAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHits;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregator;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregator.TopHitsAggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountAggregator;
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountAggregator.ValueCountAggregatorBuilder;
/**
* Utility class to create aggregations.
@ -92,234 +93,234 @@ public class AggregationBuilders {
/**
* Create a new {@link ValueCount} aggregation with the given name.
*/
public static ValueCountAggregator.Factory count(String name) {
return new ValueCountAggregator.Factory(name, null);
public static ValueCountAggregatorBuilder count(String name) {
return new ValueCountAggregatorBuilder(name, null);
}
/**
* Create a new {@link Avg} aggregation with the given name.
*/
public static AvgAggregator.Factory avg(String name) {
return new AvgAggregator.Factory(name);
public static AvgAggregatorBuilder avg(String name) {
return new AvgAggregatorBuilder(name);
}
/**
* Create a new {@link Max} aggregation with the given name.
*/
public static MaxAggregator.Factory max(String name) {
return new MaxAggregator.Factory(name);
public static MaxAggregatorBuilder max(String name) {
return new MaxAggregatorBuilder(name);
}
/**
* Create a new {@link Min} aggregation with the given name.
*/
public static MinAggregator.Factory min(String name) {
return new MinAggregator.Factory(name);
public static MinAggregatorBuilder min(String name) {
return new MinAggregatorBuilder(name);
}
/**
* Create a new {@link Sum} aggregation with the given name.
*/
public static SumAggregator.Factory sum(String name) {
return new SumAggregator.Factory(name);
public static SumAggregatorBuilder sum(String name) {
return new SumAggregatorBuilder(name);
}
/**
* Create a new {@link Stats} aggregation with the given name.
*/
public static StatsAggregator.Factory stats(String name) {
return new StatsAggregator.Factory(name);
public static StatsAggregatorBuilder stats(String name) {
return new StatsAggregatorBuilder(name);
}
/**
* Create a new {@link ExtendedStats} aggregation with the given name.
*/
public static ExtendedStatsAggregator.Factory extendedStats(String name) {
return new ExtendedStatsAggregator.Factory(name);
public static ExtendedStatsAggregatorBuilder extendedStats(String name) {
return new ExtendedStatsAggregatorBuilder(name);
}
/**
* Create a new {@link Filter} aggregation with the given name.
*/
public static FilterAggregator.Factory filter(String name, QueryBuilder<?> filter) {
return new FilterAggregator.Factory(name, filter);
public static FilterAggregatorBuilder filter(String name, QueryBuilder<?> filter) {
return new FilterAggregatorBuilder(name, filter);
}
/**
* Create a new {@link Filters} aggregation with the given name.
*/
public static FiltersAggregator.Factory filters(String name, KeyedFilter... filters) {
return new FiltersAggregator.Factory(name, filters);
public static FiltersAggregatorBuilder filters(String name, KeyedFilter... filters) {
return new FiltersAggregatorBuilder(name, filters);
}
/**
* Create a new {@link Filters} aggregation with the given name.
*/
public static FiltersAggregator.Factory filters(String name, QueryBuilder<?>... filters) {
return new FiltersAggregator.Factory(name, filters);
public static FiltersAggregatorBuilder filters(String name, QueryBuilder<?>... filters) {
return new FiltersAggregatorBuilder(name, filters);
}
/**
* Create a new {@link Sampler} aggregation with the given name.
*/
public static SamplerAggregator.Factory sampler(String name) {
return new SamplerAggregator.Factory(name);
public static SamplerAggregatorBuilder sampler(String name) {
return new SamplerAggregatorBuilder(name);
}
/**
* Create a new {@link Sampler} aggregation with the given name.
*/
public static SamplerAggregator.DiversifiedFactory diversifiedSampler(String name) {
return new SamplerAggregator.DiversifiedFactory(name);
public static DiversifiedAggregatorBuilder diversifiedSampler(String name) {
return new DiversifiedAggregatorBuilder(name);
}
/**
* Create a new {@link Global} aggregation with the given name.
*/
public static GlobalAggregator.Factory global(String name) {
return new GlobalAggregator.Factory(name);
public static GlobalAggregatorBuilder global(String name) {
return new GlobalAggregatorBuilder(name);
}
/**
* Create a new {@link Missing} aggregation with the given name.
*/
public static MissingAggregator.Factory missing(String name) {
return new MissingAggregator.Factory(name, null);
public static MissingAggregatorBuilder missing(String name) {
return new MissingAggregatorBuilder(name, null);
}
/**
* Create a new {@link Nested} aggregation with the given name.
*/
public static NestedAggregator.Factory nested(String name, String path) {
return new NestedAggregator.Factory(name, path);
public static NestedAggregatorBuilder nested(String name, String path) {
return new NestedAggregatorBuilder(name, path);
}
/**
* Create a new {@link ReverseNested} aggregation with the given name.
*/
public static ReverseNestedAggregator.Factory reverseNested(String name) {
return new ReverseNestedAggregator.Factory(name);
public static ReverseNestedAggregatorBuilder reverseNested(String name) {
return new ReverseNestedAggregatorBuilder(name);
}
/**
* Create a new {@link Children} aggregation with the given name.
*/
public static ParentToChildrenAggregator.Factory children(String name, String childType) {
return new ParentToChildrenAggregator.Factory(name, childType);
public static ChildrenAggregatorBuilder children(String name, String childType) {
return new ChildrenAggregatorBuilder(name, childType);
}
/**
* Create a new {@link GeoDistance} aggregation with the given name.
*/
public static GeoDistanceFactory geoDistance(String name, GeoPoint origin) {
return new GeoDistanceFactory(name, origin);
public static GeoDistanceAggregatorBuilder geoDistance(String name, GeoPoint origin) {
return new GeoDistanceAggregatorBuilder(name, origin);
}
/**
* Create a new {@link Histogram} aggregation with the given name.
*/
public static HistogramAggregator.Factory<?> histogram(String name) {
return new HistogramAggregator.Factory<>(name);
public static HistogramAggregatorBuilder histogram(String name) {
return new HistogramAggregatorBuilder(name);
}
/**
* Create a new {@link GeoHashGrid} aggregation with the given name.
*/
public static GeoHashGridParser.GeoGridFactory geohashGrid(String name) {
return new GeoHashGridParser.GeoGridFactory(name);
public static GeoGridAggregatorBuilder geohashGrid(String name) {
return new GeoGridAggregatorBuilder(name);
}
/**
* Create a new {@link SignificantTerms} aggregation with the given name.
*/
public static SignificantTermsAggregatorFactory significantTerms(String name) {
return new SignificantTermsAggregatorFactory(name, null);
public static SignificantTermsAggregatorBuilder significantTerms(String name) {
return new SignificantTermsAggregatorBuilder(name, null);
}
/**
* Create a new {@link DateHistogramFactory} aggregation with the given
* Create a new {@link DateHistogramAggregatorBuilder} aggregation with the given
* name.
*/
public static DateHistogramFactory dateHistogram(String name) {
return new DateHistogramFactory(name);
public static DateHistogramAggregatorBuilder dateHistogram(String name) {
return new DateHistogramAggregatorBuilder(name);
}
/**
* Create a new {@link Range} aggregation with the given name.
*/
public static RangeAggregator.Factory range(String name) {
return new RangeAggregator.Factory(name);
public static RangeAggregatorBuilder range(String name) {
return new RangeAggregatorBuilder(name);
}
/**
* Create a new {@link DateRangeAggregatorFactory} aggregation with the
* Create a new {@link DateRangeAggregatorBuilder} aggregation with the
* given name.
*/
public static DateRangeAggregatorFactory dateRange(String name) {
return new DateRangeAggregatorFactory(name);
public static DateRangeAggregatorBuilder dateRange(String name) {
return new DateRangeAggregatorBuilder(name);
}
/**
* Create a new {@link IPv4RangeAggregatorFactory} aggregation with the
* Create a new {@link IPv4RangeAggregatorBuilder} aggregation with the
* given name.
*/
public static IPv4RangeAggregatorFactory ipRange(String name) {
return new IPv4RangeAggregatorFactory(name);
public static IPv4RangeAggregatorBuilder ipRange(String name) {
return new IPv4RangeAggregatorBuilder(name);
}
/**
* Create a new {@link Terms} aggregation with the given name.
*/
public static TermsAggregatorFactory terms(String name) {
return new TermsAggregatorFactory(name, null);
public static TermsAggregatorBuilder terms(String name) {
return new TermsAggregatorBuilder(name, null);
}
/**
* Create a new {@link Percentiles} aggregation with the given name.
*/
public static PercentilesAggregatorFactory percentiles(String name) {
return new PercentilesAggregatorFactory(name);
public static PercentilesAggregatorBuilder percentiles(String name) {
return new PercentilesAggregatorBuilder(name);
}
/**
* Create a new {@link PercentileRanks} aggregation with the given name.
*/
public static PercentileRanksAggregatorFactory percentileRanks(String name) {
return new PercentileRanksAggregatorFactory(name);
public static PercentileRanksAggregatorBuilder percentileRanks(String name) {
return new PercentileRanksAggregatorBuilder(name);
}
/**
* Create a new {@link Cardinality} aggregation with the given name.
*/
public static CardinalityAggregatorFactory cardinality(String name) {
return new CardinalityAggregatorFactory(name, null);
public static CardinalityAggregatorBuilder cardinality(String name) {
return new CardinalityAggregatorBuilder(name, null);
}
/**
* Create a new {@link TopHits} aggregation with the given name.
*/
public static TopHitsAggregator.Factory topHits(String name) {
return new TopHitsAggregator.Factory(name);
public static TopHitsAggregatorBuilder topHits(String name) {
return new TopHitsAggregatorBuilder(name);
}
/**
* Create a new {@link GeoBounds} aggregation with the given name.
*/
public static GeoBoundsAggregator.Factory geoBounds(String name) {
return new GeoBoundsAggregator.Factory(name);
public static GeoBoundsAggregatorBuilder geoBounds(String name) {
return new GeoBoundsAggregatorBuilder(name);
}
/**
* Create a new {@link GeoCentroid} aggregation with the given name.
*/
public static GeoCentroidAggregator.Factory geoCentroid(String name) {
return new GeoCentroidAggregator.Factory(name);
public static GeoCentroidAggregatorBuilder geoCentroid(String name) {
return new GeoCentroidAggregatorBuilder(name);
}
/**
* Create a new {@link ScriptedMetric} aggregation with the given name.
*/
public static ScriptedMetricAggregator.Factory scriptedMetric(String name) {
return new ScriptedMetricAggregator.Factory(name);
public static ScriptedMetricAggregatorBuilder scriptedMetric(String name) {
return new ScriptedMetricAggregatorBuilder(name);
}
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
/**
@ -64,7 +65,8 @@ public class AggregationParseElement implements SearchParseElement {
QueryParseContext parseContext = new QueryParseContext(queriesRegistry);
parseContext.reset(parser);
parseContext.parseFieldMatcher(context.parseFieldMatcher());
AggregatorFactories factories = aggregatorParsers.parseAggregators(parser, parseContext);
context.aggregations(new SearchContextAggregations(factories));
AggregatorFactories.Builder factories = aggregatorParsers.parseAggregators(parser, parseContext);
AggregationContext aggContext = new AggregationContext(context);
context.aggregations(new SearchContextAggregations(factories.build(aggContext)));
}
}

View File

@ -43,7 +43,7 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
/**
* Parses the aggregation request and creates the appropriate aggregator factory for it.
*
* @see AggregatorFactory
* @see AggregatorBuilder
*/
public interface Parser {
@ -62,13 +62,13 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
* @return The resolved aggregator factory or {@code null} in case the aggregation should be skipped
* @throws java.io.IOException When parsing fails
*/
AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException;
AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException;
/**
* @return an empty {@link AggregatorFactory} instance for this parser
* @return an empty {@link AggregatorBuilder} instance for this parser
* that can be used for deserialization
*/
AggregatorFactory<?> getFactoryPrototypes();
AggregatorBuilder<?> getFactoryPrototypes();
}
/**

View File

@ -0,0 +1,173 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Objects;
/**
* A factory that knows how to create an {@link Aggregator} of a specific type.
*/
public abstract class AggregatorBuilder<AB extends AggregatorBuilder<AB>> extends AggregatorFactory<AB>
implements NamedWriteable<AB>, ToXContent {
protected AggregatorFactories.Builder factoriesBuilder = AggregatorFactories.builder();
/**
* Constructs a new aggregator factory.
*
* @param name The aggregation name
* @param type The aggregation type
*/
public AggregatorBuilder(String name, Type type) {
super(name, type);
}
/**
* Add a sub aggregation to this aggregation.
*/
@SuppressWarnings("unchecked")
public AB subAggregation(AggregatorBuilder<?> aggregation) {
factoriesBuilder.addAggregator(aggregation);
return (AB) this;
}
/**
* Add a sub aggregation to this aggregation.
*/
@SuppressWarnings("unchecked")
public AB subAggregation(PipelineAggregatorFactory aggregation) {
factoriesBuilder.addPipelineAggregator(aggregation);
return (AB) this;
}
/**
* Registers sub-factories with this factory. The sub-factory will be
* responsible for the creation of sub-aggregators under the aggregator
* created by this factory.
*
* @param subFactories
* The sub-factories
* @return this factory (fluent interface)
*/
@SuppressWarnings("unchecked")
public AB subAggregations(AggregatorFactories.Builder subFactories) {
this.factoriesBuilder = subFactories;
return (AB) this;
}
public final AggregatorFactory<?> build(AggregationContext context) throws IOException {
AggregatorFactory<?> factory = doBuild(context);
if (factoriesBuilder != null && factoriesBuilder.count() > 0) {
factory.subFactories(factoriesBuilder.build(context));
}
factory.setMetaData(metaData);
return factory;
}
// NORELEASE make this method abstract when agg refactoring is complete
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return this;
}
@Override
public final AB readFrom(StreamInput in) throws IOException {
String name = in.readString();
AB factory = doReadFrom(name, in);
factory.factoriesBuilder = AggregatorFactories.Builder.PROTOTYPE.readFrom(in);
factory.metaData = in.readMap();
return factory;
}
protected abstract AB doReadFrom(String name, StreamInput in) throws IOException;
@Override
public final void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
doWriteTo(out);
factoriesBuilder.writeTo(out);
out.writeMap(metaData);
}
protected abstract void doWriteTo(StreamOutput out) throws IOException;
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
if (this.metaData != null) {
builder.field("meta", this.metaData);
}
builder.field(type.name());
internalXContent(builder, params);
if (factoriesBuilder != null && (factoriesBuilder.count()) > 0) {
builder.field("aggregations");
factoriesBuilder.toXContent(builder, params);
}
return builder.endObject();
}
protected abstract XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException;
@Override
public String getWriteableName() {
return type.stream().toUtf8();
}
@Override
public int hashCode() {
return Objects.hash(factoriesBuilder, metaData, name, type, doHashCode());
}
protected abstract int doHashCode();
@Override
public boolean equals(Object obj) {
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
AggregatorBuilder<AB> other = (AggregatorBuilder<AB>) obj;
if (!Objects.equals(name, other.name))
return false;
if (!Objects.equals(type, other.type))
return false;
if (!Objects.equals(metaData, other.metaData))
return false;
if (!Objects.equals(factoriesBuilder, other.factoriesBuilder))
return false;
return doEquals(obj);
}
protected abstract boolean doEquals(Object obj);
}

View File

@ -31,8 +31,6 @@ import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@ -44,9 +42,9 @@ import java.util.Set;
/**
*
*/
public class AggregatorFactories extends ToXContentToBytes implements Writeable<AggregatorFactories> {
public class AggregatorFactories {
public static final AggregatorFactories EMPTY = new AggregatorFactories(new AggregatorFactory<?>[0],
public static final AggregatorFactories EMPTY = new AggregatorFactories(new AggregatorBuilder<?>[0],
new ArrayList<PipelineAggregatorFactory>());
private AggregatorFactory<?> parent;
@ -136,28 +134,24 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
}
}
public static class Builder {
public static class Builder extends ToXContentToBytes implements Writeable<Builder> {
public final static Builder PROTOTYPE = new Builder();
private final Set<String> names = new HashSet<>();
private final List<AggregatorFactory<?>> factories = new ArrayList<>();
private final List<AggregatorBuilder<?>> aggregatorBuilders = new ArrayList<>();
private final List<PipelineAggregatorFactory> pipelineAggregatorFactories = new ArrayList<>();
private boolean skipResolveOrder;
public Builder addAggregators(AggregatorFactories factories) {
for (AggregatorFactory<?> factory : factories.factories) {
addAggregator(factory);
}
for (PipelineAggregatorFactory factory : factories.pipelineAggregatorFactories) {
addPipelineAggregator(factory);
}
return this;
throw new UnsupportedOperationException("This needs to be removed");
}
public Builder addAggregator(AggregatorFactory<?> factory) {
public Builder addAggregator(AggregatorBuilder<?> factory) {
if (!names.add(factory.name)) {
throw new IllegalArgumentException("Two sibling aggregations cannot have the same name: [" + factory.name + "]");
}
factories.add(factory);
aggregatorBuilders.add(factory);
return this;
}
@ -174,27 +168,31 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
return this;
}
public AggregatorFactories build() {
if (factories.isEmpty() && pipelineAggregatorFactories.isEmpty()) {
public AggregatorFactories build(AggregationContext context) throws IOException {
if (aggregatorBuilders.isEmpty() && pipelineAggregatorFactories.isEmpty()) {
return EMPTY;
}
List<PipelineAggregatorFactory> orderedpipelineAggregators = null;
if (skipResolveOrder) {
orderedpipelineAggregators = new ArrayList<>(pipelineAggregatorFactories);
} else {
orderedpipelineAggregators = resolvePipelineAggregatorOrder(this.pipelineAggregatorFactories, this.factories);
orderedpipelineAggregators = resolvePipelineAggregatorOrder(this.pipelineAggregatorFactories, this.aggregatorBuilders);
}
return new AggregatorFactories(factories.toArray(new AggregatorFactory<?>[factories.size()]), orderedpipelineAggregators);
AggregatorFactory<?>[] aggFactories = new AggregatorFactory<?>[aggregatorBuilders.size()];
for (int i = 0; i < aggregatorBuilders.size(); i++) {
aggFactories[i] = aggregatorBuilders.get(i).build(context);
}
return new AggregatorFactories(aggFactories, orderedpipelineAggregators);
}
private List<PipelineAggregatorFactory> resolvePipelineAggregatorOrder(List<PipelineAggregatorFactory> pipelineAggregatorFactories,
List<AggregatorFactory<?>> aggFactories) {
List<AggregatorBuilder<?>> aggFactories) {
Map<String, PipelineAggregatorFactory> pipelineAggregatorFactoriesMap = new HashMap<>();
for (PipelineAggregatorFactory factory : pipelineAggregatorFactories) {
pipelineAggregatorFactoriesMap.put(factory.getName(), factory);
}
Map<String, AggregatorFactory<?>> aggFactoriesMap = new HashMap<>();
for (AggregatorFactory<?> aggFactory : aggFactories) {
Map<String, AggregatorBuilder<?>> aggFactoriesMap = new HashMap<>();
for (AggregatorBuilder<?> aggFactory : aggFactories) {
aggFactoriesMap.put(aggFactory.name, aggFactory);
}
List<PipelineAggregatorFactory> orderedPipelineAggregatorrs = new LinkedList<>();
@ -208,7 +206,7 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
return orderedPipelineAggregatorrs;
}
private void resolvePipelineAggregatorOrder(Map<String, AggregatorFactory<?>> aggFactoriesMap,
private void resolvePipelineAggregatorOrder(Map<String, AggregatorBuilder<?>> aggFactoriesMap,
Map<String, PipelineAggregatorFactory> pipelineAggregatorFactoriesMap,
List<PipelineAggregatorFactory> orderedPipelineAggregators, List<PipelineAggregatorFactory> unmarkedFactories, Set<PipelineAggregatorFactory> temporarilyMarked,
PipelineAggregatorFactory factory) {
@ -223,7 +221,7 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
if (bucketsPath.equals("_count") || bucketsPath.equals("_key")) {
continue;
} else if (aggFactoriesMap.containsKey(firstAggName)) {
AggregatorFactory<?> aggFactory = aggFactoriesMap.get(firstAggName);
AggregatorBuilder<?> aggFactory = aggFactoriesMap.get(firstAggName);
for (int i = 1; i < bucketsPathElements.size(); i++) {
PathElement pathElement = bucketsPathElements.get(i);
String aggName = pathElement.name;
@ -232,9 +230,9 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
} else {
// Check the non-pipeline sub-aggregator
// factories
AggregatorFactory<?>[] subFactories = aggFactory.factories.factories;
AggregatorBuilder<?>[] subFactories = aggFactory.factoriesBuilder.getAggregatorFactories();
boolean foundSubFactory = false;
for (AggregatorFactory<?> subFactory : subFactories) {
for (AggregatorBuilder<?> subFactory : subFactories) {
if (aggName.equals(subFactory.name)) {
aggFactory = subFactory;
foundSubFactory = true;
@ -243,7 +241,7 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
}
// Check the pipeline sub-aggregator factories
if (!foundSubFactory && (i == bucketsPathElements.size() - 1)) {
List<PipelineAggregatorFactory> subPipelineFactories = aggFactory.factories.pipelineAggregatorFactories;
List<PipelineAggregatorFactory> subPipelineFactories = aggFactory.factoriesBuilder.pipelineAggregatorFactories;
for (PipelineAggregatorFactory subFactory : subPipelineFactories) {
if (aggName.equals(subFactory.name())) {
foundSubFactory = true;
@ -275,79 +273,80 @@ public class AggregatorFactories extends ToXContentToBytes implements Writeable<
}
}
AggregatorFactory<?>[] getAggregatorFactories() {
return this.factories.toArray(new AggregatorFactory<?>[this.factories.size()]);
AggregatorBuilder<?>[] getAggregatorFactories() {
return this.aggregatorBuilders.toArray(new AggregatorBuilder<?>[this.aggregatorBuilders.size()]);
}
List<PipelineAggregatorFactory> getPipelineAggregatorFactories() {
return this.pipelineAggregatorFactories;
}
}
@Override
public AggregatorFactories readFrom(StreamInput in) throws IOException {
int factoriesSize = in.readVInt();
AggregatorFactory<?>[] factoriesList = new AggregatorFactory<?>[factoriesSize];
for (int i = 0; i < factoriesSize; i++) {
AggregatorFactory<?> factory = in.readAggregatorFactory();
factoriesList[i] = factory;
public int count() {
return aggregatorBuilders.size() + pipelineAggregatorFactories.size();
}
int pipelineFactoriesSize = in.readVInt();
List<PipelineAggregatorFactory> pipelineAggregatorFactoriesList = new ArrayList<PipelineAggregatorFactory>(pipelineFactoriesSize);
for (int i = 0; i < pipelineFactoriesSize; i++) {
PipelineAggregatorFactory factory = in.readPipelineAggregatorFactory();
pipelineAggregatorFactoriesList.add(factory);
}
AggregatorFactories aggregatorFactories = new AggregatorFactories(factoriesList,
Collections.unmodifiableList(pipelineAggregatorFactoriesList));
return aggregatorFactories;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(this.factories.length);
for (AggregatorFactory<?> factory : factories) {
out.writeAggregatorFactory(factory);
@Override
public Builder readFrom(StreamInput in) throws IOException {
Builder builder = new Builder();
int factoriesSize = in.readVInt();
for (int i = 0; i < factoriesSize; i++) {
AggregatorBuilder<?> factory = in.readAggregatorFactory();
builder.addAggregator(factory);
}
int pipelineFactoriesSize = in.readVInt();
for (int i = 0; i < pipelineFactoriesSize; i++) {
PipelineAggregatorFactory factory = in.readPipelineAggregatorFactory();
builder.addPipelineAggregator(factory);
}
return builder;
}
out.writeVInt(this.pipelineAggregatorFactories.size());
for (PipelineAggregatorFactory factory : pipelineAggregatorFactories) {
out.writePipelineAggregatorFactory(factory);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
if (factories != null) {
for (AggregatorFactory<?> subAgg : factories) {
subAgg.toXContent(builder, params);
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(this.aggregatorBuilders.size());
for (AggregatorBuilder<?> factory : aggregatorBuilders) {
out.writeAggregatorFactory(factory);
}
out.writeVInt(this.pipelineAggregatorFactories.size());
for (PipelineAggregatorFactory factory : pipelineAggregatorFactories) {
out.writePipelineAggregatorFactory(factory);
}
}
if (pipelineAggregatorFactories != null) {
for (PipelineAggregatorFactory subAgg : pipelineAggregatorFactories) {
subAgg.toXContent(builder, params);
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
if (aggregatorBuilders != null) {
for (AggregatorBuilder<?> subAgg : aggregatorBuilders) {
subAgg.toXContent(builder, params);
}
}
if (pipelineAggregatorFactories != null) {
for (PipelineAggregatorFactory subAgg : pipelineAggregatorFactories) {
subAgg.toXContent(builder, params);
}
}
builder.endObject();
return builder;
}
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(Arrays.hashCode(factories), pipelineAggregatorFactories);
}
@Override
public int hashCode() {
return Objects.hash(aggregatorBuilders, pipelineAggregatorFactories);
}
@Override
public boolean equals(Object obj) {
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
AggregatorFactories other = (AggregatorFactories) obj;
if (!Objects.deepEquals(factories, other.factories))
return false;
if (!Objects.equals(pipelineAggregatorFactories, other.pipelineAggregatorFactories))
return false;
return true;
@Override
public boolean equals(Object obj) {
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
Builder other = (Builder) obj;
if (!Objects.equals(aggregatorBuilders, other.aggregatorBuilders))
return false;
if (!Objects.equals(pipelineAggregatorFactories, other.pipelineAggregatorFactories))
return false;
return true;
}
}
}

View File

@ -16,35 +16,33 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations;
package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* A factory that knows how to create an {@link Aggregator} of a specific type.
*/
public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extends ToXContentToBytes
implements NamedWriteable<AggregatorFactory<AF>> {
// NORELEASE remove ToXContentToBytes here when agg refactoring complete
public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extends ToXContentToBytes {
// NORELEASE remove this method when agg refactoring complete
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
throw new UnsupportedOperationException("This should never be called");
}
protected String name;
protected Type type;
@ -56,8 +54,10 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
/**
* Constructs a new aggregator factory.
*
* @param name The aggregation name
* @param type The aggregation type
* @param name
* The aggregation name
* @param type
* The aggregation type
*/
public AggregatorFactory(String name, Type type) {
this.name = name;
@ -85,11 +85,13 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
}
/**
* Registers sub-factories with this factory. The sub-factory will be responsible for the creation of sub-aggregators under the
* aggregator created by this factory.
* Registers sub-factories with this factory. The sub-factory will be
* responsible for the creation of sub-aggregators under the aggregator
* created by this factory.
*
* @param subFactories The sub-factories
* @return this factory (fluent interface)
* @param subFactories
* The sub-factories
* @return this factory (fluent interface)
*/
public AF subFactories(AggregatorFactories subFactories) {
this.factories = subFactories;
@ -97,36 +99,13 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
return (AF) this;
}
/**
* Add a sub aggregation to this aggregation.
*/
@SuppressWarnings("unchecked")
public AF subAggregation(AggregatorFactory<?> aggregation) {
AggregatorFactories.Builder builder = AggregatorFactories.builder();
builder.addAggregators(factories);
builder.addAggregator(aggregation);
factories = builder.build();
return (AF) this;
}
/**
* Add a sub aggregation to this aggregation.
*/
@SuppressWarnings("unchecked")
public AF subAggregation(PipelineAggregatorFactory aggregation) {
AggregatorFactories.Builder builder = AggregatorFactories.builder();
builder.addAggregators(factories);
builder.addPipelineAggregator(aggregation);
factories = builder.build();
return (AF) this;
}
public String name() {
return name;
}
/**
* Validates the state of this factory (makes sure the factory is properly configured)
* Validates the state of this factory (makes sure the factory is properly
* configured)
*/
public final void validate() {
doValidate();
@ -134,23 +113,32 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
}
/**
* @return The parent factory if one exists (will always return {@code null} for top level aggregator factories).
* @return The parent factory if one exists (will always return {@code null}
* for top level aggregator factories).
*/
public AggregatorFactory<?> parent() {
return parent;
}
protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException;
// NORELEASE make this abstract when agg refactoring is complete
protected Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
throw new UnsupportedOperationException("THIS SHOULD NEVER BE CALLED");
}
/**
* Creates the aggregator
*
* @param parent The parent aggregator (if this is a top level factory, the parent will be {@code null})
* @param collectsFromSingleBucket If true then the created aggregator will only be collected with <tt>0</tt> as a bucket ordinal.
* Some factories can take advantage of this in order to return more optimized implementations.
* @param parent
* The parent aggregator (if this is a top level factory, the
* parent will be {@code null})
* @param collectsFromSingleBucket
* If true then the created aggregator will only be collected
* with <tt>0</tt> as a bucket ordinal. Some factories can take
* advantage of this in order to return more optimized
* implementations.
*
* @return The created aggregator
* @return The created aggregator
*/
public final Aggregator create(Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
return createInternal(context, parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData);
@ -164,64 +152,17 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
return (AF) this;
}
@Override
public final AggregatorFactory<AF> readFrom(StreamInput in) throws IOException {
String name = in.readString();
AggregatorFactory<AF> factory = doReadFrom(name, in);
factory.factories = AggregatorFactories.EMPTY.readFrom(in);
factory.factories.setParent(this);
factory.metaData = in.readMap();
return factory;
}
protected abstract AggregatorFactory<AF> doReadFrom(String name, StreamInput in) throws IOException;
@Override
public final void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
doWriteTo(out);
factories.writeTo(out);
out.writeMap(metaData);
}
protected abstract void doWriteTo(StreamOutput out) throws IOException;
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
if (this.metaData != null) {
builder.field("meta", this.metaData);
}
builder.field(type.name());
internalXContent(builder, params);
if (factories != null && (factories.countAggregators() + factories.countPipelineAggregators()) > 0) {
builder.field("aggregations");
factories.toXContent(builder, params);
}
return builder.endObject();
}
protected abstract XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException;
@Override
public String getWriteableName() {
return type.stream().toUtf8();
}
public String getType() {
return type.name();
}
/**
* Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how
* to collect bucket <tt>0</tt>, this returns an aggregator that can collect any bucket.
* Utility method. Given an {@link AggregatorFactory} that creates
* {@link Aggregator}s that only know how to collect bucket <tt>0</tt>, this
* returns an aggregator that can collect any bucket.
*/
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory,
final AggregationContext context, final Aggregator parent) throws IOException {
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory<?> factory, final AggregationContext context,
final Aggregator parent) throws IOException {
final Aggregator first = factory.create(parent, true);
final BigArrays bigArrays = context.bigArrays();
return new Aggregator() {
@ -340,31 +281,4 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> extend
};
}
@Override
public int hashCode() {
return Objects.hash(factories, metaData, name, type, doHashCode());
}
protected abstract int doHashCode();
@Override
public boolean equals(Object obj) {
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
AggregatorFactory<AF> other = (AggregatorFactory<AF>) obj;
if (!Objects.equals(name, other.name))
return false;
if (!Objects.equals(type, other.type))
return false;
if (!Objects.equals(metaData, other.metaData))
return false;
if (!Objects.equals(factories, other.factories))
return false;
return doEquals(obj);
}
protected abstract boolean doEquals(Object obj);
}
}

View File

@ -60,8 +60,8 @@ public class AggregatorParsers {
Map<String, Aggregator.Parser> aggParsersBuilder = new HashMap<>(aggParsers.size());
for (Aggregator.Parser parser : aggParsers) {
aggParsersBuilder.put(parser.type(), parser);
AggregatorFactory<?> factoryPrototype = parser.getFactoryPrototypes();
namedWriteableRegistry.registerPrototype(AggregatorFactory.class, factoryPrototype);
AggregatorBuilder<?> factoryPrototype = parser.getFactoryPrototypes();
namedWriteableRegistry.registerPrototype(AggregatorBuilder.class, factoryPrototype);
}
this.aggParsers = unmodifiableMap(aggParsersBuilder);
Map<String, PipelineAggregator.Parser> pipelineAggregatorParsersBuilder = new HashMap<>(pipelineAggregatorParsers.size());
@ -105,12 +105,13 @@ public class AggregatorParsers {
*
* @throws IOException When parsing fails for unknown reasons.
*/
public AggregatorFactories parseAggregators(XContentParser parser, QueryParseContext parseContext) throws IOException {
public AggregatorFactories.Builder parseAggregators(XContentParser parser, QueryParseContext parseContext) throws IOException {
return parseAggregators(parser, parseContext, 0);
}
private AggregatorFactories parseAggregators(XContentParser parser, QueryParseContext parseContext, int level) throws IOException {
private AggregatorFactories.Builder parseAggregators(XContentParser parser, QueryParseContext parseContext, int level)
throws IOException {
Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
@ -132,9 +133,9 @@ public class AggregatorParsers {
+ token + "], expected a [" + XContentParser.Token.START_OBJECT + "].");
}
AggregatorFactory aggFactory = null;
AggregatorBuilder<?> aggFactory = null;
PipelineAggregatorFactory pipelineAggregatorFactory = null;
AggregatorFactories subFactories = null;
AggregatorFactories.Builder subFactories = null;
Map<String, Object> metaData = null;
@ -222,7 +223,7 @@ public class AggregatorParsers {
}
if (subFactories != null) {
aggFactory.subFactories(subFactories);
aggFactory.subAggregations(subFactories);
}
if (level == 0) {
@ -245,7 +246,7 @@ public class AggregatorParsers {
}
}
return factories.build();
return factories;
}
}

View File

@ -0,0 +1,75 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.children;
import org.apache.lucene.search.Query;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class ChildrenAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, ChildrenAggregatorFactory> {
private final String parentType;
private final Query parentFilter;
private final Query childFilter;
public ChildrenAggregatorFactory(String name, Type type, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
Query parentFilter) {
super(name, type, config);
this.parentType = parentType;
this.childFilter = childFilter;
this.parentFilter = parentFilter;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
};
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, AggregationContext aggregationContext,
Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType);
return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter,
valuesSource, maxOrd, pipelineAggregators, metaData);
}
}

View File

@ -22,7 +22,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -37,7 +37,7 @@ public class ChildrenParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String childType = null;
XContentParser.Token token;
@ -63,11 +63,11 @@ public class ChildrenParser implements Aggregator.Parser {
}
return new ParentToChildrenAggregator.Factory(aggregationName, childType);
return new ParentToChildrenAggregator.ChildrenAggregatorBuilder(aggregationName, childType);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new ParentToChildrenAggregator.Factory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new ParentToChildrenAggregator.ChildrenAggregatorBuilder(null, null);
}
}

View File

@ -43,16 +43,17 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.FieldContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
import java.io.IOException;
import java.util.Arrays;
@ -189,7 +190,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets);
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, Factory> {
public static class ChildrenAggregatorBuilder extends ValuesSourceAggregatorBuilder<ParentChild, ChildrenAggregatorBuilder> {
private String parentType;
private final String childType;
@ -202,40 +203,20 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
* @param childType
* the type of children documents
*/
public Factory(String name, String childType) {
public ChildrenAggregatorBuilder(String name, String childType) {
super(name, InternalChildren.TYPE, ValuesSourceType.BYTES, ValueType.STRING);
this.childType = childType;
}
@Override
public void doInit(AggregationContext context) {
resolveConfig(context);
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ParentChild> config) {
return new ChildrenAggregatorFactory(name, type, config, parentType, childFilter, parentFilter);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
};
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource,
AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType);
return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter,
valuesSource, maxOrd, pipelineAggregators, metaData);
}
private void resolveConfig(AggregationContext aggregationContext) {
config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
protected ValuesSourceConfig<ParentChild> resolveConfig(AggregationContext aggregationContext) {
ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
DocumentMapper childDocMapper = aggregationContext.searchContext().mapperService().documentMapper(childType);
if (childDocMapper != null) {
@ -259,6 +240,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
} else {
config.unmapped(true);
}
return config;
}
@Override
@ -268,10 +250,10 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected ChildrenAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
String childType = in.readString();
Factory factory = new Factory(name, childType);
ChildrenAggregatorBuilder factory = new ChildrenAggregatorBuilder(name, childType);
return factory;
}
@ -287,7 +269,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
ChildrenAggregatorBuilder other = (ChildrenAggregatorBuilder) obj;
return Objects.equals(childType, other.childType);
}

View File

@ -19,8 +19,6 @@
package org.elasticsearch.search.aggregations.bucket.filter;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits;
import org.elasticsearch.common.io.stream.StreamInput;
@ -31,6 +29,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@ -86,7 +85,7 @@ public class FilterAggregator extends SingleBucketAggregator {
return new InternalFilter(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
public static class Factory extends AggregatorFactory<Factory> {
public static class FilterAggregatorBuilder extends AggregatorBuilder<FilterAggregatorBuilder> {
private QueryBuilder<?> filter;
@ -98,28 +97,14 @@ public class FilterAggregator extends SingleBucketAggregator {
* filter will fall into the bucket defined by this
* {@link Filter} aggregation.
*/
public Factory(String name, QueryBuilder<?> filter) {
public FilterAggregatorBuilder(String name, QueryBuilder<?> filter) {
super(name, InternalFilter.TYPE);
this.filter = filter;
}
// TODO: refactor in order to initialize the factory once with its parent,
// the context, etc. and then have a no-arg lightweight create method
// (since create may be called thousands of times)
private IndexSearcher searcher;
private Weight weight;
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
IndexSearcher contextSearcher = context.searchContext().searcher();
if (searcher != contextSearcher) {
searcher = contextSearcher;
Query filter = this.filter.toQuery(context.searchContext().indexShard().getQueryShardContext());
weight = contextSearcher.createNormalizedWeight(filter, false);
}
return new FilterAggregator(name, weight, factories, context, parent, pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return new FilterAggregatorFactory(name, type, filter, context);
}
@Override
@ -131,8 +116,8 @@ public class FilterAggregator extends SingleBucketAggregator {
}
@Override
protected Factory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory = new Factory(name, in.readQuery());
protected FilterAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
FilterAggregatorBuilder factory = new FilterAggregatorBuilder(name, in.readQuery());
return factory;
}
@ -148,7 +133,7 @@ public class FilterAggregator extends SingleBucketAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
FilterAggregatorBuilder other = (FilterAggregatorBuilder) obj;
return Objects.equals(filter, other.filter);
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class FilterAggregatorFactory extends AggregatorFactory<FilterAggregatorFactory> {
private final Weight weight;
public FilterAggregatorFactory(String name, Type type, QueryBuilder<?> filterBuilder, AggregationContext context) throws IOException {
super(name, type);
IndexSearcher contextSearcher = context.searchContext().searcher();
Query filter = filterBuilder.toQuery(context.searchContext().indexShard().getQueryShardContext());
weight = contextSearcher.createNormalizedWeight(filter, false);
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new FilterAggregator(name, weight, factories, context, parent, pipelineAggregators, metaData);
}
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -39,21 +39,22 @@ public class FilterParser implements Aggregator.Parser {
}
@Override
public FilterAggregator.Factory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public FilterAggregator.FilterAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
throws IOException {
QueryBuilder<?> filter = context.parseInnerQueryBuilder();
if (filter == null) {
throw new ParsingException(null, "filter cannot be null in filter aggregation [{}]", aggregationName);
}
FilterAggregator.Factory factory = new FilterAggregator.Factory(aggregationName,
FilterAggregator.FilterAggregatorBuilder factory = new FilterAggregator.FilterAggregatorBuilder(aggregationName,
filter == null ? new MatchAllQueryBuilder() : filter);
return factory;
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new FilterAggregator.Factory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new FilterAggregator.FilterAggregatorBuilder(null, null);
}
}

View File

@ -20,8 +20,6 @@
package org.elasticsearch.search.aggregations.bucket.filters;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits;
import org.elasticsearch.common.ParseField;
@ -35,6 +33,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
@ -198,7 +197,7 @@ public class FiltersAggregator extends BucketsAggregator {
return owningBucketOrdinal * totalNumKeys + filterOrd;
}
public static class Factory extends AggregatorFactory<Factory> {
public static class FiltersAggregatorBuilder extends AggregatorBuilder<FiltersAggregatorBuilder> {
private final List<KeyedFilter> filters;
private final boolean keyed;
@ -211,11 +210,11 @@ public class FiltersAggregator extends BucketsAggregator {
* @param filters
* the KeyedFilters to use with this aggregation.
*/
public Factory(String name, KeyedFilter... filters) {
public FiltersAggregatorBuilder(String name, KeyedFilter... filters) {
this(name, Arrays.asList(filters));
}
private Factory(String name, List<KeyedFilter> filters) {
private FiltersAggregatorBuilder(String name, List<KeyedFilter> filters) {
super(name, InternalFilters.TYPE);
this.filters = filters;
this.keyed = true;
@ -227,7 +226,7 @@ public class FiltersAggregator extends BucketsAggregator {
* @param filters
* the filters to use with this aggregation
*/
public Factory(String name, QueryBuilder<?>... filters) {
public FiltersAggregatorBuilder(String name, QueryBuilder<?>... filters) {
super(name, InternalFilters.TYPE);
List<KeyedFilter> keyedFilters = new ArrayList<>(filters.length);
for (int i = 0; i < filters.length; i++) {
@ -240,7 +239,7 @@ public class FiltersAggregator extends BucketsAggregator {
/**
* Set whether to include a bucket for documents not matching any filter
*/
public Factory otherBucket(boolean otherBucket) {
public FiltersAggregatorBuilder otherBucket(boolean otherBucket) {
this.otherBucket = otherBucket;
return this;
}
@ -256,7 +255,7 @@ public class FiltersAggregator extends BucketsAggregator {
* Set the key to use for the bucket for documents not matching any
* filter.
*/
public Factory otherBucketKey(String otherBucketKey) {
public FiltersAggregatorBuilder otherBucketKey(String otherBucketKey) {
this.otherBucketKey = otherBucketKey;
return this;
}
@ -269,31 +268,9 @@ public class FiltersAggregator extends BucketsAggregator {
return otherBucketKey;
}
// TODO: refactor in order to initialize the factory once with its parent,
// the context, etc. and then have a no-arg lightweight create method
// (since create may be called thousands of times)
private IndexSearcher searcher;
private String[] keys;
private Weight[] weights;
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
IndexSearcher contextSearcher = context.searchContext().searcher();
if (searcher != contextSearcher) {
searcher = contextSearcher;
weights = new Weight[filters.size()];
keys = new String[filters.size()];
for (int i = 0; i < filters.size(); ++i) {
KeyedFilter keyedFilter = filters.get(i);
this.keys[i] = keyedFilter.key;
Query filter = keyedFilter.filter.toFilter(context.searchContext().indexShard().getQueryShardContext());
this.weights[i] = contextSearcher.createNormalizedWeight(filter, false);
}
}
return new FiltersAggregator(name, factories, keys, weights, keyed, otherBucket ? otherBucketKey : null, context, parent,
pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return new FiltersAggregatorFactory(name, type, filters, keyed, otherBucket, otherBucketKey, context);
}
@Override
@ -319,22 +296,22 @@ public class FiltersAggregator extends BucketsAggregator {
}
@Override
protected Factory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory;
protected FiltersAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
FiltersAggregatorBuilder factory;
if (in.readBoolean()) {
int size = in.readVInt();
List<KeyedFilter> filters = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
filters.add(KeyedFilter.PROTOTYPE.readFrom(in));
}
factory = new Factory(name, filters);
factory = new FiltersAggregatorBuilder(name, filters);
} else {
int size = in.readVInt();
QueryBuilder<?>[] filters = new QueryBuilder<?>[size];
for (int i = 0; i < size; i++) {
filters[i] = in.readQuery();
}
factory = new Factory(name, filters);
factory = new FiltersAggregatorBuilder(name, filters);
}
factory.otherBucket = in.readBoolean();
factory.otherBucketKey = in.readString();
@ -366,7 +343,7 @@ public class FiltersAggregator extends BucketsAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
FiltersAggregatorBuilder other = (FiltersAggregatorBuilder) obj;
return Objects.equals(filters, other.filters)
&& Objects.equals(keyed, other.keyed)
&& Objects.equals(otherBucket, other.otherBucket)

View File

@ -0,0 +1,69 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.filters;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class FiltersAggregatorFactory extends AggregatorFactory<FiltersAggregatorFactory> {
private final String[] keys;
private final Weight[] weights;
private final boolean keyed;
private final boolean otherBucket;
private final String otherBucketKey;
public FiltersAggregatorFactory(String name, Type type, List<KeyedFilter> filters, boolean keyed, boolean otherBucket,
String otherBucketKey, AggregationContext context) throws IOException {
super(name, type);
this.keyed = keyed;
this.otherBucket = otherBucket;
this.otherBucketKey = otherBucketKey;
IndexSearcher contextSearcher = context.searchContext().searcher();
weights = new Weight[filters.size()];
keys = new String[filters.size()];
for (int i = 0; i < filters.size(); ++i) {
KeyedFilter keyedFilter = filters.get(i);
this.keys[i] = keyedFilter.key();
Query filter = keyedFilter.filter().toFilter(context.searchContext().indexShard().getQueryShardContext());
this.weights[i] = contextSearcher.createNormalizedWeight(filter, false);
}
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new FiltersAggregator(name, factories, keys, weights, keyed, otherBucket ? otherBucketKey : null, context, parent,
pipelineAggregators, metaData);
}
}

View File

@ -28,7 +28,7 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
import java.util.ArrayList;
@ -55,7 +55,7 @@ public class FiltersParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
List<FiltersAggregator.KeyedFilter> keyedFilters = null;
List<QueryBuilder<?>> nonKeyedFilters = null;
@ -125,11 +125,11 @@ public class FiltersParser implements Aggregator.Parser {
otherBucketKey = "_other_";
}
FiltersAggregator.Factory factory;
FiltersAggregator.FiltersAggregatorBuilder factory;
if (keyedFilters != null) {
factory = new FiltersAggregator.Factory(aggregationName, keyedFilters.toArray(new FiltersAggregator.KeyedFilter[keyedFilters.size()]));
factory = new FiltersAggregator.FiltersAggregatorBuilder(aggregationName, keyedFilters.toArray(new FiltersAggregator.KeyedFilter[keyedFilters.size()]));
} else {
factory = new FiltersAggregator.Factory(aggregationName, nonKeyedFilters.toArray(new QueryBuilder<?>[nonKeyedFilters.size()]));
factory = new FiltersAggregator.FiltersAggregatorBuilder(aggregationName, nonKeyedFilters.toArray(new QueryBuilder<?>[nonKeyedFilters.size()]));
}
if (otherBucket != null) {
factory.otherBucket(otherBucket);
@ -141,8 +141,8 @@ public class FiltersParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new FiltersAggregator.Factory(null, new FiltersAggregator.KeyedFilter[0]);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new FiltersAggregator.FiltersAggregatorBuilder(null, new FiltersAggregator.KeyedFilter[0]);
}
}

View File

@ -46,10 +46,10 @@ public class GeoHashGridAggregator extends BucketsAggregator {
private final int requiredSize;
private final int shardSize;
private final GeoHashGridParser.GeoGridFactory.CellIdSource valuesSource;
private final GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource valuesSource;
private final LongHash bucketOrds;
public GeoHashGridAggregator(String name, AggregatorFactories factories, GeoHashGridParser.GeoGridFactory.CellIdSource valuesSource,
public GeoHashGridAggregator(String name, AggregatorFactories factories, GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource valuesSource,
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);

View File

@ -0,0 +1,80 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.GeoPoint;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoHashGridAggregatorFactory> {
private final int precision;
private final int requiredSize;
private final int shardSize;
public GeoHashGridAggregatorFactory(String name, Type type, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
int shardSize) {
super(name, type, config);
this.precision = precision;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize,
Collections.<InternalGeoHashGrid.Bucket> emptyList(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext,
Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
CellIdSource cellIdSource = new CellIdSource(valuesSource, precision);
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent,
pipelineAggregators, metaData);
}
}

View File

@ -34,22 +34,18 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.SortingNumericDocValues;
import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -72,15 +68,15 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return InternalGeoHashGrid.TYPE.name();
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new GeoGridFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new GeoGridAggregatorBuilder(null);
}
@Override
protected GeoGridFactory createFactory(
protected GeoGridAggregatorBuilder createFactory(
String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
GeoGridFactory factory = new GeoGridFactory(aggregationName);
GeoGridAggregatorBuilder factory = new GeoGridAggregatorBuilder(aggregationName);
Integer precision = (Integer) otherOptions.get(GeoHashGridParams.FIELD_PRECISION);
if (precision != null) {
factory.precision(precision);
@ -114,17 +110,17 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return false;
}
public static class GeoGridFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoGridFactory> {
public static class GeoGridAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoGridAggregatorBuilder> {
private int precision = DEFAULT_PRECISION;
private int requiredSize = DEFAULT_MAX_NUM_CELLS;
private int shardSize = -1;
public GeoGridFactory(String name) {
public GeoGridAggregatorBuilder(String name) {
super(name, InternalGeoHashGrid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
public GeoGridFactory precision(int precision) {
public GeoGridAggregatorBuilder precision(int precision) {
this.precision = GeoHashGridParams.checkPrecision(precision);
return this;
}
@ -133,7 +129,7 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return precision;
}
public GeoGridFactory size(int size) {
public GeoGridAggregatorBuilder size(int size) {
this.requiredSize = size;
return this;
}
@ -142,7 +138,7 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return requiredSize;
}
public GeoGridFactory shardSize(int shardSize) {
public GeoGridAggregatorBuilder shardSize(int shardSize) {
this.shardSize = shardSize;
return this;
}
@ -152,53 +148,34 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize,
Collections.<InternalGeoHashGrid.Bucket> emptyList(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext,
Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config) {
int shardSize = this.shardSize;
if (shardSize == 0) {
shardSize = Integer.MAX_VALUE;
}
int requiredSize = this.requiredSize;
if (requiredSize == 0) {
requiredSize = Integer.MAX_VALUE;
}
if (shardSize < 0) {
// Use default heuristic to avoid any wrong-ranking caused by
// distributed counting
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, aggregationContext.searchContext().numberOfShards());
// Use default heuristic to avoid any wrong-ranking caused by distributed counting
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.searchContext().numberOfShards());
}
if (shardSize < requiredSize) {
shardSize = requiredSize;
}
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
CellIdSource cellIdSource = new CellIdSource(valuesSource, precision);
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, pipelineAggregators,
metaData);
return new GeoHashGridAggregatorFactory(name, type, config, precision, requiredSize, shardSize);
}
@Override
protected GeoGridFactory innerReadFrom(
protected GeoGridAggregatorBuilder innerReadFrom(
String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
GeoGridFactory factory = new GeoGridFactory(name);
GeoGridAggregatorBuilder factory = new GeoGridAggregatorBuilder(name);
factory.precision = in.readVInt();
factory.requiredSize = in.readVInt();
factory.shardSize = in.readVInt();
@ -222,7 +199,7 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
@Override
protected boolean innerEquals(Object obj) {
GeoGridFactory other = (GeoGridFactory) obj;
GeoGridAggregatorBuilder other = (GeoGridAggregatorBuilder) obj;
if (precision != other.precision) {
return false;
}

View File

@ -22,10 +22,9 @@ import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@ -71,28 +70,20 @@ public class GlobalAggregator extends SingleBucketAggregator {
throw new UnsupportedOperationException("global aggregations cannot serve as sub-aggregations, hence should never be called on #buildEmptyAggregations");
}
public static class Factory extends AggregatorFactory<Factory> {
public static class GlobalAggregatorBuilder extends AggregatorBuilder<GlobalAggregatorBuilder> {
public Factory(String name) {
public GlobalAggregatorBuilder(String name) {
super(name, InternalGlobal.TYPE);
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> 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 IllegalStateException();
}
return new GlobalAggregator(name, factories, context, pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return new GlobalAggregatorFactory(name, type);
}
@Override
protected Factory doReadFrom(String name, StreamInput in) throws IOException {
return new Factory(name);
protected GlobalAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
return new GlobalAggregatorBuilder(name);
}
@Override

View File

@ -0,0 +1,51 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class GlobalAggregatorFactory extends AggregatorFactory<GlobalAggregatorFactory> {
public GlobalAggregatorFactory(String name, Type type) {
super(name, type);
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> 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 IllegalStateException();
}
return new GlobalAggregator(name, factories, context, pipelineAggregators, metaData);
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -36,14 +36,14 @@ public class GlobalParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
parser.nextToken();
return new GlobalAggregator.Factory(aggregationName);
return new GlobalAggregator.GlobalAggregatorBuilder(aggregationName);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new GlobalAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new GlobalAggregator.GlobalAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
abstract class AbstractHistogramAggregatorFactory<AF extends AbstractHistogramAggregatorFactory<AF>>
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AF> {
protected final long interval;
protected final long offset;
protected final InternalOrder order;
protected final boolean keyed;
protected final long minDocCount;
protected final ExtendedBounds extendedBounds;
private final InternalHistogram.Factory<?> histogramFactory;
public AbstractHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval, long offset,
InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds,
InternalHistogram.Factory<?> histogramFactory) {
super(name, type, config);
this.interval = interval;
this.offset = offset;
this.order = order;
this.keyed = keyed;
this.minDocCount = minDocCount;
this.extendedBounds = extendedBounds;
this.histogramFactory = histogramFactory;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
Rounding rounding = createRounding();
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, extendedBounds, null, config.formatter(),
histogramFactory, aggregationContext, parent, pipelineAggregators, metaData);
}
protected Rounding createRounding() {
if (interval < 1) {
throw new ParsingException(null, "[interval] must be 1 or greater for histogram aggregation [" + name() + "]: " + interval);
}
Rounding rounding = new Rounding.Interval(interval);
if (offset != 0) {
rounding = new Rounding.OffsetRounding(rounding, offset);
}
return rounding;
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
Rounding rounding = createRounding();
// we need to round the bounds given by the user and we have to do it
// for every aggregator we create
// 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
ExtendedBounds roundedBounds = null;
if (extendedBounds != null) {
// we need to process & validate here using the parser
extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser());
roundedBounds = extendedBounds.round(rounding);
}
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource,
config.formatter(), histogramFactory, aggregationContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -0,0 +1,91 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.rounding.DateTimeUnit;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.rounding.TimeZoneRounding;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.util.HashMap;
import java.util.Map;
import static java.util.Collections.unmodifiableMap;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
public class DateHistogramAggregatorFactory extends AbstractHistogramAggregatorFactory<DateHistogramAggregatorFactory> {
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
private final DateHistogramInterval dateHistogramInterval;
static {
Map<String, DateTimeUnit> dateFieldUnits = new HashMap<>();
dateFieldUnits.put("year", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("1y", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("quarter", DateTimeUnit.QUARTER);
dateFieldUnits.put("1q", DateTimeUnit.QUARTER);
dateFieldUnits.put("month", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("1M", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("week", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("1w", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("day", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("1d", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("hour", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("1h", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("minute", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("1m", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("second", DateTimeUnit.SECOND_OF_MINUTE);
dateFieldUnits.put("1s", DateTimeUnit.SECOND_OF_MINUTE);
DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits);
}
public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval,
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
ExtendedBounds extendedBounds) {
super(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.HISTOGRAM_FACTORY);
this.dateHistogramInterval = dateHistogramInterval;
}
@Override
protected Rounding createRounding() {
TimeZoneRounding.Builder tzRoundingBuilder;
if (dateHistogramInterval != null) {
DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString());
if (dateTimeUnit != null) {
tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit);
} else {
// the interval is a time value?
tzRoundingBuilder = TimeZoneRounding.builder(
TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass().getSimpleName() + ".interval"));
}
} else {
// the interval is an integer time value in millis?
tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.timeValueMillis(interval));
}
if (timeZone() != null) {
tzRoundingBuilder.timeZone(timeZone());
}
Rounding rounding = tzRoundingBuilder.offset(offset).build();
return rounding;
}
}

View File

@ -21,8 +21,8 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -49,9 +49,9 @@ public class DateHistogramParser extends HistogramParser {
}
@Override
protected DateHistogramFactory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected DateHistogramAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
HistogramAggregator.DateHistogramFactory factory = new HistogramAggregator.DateHistogramFactory(aggregationName);
HistogramAggregator.DateHistogramAggregatorBuilder factory = new HistogramAggregator.DateHistogramAggregatorBuilder(aggregationName);
Object interval = otherOptions.get(Rounding.Interval.INTERVAL_FIELD);
if (interval == null) {
throw new ParsingException(null, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
@ -96,11 +96,11 @@ public class DateHistogramParser extends HistogramParser {
@Override
protected long parseStringOffset(String offset) throws IOException {
return DateHistogramFactory.parseStringOffset(offset);
return DateHistogramAggregatorBuilder.parseStringOffset(offset);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return HistogramAggregator.DateHistogramFactory.PROTOTYPE;
public AggregatorBuilder<?> getFactoryPrototypes() {
return HistogramAggregator.DateHistogramAggregatorBuilder.PROTOTYPE;
}
}

View File

@ -22,14 +22,11 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.rounding.DateTimeUnit;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.rounding.TimeZoneRounding;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -44,20 +41,18 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static java.util.Collections.unmodifiableMap;
public class HistogramAggregator extends BucketsAggregator {
public static final ParseField ORDER_FIELD = new ParseField("order");
@ -162,125 +157,91 @@ public class HistogramAggregator extends BucketsAggregator {
Releasables.close(bucketOrds);
}
public static class Factory<AF extends Factory<AF>> extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AF> {
public static class HistogramAggregatorBuilder extends AbstractBuilder<HistogramAggregatorBuilder> {
public static final HistogramAggregatorBuilder PROTOTYPE = new HistogramAggregatorBuilder("");
public static final Factory PROTOTYPE = new Factory("");
private long interval;
private long offset = 0;
private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
private boolean keyed = false;
private long minDocCount = 0;
private ExtendedBounds extendedBounds;
private final InternalHistogram.Factory<?> histogramFactory;
public Factory(String name) {
this(name, InternalHistogram.HISTOGRAM_FACTORY);
public HistogramAggregatorBuilder(String name) {
super(name, InternalHistogram.HISTOGRAM_FACTORY);
}
private Factory(String name, InternalHistogram.Factory<?> histogramFactory) {
@Override
protected HistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
return new HistogramAggregatorBuilder(name);
}
@Override
protected HistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds);
}
}
public static abstract class AbstractBuilder<AB extends AbstractBuilder<AB>>
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
protected long interval;
protected long offset = 0;
protected InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
protected boolean keyed = false;
protected long minDocCount = 0;
protected ExtendedBounds extendedBounds;
private AbstractBuilder(String name, InternalHistogram.Factory<?> histogramFactory) {
super(name, histogramFactory.type(), ValuesSourceType.NUMERIC, histogramFactory.valueType());
this.histogramFactory = histogramFactory;
}
public long interval() {
return interval;
}
public AF interval(long interval) {
public AB interval(long interval) {
this.interval = interval;
return (AF) this;
return (AB) this;
}
public long offset() {
return offset;
}
public AF offset(long offset) {
public AB offset(long offset) {
this.offset = offset;
return (AF) this;
return (AB) this;
}
public Histogram.Order order() {
return order;
}
public AF order(Histogram.Order order) {
public AB order(Histogram.Order order) {
this.order = (InternalOrder) order;
return (AF) this;
return (AB) this;
}
public boolean keyed() {
return keyed;
}
public AF keyed(boolean keyed) {
public AB keyed(boolean keyed) {
this.keyed = keyed;
return (AF) this;
return (AB) this;
}
public long minDocCount() {
return minDocCount;
}
public AF minDocCount(long minDocCount) {
public AB minDocCount(long minDocCount) {
this.minDocCount = minDocCount;
return (AF) this;
return (AB) this;
}
public ExtendedBounds extendedBounds() {
return extendedBounds;
}
public AF extendedBounds(ExtendedBounds extendedBounds) {
public AB extendedBounds(ExtendedBounds extendedBounds) {
this.extendedBounds = extendedBounds;
return (AF) this;
}
public InternalHistogram.Factory<?> getHistogramFactory() {
return histogramFactory;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
Rounding rounding = createRounding();
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, extendedBounds, null, config.formatter(),
histogramFactory, aggregationContext, parent, pipelineAggregators, metaData);
}
protected Rounding createRounding() {
if (interval < 1) {
throw new ParsingException(null, "[interval] must be 1 or greater for histogram aggregation [" + name() + "]: " + interval);
}
Rounding rounding = new Rounding.Interval(interval);
if (offset != 0) {
rounding = new Rounding.OffsetRounding(rounding, offset);
}
return rounding;
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
Rounding rounding = createRounding();
// we need to round the bounds given by the user and we have to do it for every aggregator we create
// 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
ExtendedBounds roundedBounds = null;
if (extendedBounds != null) {
// we need to process & validate here using the parser
extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser());
roundedBounds = extendedBounds.round(rounding);
}
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource,
config.formatter(), histogramFactory, aggregationContext, parent, pipelineAggregators, metaData);
return (AB) this;
}
@Override
@ -317,9 +278,9 @@ public class HistogramAggregator extends BucketsAggregator {
}
@Override
protected AF innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in)
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in)
throws IOException {
Factory<AF> factory = createFactoryFromStream(name, in);
AbstractBuilder<AB> factory = createFactoryFromStream(name, in);
factory.interval = in.readVLong();
factory.offset = in.readVLong();
if (in.readBoolean()) {
@ -330,13 +291,10 @@ public class HistogramAggregator extends BucketsAggregator {
if (in.readBoolean()) {
factory.extendedBounds = ExtendedBounds.readFrom(in);
}
return (AF) factory;
return (AB) factory;
}
protected Factory<AF> createFactoryFromStream(String name, StreamInput in)
throws IOException {
return new Factory<AF>(name);
}
protected abstract AB createFactoryFromStream(String name, StreamInput in) throws IOException;
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
@ -363,14 +321,13 @@ public class HistogramAggregator extends BucketsAggregator {
@Override
protected int innerHashCode() {
return Objects.hash(histogramFactory, interval, offset, order, keyed, minDocCount, extendedBounds);
return Objects.hash(interval, offset, order, keyed, minDocCount, extendedBounds);
}
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.equals(histogramFactory, other.histogramFactory)
&& Objects.equals(interval, other.interval)
AbstractBuilder other = (AbstractBuilder) obj;
return Objects.equals(interval, other.interval)
&& Objects.equals(offset, other.offset)
&& Objects.equals(order, other.order)
&& Objects.equals(keyed, other.keyed)
@ -379,57 +336,35 @@ public class HistogramAggregator extends BucketsAggregator {
}
}
public static class DateHistogramFactory extends Factory<DateHistogramFactory> {
public static class DateHistogramAggregatorBuilder extends AbstractBuilder<DateHistogramAggregatorBuilder> {
public static final DateHistogramFactory PROTOTYPE = new DateHistogramFactory("");
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
static {
Map<String, DateTimeUnit> dateFieldUnits = new HashMap<>();
dateFieldUnits.put("year", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("1y", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("quarter", DateTimeUnit.QUARTER);
dateFieldUnits.put("1q", DateTimeUnit.QUARTER);
dateFieldUnits.put("month", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("1M", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("week", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("1w", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("day", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("1d", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("hour", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("1h", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("minute", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("1m", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("second", DateTimeUnit.SECOND_OF_MINUTE);
dateFieldUnits.put("1s", DateTimeUnit.SECOND_OF_MINUTE);
DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits);
}
public static final DateHistogramAggregatorBuilder PROTOTYPE = new DateHistogramAggregatorBuilder("");
private DateHistogramInterval dateHistogramInterval;
public DateHistogramFactory(String name) {
public DateHistogramAggregatorBuilder(String name) {
super(name, InternalDateHistogram.HISTOGRAM_FACTORY);
}
/**
* Set the interval.
*/
public DateHistogramFactory dateHistogramInterval(DateHistogramInterval dateHistogramInterval) {
public DateHistogramAggregatorBuilder dateHistogramInterval(DateHistogramInterval dateHistogramInterval) {
this.dateHistogramInterval = dateHistogramInterval;
return this;
}
public DateHistogramFactory offset(String offset) {
public DateHistogramAggregatorBuilder offset(String offset) {
return offset(parseStringOffset(offset));
}
protected static long parseStringOffset(String offset) {
if (offset.charAt(0) == '-') {
return -TimeValue.parseTimeValue(offset.substring(1), null, DateHistogramFactory.class.getSimpleName() + ".parseOffset")
return -TimeValue.parseTimeValue(offset.substring(1), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
.millis();
}
int beginIndex = offset.charAt(0) == '+' ? 1 : 0;
return TimeValue.parseTimeValue(offset.substring(beginIndex), null, DateHistogramFactory.class.getSimpleName() + ".parseOffset")
return TimeValue.parseTimeValue(offset.substring(beginIndex), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
.millis();
}
@ -438,40 +373,9 @@ public class HistogramAggregator extends BucketsAggregator {
}
@Override
protected Rounding createRounding() {
TimeZoneRounding.Builder tzRoundingBuilder;
if (dateHistogramInterval != null) {
DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString());
if (dateTimeUnit != null) {
tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit);
} else {
// the interval is a time value?
tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass()
.getSimpleName() + ".interval"));
}
} else {
// the interval is an integer time value in millis?
tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.timeValueMillis(interval()));
}
if (timeZone() != null) {
tzRoundingBuilder.timeZone(timeZone());
}
Rounding rounding = tzRoundingBuilder.offset(offset()).build();
return rounding;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return super.createUnmapped(aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return super
.doCreateInternal(valuesSource, aggregationContext, parent, collectsFromSingleBucket, pipelineAggregators, metaData);
protected DateHistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed,
minDocCount, extendedBounds);
}
@Override
@ -490,9 +394,9 @@ public class HistogramAggregator extends BucketsAggregator {
}
@Override
protected DateHistogramFactory createFactoryFromStream(String name, StreamInput in)
protected DateHistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in)
throws IOException {
DateHistogramFactory factory = new DateHistogramFactory(name);
DateHistogramAggregatorBuilder factory = new DateHistogramAggregatorBuilder(name);
if (in.readBoolean()) {
factory.dateHistogramInterval = DateHistogramInterval.readFromStream(in);
}
@ -515,7 +419,7 @@ public class HistogramAggregator extends BucketsAggregator {
@Override
protected boolean innerEquals(Object obj) {
DateHistogramFactory other = (DateHistogramFactory) obj;
DateHistogramAggregatorBuilder other = (DateHistogramAggregatorBuilder) obj;
return super.innerEquals(obj)
&& Objects.equals(dateHistogramInterval, other.dateHistogramInterval);
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
public class HistogramAggregatorFactory extends AbstractHistogramAggregatorFactory<HistogramAggregatorFactory> {
public HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval, long offset,
InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds) {
super(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, InternalHistogram.HISTOGRAM_FACTORY);
}
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -51,9 +51,9 @@ public class HistogramParser extends NumericValuesSourceParser {
}
@Override
protected HistogramAggregator.Factory<?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected HistogramAggregator.AbstractBuilder<?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
HistogramAggregator.Factory factory = new HistogramAggregator.Factory(aggregationName);
HistogramAggregator.HistogramAggregatorBuilder factory = new HistogramAggregator.HistogramAggregatorBuilder(aggregationName);
Long interval = (Long) otherOptions.get(Rounding.Interval.INTERVAL_FIELD);
if (interval == null) {
throw new ParsingException(null, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
@ -163,7 +163,7 @@ public class HistogramParser extends NumericValuesSourceParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return HistogramAggregator.Factory.PROTOTYPE;
public AggregatorBuilder<?> getFactoryPrototypes() {
return HistogramAggregator.HistogramAggregatorBuilder.PROTOTYPE;
}
}

View File

@ -33,7 +33,9 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -85,28 +87,22 @@ public class MissingAggregator extends SingleBucketAggregator {
return new InternalMissing(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource, Factory> {
public static class MissingAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, MissingAggregatorBuilder> {
public Factory(String name, ValueType targetValueType) {
public MissingAggregatorBuilder(String name, ValueType targetValueType) {
super(name, InternalMissing.TYPE, ValuesSourceType.ANY, targetValueType);
}
@Override
protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, null, aggregationContext, parent, pipelineAggregators, metaData);
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource> config) {
return new MissingAggregatorFactory(name, type, config);
}
@Override
protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected MissingAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) {
return new Factory(name, targetValueType);
return new MissingAggregatorBuilder(name, targetValueType);
}
@Override

View File

@ -0,0 +1,53 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
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.List;
import java.util.Map;
public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, MissingAggregatorFactory> {
public MissingAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config) {
super(name, type, config);
}
@Override
protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, null, aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -47,13 +47,13 @@ public class MissingParser extends AnyValuesSourceParser {
}
@Override
protected MissingAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected MissingAggregator.MissingAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
return new MissingAggregator.Factory(aggregationName, targetValueType);
return new MissingAggregator.MissingAggregatorBuilder(aggregationName, targetValueType);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new MissingAggregator.Factory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new MissingAggregator.MissingAggregatorBuilder(null, null);
}
}

View File

@ -34,14 +34,13 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -150,7 +149,7 @@ public class NestedAggregator extends SingleBucketAggregator {
return null;
}
public static class Factory extends AggregatorFactory<Factory> {
public static class NestedAggregatorBuilder extends AggregatorBuilder<NestedAggregatorBuilder> {
private final String path;
@ -161,7 +160,7 @@ public class NestedAggregator extends SingleBucketAggregator {
* the path to use for this nested aggregation. The path must
* match the path to a nested object in the mappings.
*/
public Factory(String name, String path) {
public NestedAggregatorBuilder(String name, String path) {
super(name, InternalNested.TYPE);
this.path = path;
}
@ -174,19 +173,8 @@ public class NestedAggregator extends SingleBucketAggregator {
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
}
ObjectMapper objectMapper = context.searchContext().getObjectMapper(path);
if (objectMapper == null) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
}
if (!objectMapper.nested().isNested()) {
throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested");
}
return new NestedAggregator(name, factories, objectMapper, context, parent, pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return new NestedAggregatorFactory(name, type, path);
}
@Override
@ -198,9 +186,9 @@ public class NestedAggregator extends SingleBucketAggregator {
}
@Override
protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
protected NestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
String path = in.readString();
Factory factory = new Factory(name, path);
NestedAggregatorBuilder factory = new NestedAggregatorBuilder(name, path);
return factory;
}
@ -216,22 +204,9 @@ public class NestedAggregator extends SingleBucketAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
NestedAggregatorBuilder other = (NestedAggregatorBuilder) obj;
return Objects.equals(path, other.path);
}
private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalNested(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
}
}
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class NestedAggregatorFactory extends AggregatorFactory<NestedAggregatorFactory> {
private final String path;
public NestedAggregatorFactory(String name, Type type, String path) {
super(name, type);
this.path = path;
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
}
ObjectMapper objectMapper = context.searchContext().getObjectMapper(path);
if (objectMapper == null) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
}
if (!objectMapper.nested().isNested()) {
throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested");
}
return new NestedAggregator(name, factories, objectMapper, context, parent, pipelineAggregators, metaData);
}
private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalNested(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
}
}

View File

@ -22,7 +22,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -37,7 +37,7 @@ public class NestedParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String path = null;
XContentParser.Token token;
@ -62,11 +62,11 @@ public class NestedParser implements Aggregator.Parser {
throw new ParsingException(parser.getTokenLocation(), "Missing [path] field for nested aggregation [" + aggregationName + "]");
}
return new NestedAggregator.Factory(aggregationName, path);
return new NestedAggregator.NestedAggregatorBuilder(aggregationName, path);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new NestedAggregator.Factory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new NestedAggregator.NestedAggregatorBuilder(null, null);
}
}

View File

@ -30,15 +30,13 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -101,15 +99,6 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
};
}
private static NestedAggregator findClosestNestedAggregator(Aggregator parent) {
for (; parent != null; parent = parent.parent()) {
if (parent instanceof NestedAggregator) {
return (NestedAggregator) parent;
}
}
return null;
}
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), pipelineAggregators(),
@ -125,11 +114,11 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
return parentFilter;
}
public static class Factory extends AggregatorFactory<Factory> {
public static class ReverseNestedAggregatorBuilder extends AggregatorBuilder<ReverseNestedAggregatorBuilder> {
private String path;
public Factory(String name) {
public ReverseNestedAggregatorBuilder(String name) {
super(name, InternalReverseNested.TYPE);
}
@ -138,7 +127,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
* the path to a nested object in the mappings. If it is not specified
* then this aggregation will go back to the root document.
*/
public Factory path(String path) {
public ReverseNestedAggregatorBuilder path(String path) {
this.path = path;
return this;
}
@ -151,28 +140,8 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
// Early validation
NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
if (closestNestedAggregator == null) {
throw new SearchParseException(context.searchContext(), "Reverse nested aggregation [" + name
+ "] can only be used inside a [nested] aggregation", null);
}
final ObjectMapper objectMapper;
if (path != null) {
objectMapper = context.searchContext().getObjectMapper(path);
if (objectMapper == null) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
}
if (!objectMapper.nested().isNested()) {
throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested");
}
} else {
objectMapper = null;
}
return new ReverseNestedAggregator(name, factories, objectMapper, context, parent, pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) throws IOException {
return new ReverseNestedAggregatorFactory(name, type, path);
}
@Override
@ -186,8 +155,8 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
}
@Override
protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory = new Factory(name);
protected ReverseNestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregatorBuilder(name);
factory.path = in.readOptionalString();
return factory;
}
@ -204,21 +173,8 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
ReverseNestedAggregatorBuilder other = (ReverseNestedAggregatorBuilder) obj;
return Objects.equals(path, other.path);
}
private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalReverseNested(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
}
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class ReverseNestedAggregatorFactory extends AggregatorFactory<ReverseNestedAggregatorFactory> {
private final String path;
public ReverseNestedAggregatorFactory(String name, Type type, String path) {
super(name, type);
this.path = path;
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
// Early validation
NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
if (closestNestedAggregator == null) {
throw new SearchParseException(context.searchContext(),
"Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation", null);
}
final ObjectMapper objectMapper;
if (path != null) {
objectMapper = context.searchContext().getObjectMapper(path);
if (objectMapper == null) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
}
if (!objectMapper.nested().isNested()) {
throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested");
}
} else {
objectMapper = null;
}
return new ReverseNestedAggregator(name, factories, objectMapper, context, parent, pipelineAggregators, metaData);
}
private static NestedAggregator findClosestNestedAggregator(Aggregator parent) {
for (; parent != null; parent = parent.parent()) {
if (parent instanceof NestedAggregator) {
return (NestedAggregator) parent;
}
}
return null;
}
private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalReverseNested(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
}
}

View File

@ -22,7 +22,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -37,7 +37,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String path = null;
XContentParser.Token token;
@ -57,7 +57,8 @@ public class ReverseNestedParser implements Aggregator.Parser {
}
}
ReverseNestedAggregator.Factory factory = new ReverseNestedAggregator.Factory(aggregationName);
ReverseNestedAggregator.ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregator.ReverseNestedAggregatorBuilder(
aggregationName);
if (path != null) {
factory.path(path);
}
@ -65,7 +66,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new ReverseNestedAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new ReverseNestedAggregator.ReverseNestedAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class AbstractRangeAggregatorFactory<AF extends AbstractRangeAggregatorFactory<AF, R>, R extends Range>
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AF> {
private final InternalRange.Factory<?, ?> rangeFactory;
private final List<R> ranges;
private final boolean keyed;
public AbstractRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, List<R> ranges, boolean keyed,
InternalRange.Factory<?, ?> rangeFactory) {
super(name, type, config);
this.ranges = ranges;
this.keyed = keyed;
this.rangeFactory = rangeFactory;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent,
pipelineAggregators, metaData);
}
}

View File

@ -41,8 +41,10 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import org.elasticsearch.search.aggregations.support.format.ValueParser;
@ -396,48 +398,36 @@ public class RangeAggregator extends BucketsAggregator {
}
}
public static abstract class AbstractFactory<AF extends AbstractFactory<AF, R>, R extends Range>
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AF> {
public static abstract class AbstractBuilder<AB extends AbstractBuilder<AB, R>, R extends Range>
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
private final InternalRange.Factory rangeFactory;
private List<R> ranges = new ArrayList<>();
private boolean keyed = false;
protected final InternalRange.Factory<?, ?> rangeFactory;
protected List<R> ranges = new ArrayList<>();
protected boolean keyed = false;
protected AbstractFactory(String name, InternalRange.Factory rangeFactory) {
protected AbstractBuilder(String name, InternalRange.Factory<?, ?> rangeFactory) {
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
this.rangeFactory = rangeFactory;
}
public AF addRange(R range) {
public AB addRange(R range) {
ranges.add(range);
return (AF) this;
return (AB) this;
}
public List<R> ranges() {
return ranges;
}
public AF keyed(boolean keyed) {
public AB keyed(boolean keyed) {
this.keyed = keyed;
return (AF) this;
return (AB) this;
}
public boolean keyed() {
return keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(RANGES_FIELD.getPreferredName(), ranges);
@ -446,14 +436,14 @@ public class RangeAggregator extends BucketsAggregator {
}
@Override
protected AF innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
AbstractFactory<AF, R> factory = createFactoryFromStream(name, in);
AbstractBuilder<AB, R> factory = createFactoryFromStream(name, in);
factory.keyed = in.readBoolean();
return (AF) factory;
return (AB) factory;
}
protected abstract AbstractFactory<AF, R> createFactoryFromStream(String name, StreamInput in) throws IOException;
protected abstract AbstractBuilder<AB, R> createFactoryFromStream(String name, StreamInput in) throws IOException;
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
@ -471,15 +461,15 @@ public class RangeAggregator extends BucketsAggregator {
@Override
protected boolean innerEquals(Object obj) {
AbstractFactory<AF, R> other = (AbstractFactory<AF, R>) obj;
AbstractBuilder<AB, R> other = (AbstractBuilder<AB, R>) obj;
return Objects.equals(ranges, other.ranges)
&& Objects.equals(keyed, other.keyed);
}
}
public static class Factory extends AbstractFactory<Factory, Range> {
public static class RangeAggregatorBuilder extends AbstractBuilder<RangeAggregatorBuilder, Range> {
public Factory(String name) {
public RangeAggregatorBuilder(String name) {
super(name, InternalRange.FACTORY);
}
@ -493,7 +483,7 @@ public class RangeAggregator extends BucketsAggregator {
* @param to
* the upper bound on the distances, exclusive
*/
public Factory addRange(String key, double from, double to) {
public RangeAggregatorBuilder addRange(String key, double from, double to) {
addRange(new Range(key, from, to));
return this;
}
@ -503,7 +493,7 @@ public class RangeAggregator extends BucketsAggregator {
* automatically generated based on <code>from</code> and
* <code>to</code>.
*/
public Factory addRange(double from, double to) {
public RangeAggregatorBuilder addRange(double from, double to) {
return addRange(null, from, to);
}
@ -515,7 +505,7 @@ public class RangeAggregator extends BucketsAggregator {
* @param to
* the upper bound on the distances, exclusive
*/
public Factory addUnboundedTo(String key, double to) {
public RangeAggregatorBuilder addUnboundedTo(String key, double to) {
addRange(new Range(key, null, to));
return this;
}
@ -524,7 +514,7 @@ public class RangeAggregator extends BucketsAggregator {
* Same as {@link #addUnboundedTo(String, double)} but the key will be
* computed automatically.
*/
public Factory addUnboundedTo(double to) {
public RangeAggregatorBuilder addUnboundedTo(double to) {
return addUnboundedTo(null, to);
}
@ -536,7 +526,7 @@ public class RangeAggregator extends BucketsAggregator {
* @param from
* the lower bound on the distances, inclusive
*/
public Factory addUnboundedFrom(String key, double from) {
public RangeAggregatorBuilder addUnboundedFrom(String key, double from) {
addRange(new Range(key, from, null));
return this;
}
@ -545,14 +535,19 @@ public class RangeAggregator extends BucketsAggregator {
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
* computed automatically.
*/
public Factory addUnboundedFrom(double from) {
public RangeAggregatorBuilder addUnboundedFrom(double from) {
return addUnboundedFrom(null, from);
}
@Override
protected Factory createFactoryFromStream(String name, StreamInput in) throws IOException {
protected RangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new RangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory);
}
@Override
protected RangeAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
int size = in.readVInt();
Factory factory = new Factory(name);
RangeAggregatorBuilder factory = new RangeAggregatorBuilder(name);
for (int i = 0; i < size; i++) {
factory.addRange(Range.PROTOTYPE.readFrom(in));
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.util.List;
public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory<RangeAggregatorFactory, RangeAggregator.Range> {
public RangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, List<Range> ranges, boolean keyed,
Factory<?, ?> rangeFactory) {
super(name, type, config, ranges, keyed, rangeFactory);
}
}

View File

@ -22,7 +22,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
@ -52,9 +52,9 @@ public class RangeParser extends NumericValuesSourceParser {
}
@Override
protected RangeAggregator.AbstractFactory<?, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected RangeAggregator.AbstractBuilder<?, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
RangeAggregator.Factory factory = new RangeAggregator.Factory(aggregationName);
RangeAggregator.RangeAggregatorBuilder factory = new RangeAggregator.RangeAggregatorBuilder(aggregationName);
List<? extends Range> ranges = (List<? extends Range>) otherOptions.get(RangeAggregator.RANGES_FIELD);
for (Range range : ranges) {
factory.addRange(range);
@ -94,7 +94,7 @@ public class RangeParser extends NumericValuesSourceParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new RangeAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new RangeAggregator.RangeAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,262 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.range.date;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.joda.time.DateTime;
import java.io.IOException;
public class DateRangeAggregatorBuilder extends AbstractBuilder<DateRangeAggregatorBuilder, RangeAggregator.Range> {
public DateRangeAggregatorBuilder(String name) {
super(name, InternalDateRange.FACTORY);
}
@Override
public String getWriteableName() {
return InternalDateRange.TYPE.name();
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addRange(String key, String from, String to) {
addRange(new Range(key, from, to));
return this;
}
/**
* Same as {@link #addRange(String, String, String)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorBuilder addRange(String from, String to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addUnboundedTo(String key, String to) {
addRange(new Range(key, null, to));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, String)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedTo(String to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorBuilder addUnboundedFrom(String key, String from) {
addRange(new Range(key, from, null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, String)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedFrom(String from) {
return addUnboundedFrom(null, from);
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addRange(String key, double from, double to) {
addRange(new Range(key, from, to));
return this;
}
/**
* Same as {@link #addRange(String, double, double)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorBuilder addRange(double from, double to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addUnboundedTo(String key, double to) {
addRange(new Range(key, null, to));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, double)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedTo(double to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorBuilder addUnboundedFrom(String key, double from) {
addRange(new Range(key, from, null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedFrom(double from) {
return addUnboundedFrom(null, from);
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addRange(String key, DateTime from, DateTime to) {
addRange(new Range(key, convertDateTime(from), convertDateTime(to)));
return this;
}
private Double convertDateTime(DateTime dateTime) {
if (dateTime == null) {
return null;
} else {
return (double) dateTime.getMillis();
}
}
/**
* Same as {@link #addRange(String, DateTime, DateTime)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorBuilder addRange(DateTime from, DateTime to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorBuilder addUnboundedTo(String key, DateTime to) {
addRange(new Range(key, null, convertDateTime(to)));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, DateTime)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedTo(DateTime to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorBuilder addUnboundedFrom(String key, DateTime from) {
addRange(new Range(key, convertDateTime(from), null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, DateTime)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorBuilder addUnboundedFrom(DateTime from) {
return addUnboundedFrom(null, from);
}
@Override
protected DateRangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new DateRangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory);
}
@Override
protected DateRangeAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
int size = in.readVInt();
DateRangeAggregatorBuilder factory = new DateRangeAggregatorBuilder(name);
for (int i = 0; i < size; i++) {
factory.addRange(Range.PROTOTYPE.readFrom(in));
}
return factory;
}
}

View File

@ -19,236 +19,20 @@
package org.elasticsearch.search.aggregations.bucket.range.date;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.joda.time.DateTime;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
public class DateRangeAggregatorFactory extends AbstractFactory<DateRangeAggregatorFactory, RangeAggregator.Range> {
public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory<DateRangeAggregatorFactory, Range> {
public DateRangeAggregatorFactory(String name) {
super(name, InternalDateRange.FACTORY);
}
@Override
public String getWriteableName() {
return InternalDateRange.TYPE.name();
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addRange(String key, String from, String to) {
addRange(new Range(key, from, to));
return this;
}
/**
* Same as {@link #addRange(String, String, String)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorFactory addRange(String from, String to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addUnboundedTo(String key, String to) {
addRange(new Range(key, null, to));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, String)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedTo(String to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorFactory addUnboundedFrom(String key, String from) {
addRange(new Range(key, from, null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, String)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedFrom(String from) {
return addUnboundedFrom(null, from);
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addRange(String key, double from, double to) {
addRange(new Range(key, from, to));
return this;
}
/**
* Same as {@link #addRange(String, double, double)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorFactory addRange(double from, double to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addUnboundedTo(String key, double to) {
addRange(new Range(key, null, to));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, double)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedTo(double to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorFactory addUnboundedFrom(String key, double from) {
addRange(new Range(key, from, null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedFrom(double from) {
return addUnboundedFrom(null, from);
}
/**
* Add a new range to this aggregation.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the dates, inclusive
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addRange(String key, DateTime from, DateTime to) {
addRange(new Range(key, convertDateTime(from), convertDateTime(to)));
return this;
}
private Double convertDateTime(DateTime dateTime) {
if (dateTime == null) {
return null;
} else {
return (double) dateTime.getMillis();
}
}
/**
* Same as {@link #addRange(String, DateTime, DateTime)} but the key will be
* automatically generated based on <code>from</code> and <code>to</code>.
*/
public DateRangeAggregatorFactory addRange(DateTime from, DateTime to) {
return addRange(null, from, to);
}
/**
* Add a new range with no lower bound.
*
* @param key
* the key to use for this range in the response
* @param to
* the upper bound on the dates, exclusive
*/
public DateRangeAggregatorFactory addUnboundedTo(String key, DateTime to) {
addRange(new Range(key, null, convertDateTime(to)));
return this;
}
/**
* Same as {@link #addUnboundedTo(String, DateTime)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedTo(DateTime to) {
return addUnboundedTo(null, to);
}
/**
* Add a new range with no upper bound.
*
* @param key
* the key to use for this range in the response
* @param from
* the lower bound on the distances, inclusive
*/
public DateRangeAggregatorFactory addUnboundedFrom(String key, DateTime from) {
addRange(new Range(key, convertDateTime(from), null));
return this;
}
/**
* Same as {@link #addUnboundedFrom(String, DateTime)} but the key will be
* computed automatically.
*/
public DateRangeAggregatorFactory addUnboundedFrom(DateTime from) {
return addUnboundedFrom(null, from);
}
@Override
protected DateRangeAggregatorFactory createFactoryFromStream(String name, StreamInput in) throws IOException {
int size = in.readVInt();
DateRangeAggregatorFactory factory = new DateRangeAggregatorFactory(name);
for (int i = 0; i < size; i++) {
factory.addRange(Range.PROTOTYPE.readFrom(in));
}
return factory;
public DateRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, List<Range> ranges, boolean keyed,
Factory<?, ?> rangeFactory) {
super(name, type, config, ranges, keyed, rangeFactory);
}
}

View File

@ -19,7 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.range.date;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeParser;
@ -44,9 +44,9 @@ public class DateRangeParser extends RangeParser {
}
@Override
protected DateRangeAggregatorFactory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected DateRangeAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
DateRangeAggregatorFactory factory = new DateRangeAggregatorFactory(aggregationName);
DateRangeAggregatorBuilder factory = new DateRangeAggregatorBuilder(aggregationName);
List<Range> ranges = (List<Range>) otherOptions.get(RangeAggregator.RANGES_FIELD);
for (Range range : ranges) {
factory.addRange(range);
@ -59,7 +59,7 @@ public class DateRangeParser extends RangeParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new DateRangeAggregatorFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new DateRangeAggregatorBuilder(null);
}
}

View File

@ -18,12 +18,9 @@
*/
package org.elasticsearch.search.aggregations.bucket.range.geodistance;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -31,21 +28,17 @@ import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.GeoPointParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -111,10 +104,10 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
}
@Override
protected GeoDistanceFactory createFactory(
protected GeoDistanceAggregatorBuilder createFactory(
String aggregationName, ValuesSourceType valuesSourceType, ValueType targetValueType, Map<ParseField, Object> otherOptions) {
GeoPoint origin = (GeoPoint) otherOptions.get(ORIGIN_FIELD);
GeoDistanceFactory factory = new GeoDistanceFactory(aggregationName, origin);
GeoDistanceAggregatorBuilder factory = new GeoDistanceAggregatorBuilder(aggregationName, origin);
List<Range> ranges = (List<Range>) otherOptions.get(RangeAggregator.RANGES_FIELD);
for (Range range : ranges) {
factory.addRange(range);
@ -197,7 +190,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
return false;
}
public static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoDistanceFactory> {
public static class GeoDistanceAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoDistanceAggregatorBuilder> {
private final GeoPoint origin;
private final InternalRange.Factory rangeFactory;
@ -206,17 +199,17 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
private GeoDistance distanceType = GeoDistance.DEFAULT;
private boolean keyed = false;
public GeoDistanceFactory(String name, GeoPoint origin) {
public GeoDistanceAggregatorBuilder(String name, GeoPoint origin) {
this(name, origin, InternalGeoDistance.FACTORY);
}
private GeoDistanceFactory(String name, GeoPoint origin, InternalRange.Factory rangeFactory) {
private GeoDistanceAggregatorBuilder(String name, GeoPoint origin, InternalRange.Factory rangeFactory) {
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
this.origin = origin;
this.rangeFactory = rangeFactory;
}
public GeoDistanceFactory addRange(Range range) {
public GeoDistanceAggregatorBuilder addRange(Range range) {
ranges.add(range);
return this;
}
@ -231,7 +224,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* @param to
* the upper bound on the distances, exclusive
*/
public GeoDistanceFactory addRange(String key, double from, double to) {
public GeoDistanceAggregatorBuilder addRange(String key, double from, double to) {
ranges.add(new Range(key, from, to));
return this;
}
@ -241,7 +234,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* automatically generated based on <code>from</code> and
* <code>to</code>.
*/
public GeoDistanceFactory addRange(double from, double to) {
public GeoDistanceAggregatorBuilder addRange(double from, double to) {
return addRange(null, from, to);
}
@ -253,7 +246,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* @param to
* the upper bound on the distances, exclusive
*/
public GeoDistanceFactory addUnboundedTo(String key, double to) {
public GeoDistanceAggregatorBuilder addUnboundedTo(String key, double to) {
ranges.add(new Range(key, null, to));
return this;
}
@ -262,7 +255,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* Same as {@link #addUnboundedTo(String, double)} but the key will be
* computed automatically.
*/
public GeoDistanceFactory addUnboundedTo(double to) {
public GeoDistanceAggregatorBuilder addUnboundedTo(double to) {
return addUnboundedTo(null, to);
}
@ -274,7 +267,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* @param from
* the lower bound on the distances, inclusive
*/
public GeoDistanceFactory addUnboundedFrom(String key, double from) {
public GeoDistanceAggregatorBuilder addUnboundedFrom(String key, double from) {
addRange(new Range(key, from, null));
return this;
}
@ -283,7 +276,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
* computed automatically.
*/
public GeoDistanceFactory addUnboundedFrom(double from) {
public GeoDistanceAggregatorBuilder addUnboundedFrom(double from) {
return addUnboundedFrom(null, from);
}
@ -296,7 +289,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
return InternalGeoDistance.TYPE.name();
}
public GeoDistanceFactory unit(DistanceUnit unit) {
public GeoDistanceAggregatorBuilder unit(DistanceUnit unit) {
this.unit = unit;
return this;
}
@ -305,7 +298,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
return unit;
}
public GeoDistanceFactory distanceType(GeoDistance distanceType) {
public GeoDistanceAggregatorBuilder distanceType(GeoDistance distanceType) {
this.distanceType = distanceType;
return this;
}
@ -314,7 +307,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
return distanceType;
}
public GeoDistanceFactory keyed(boolean keyed) {
public GeoDistanceAggregatorBuilder keyed(boolean keyed) {
this.keyed = keyed;
return this;
}
@ -324,21 +317,9 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, pipelineAggregators,
metaData);
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext,
Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
return new RangeAggregator(name, factories, distanceSource, config.format(), rangeFactory, ranges, keyed, aggregationContext,
parent,
pipelineAggregators, metaData);
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config) {
return new GeoDistanceRangeAggregatorFactory(name, type, config, origin, ranges, unit, distanceType, keyed);
}
@Override
@ -352,11 +333,11 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
}
@Override
protected GeoDistanceFactory innerReadFrom(
protected GeoDistanceAggregatorBuilder innerReadFrom(
String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in) throws IOException {
GeoPoint origin = new GeoPoint(in.readDouble(), in.readDouble());
int size = in.readVInt();
GeoDistanceFactory factory = new GeoDistanceFactory(name, origin);
GeoDistanceAggregatorBuilder factory = new GeoDistanceAggregatorBuilder(name, origin);
for (int i = 0; i < size; i++) {
factory.addRange(Range.PROTOTYPE.readFrom(in));
}
@ -386,7 +367,7 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
@Override
protected boolean innerEquals(Object obj) {
GeoDistanceFactory other = (GeoDistanceFactory) obj;
GeoDistanceAggregatorBuilder other = (GeoDistanceAggregatorBuilder) obj;
return Objects.equals(origin, other.origin)
&& Objects.equals(ranges, other.ranges)
&& Objects.equals(keyed, other.keyed)
@ -394,50 +375,11 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
&& Objects.equals(unit, other.unit);
}
private static class DistanceSource extends ValuesSource.Numeric {
private final ValuesSource.GeoPoint source;
private final GeoDistance distanceType;
private final DistanceUnit unit;
private final org.elasticsearch.common.geo.GeoPoint origin;
public DistanceSource(ValuesSource.GeoPoint source, GeoDistance distanceType, org.elasticsearch.common.geo.GeoPoint origin, DistanceUnit unit) {
this.source = source;
// even if the geo points are unique, there's no guarantee the distances are
this.distanceType = distanceType;
this.unit = unit;
this.origin = origin;
}
@Override
public boolean isFloatingPoint() {
return true;
}
@Override
public SortedNumericDocValues longValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException();
}
@Override
public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
final MultiGeoPointValues geoValues = source.geoPointValues(ctx);
final FixedSourceDistance distance = distanceType.fixedSourceDistance(origin.getLat(), origin.getLon(), unit);
return GeoDistance.distanceValues(geoValues, distance);
}
@Override
public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException();
}
}
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new GeoDistanceFactory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new GeoDistanceAggregatorBuilder(null, null);
}
}

View File

@ -0,0 +1,125 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.range.geodistance;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser.Range;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
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.List;
import java.util.Map;
public class GeoDistanceRangeAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoDistanceRangeAggregatorFactory> {
private final InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> rangeFactory = InternalGeoDistance.FACTORY;
private final GeoPoint origin;
private final List<Range> ranges;
private final DistanceUnit unit;
private final GeoDistance distanceType;
private final boolean keyed;
public GeoDistanceRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, GeoPoint origin,
List<Range> ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed) {
super(name, type, config);
this.origin = origin;
this.ranges = ranges;
this.unit = unit;
this.distanceType = distanceType;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext,
Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
return new RangeAggregator(name, factories, distanceSource, config.format(), rangeFactory, ranges, keyed, aggregationContext,
parent,
pipelineAggregators, metaData);
}
private static class DistanceSource extends ValuesSource.Numeric {
private final ValuesSource.GeoPoint source;
private final GeoDistance distanceType;
private final DistanceUnit unit;
private final org.elasticsearch.common.geo.GeoPoint origin;
public DistanceSource(ValuesSource.GeoPoint source, GeoDistance distanceType, org.elasticsearch.common.geo.GeoPoint origin,
DistanceUnit unit) {
this.source = source;
// even if the geo points are unique, there's no guarantee the
// distances are
this.distanceType = distanceType;
this.unit = unit;
this.origin = origin;
}
@Override
public boolean isFloatingPoint() {
return true;
}
@Override
public SortedNumericDocValues longValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException();
}
@Override
public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
final MultiGeoPointValues geoValues = source.geoPointValues(ctx);
final FixedSourceDistance distance = distanceType.fixedSourceDistance(origin.getLat(), origin.getLon(), unit);
return GeoDistance.distanceValues(geoValues, distance);
}
@Override
public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
throw new UnsupportedOperationException();
}
}
}

View File

@ -27,14 +27,17 @@ import org.elasticsearch.common.network.Cidrs;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractFactory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractBuilder;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.io.IOException;
import java.util.Objects;
public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggregatorFactory, IPv4RangeAggregatorFactory.Range> {
public class IPv4RangeAggregatorBuilder extends AbstractBuilder<IPv4RangeAggregatorBuilder, IPv4RangeAggregatorBuilder.Range> {
public IPv4RangeAggregatorFactory(String name) {
public IPv4RangeAggregatorBuilder(String name) {
super(name, InternalIPv4Range.FACTORY);
}
@ -53,7 +56,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* @param to
* the upper bound on the distances, exclusive
*/
public IPv4RangeAggregatorFactory addRange(String key, String from, String to) {
public IPv4RangeAggregatorBuilder addRange(String key, String from, String to) {
addRange(new Range(key, from, to));
return this;
}
@ -62,7 +65,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addMaskRange(String, String)} but uses the mask itself as
* a key.
*/
public IPv4RangeAggregatorFactory addMaskRange(String key, String mask) {
public IPv4RangeAggregatorBuilder addMaskRange(String key, String mask) {
return addRange(new Range(key, mask));
}
@ -70,7 +73,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addMaskRange(String, String)} but uses the mask itself as
* a key.
*/
public IPv4RangeAggregatorFactory addMaskRange(String mask) {
public IPv4RangeAggregatorBuilder addMaskRange(String mask) {
return addRange(new Range(mask, mask));
}
@ -78,7 +81,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addRange(String, String, String)} but the key will be
* automatically generated.
*/
public IPv4RangeAggregatorFactory addRange(String from, String to) {
public IPv4RangeAggregatorBuilder addRange(String from, String to) {
return addRange(null, from, to);
}
@ -86,7 +89,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addRange(String, String, String)} but there will be no
* lower bound.
*/
public IPv4RangeAggregatorFactory addUnboundedTo(String key, String to) {
public IPv4RangeAggregatorBuilder addUnboundedTo(String key, String to) {
addRange(new Range(key, null, to));
return this;
}
@ -95,7 +98,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addUnboundedTo(String, String)} but the key will be
* generated automatically.
*/
public IPv4RangeAggregatorFactory addUnboundedTo(String to) {
public IPv4RangeAggregatorBuilder addUnboundedTo(String to) {
return addUnboundedTo(null, to);
}
@ -103,7 +106,7 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addRange(String, String, String)} but there will be no
* upper bound.
*/
public IPv4RangeAggregatorFactory addUnboundedFrom(String key, String from) {
public IPv4RangeAggregatorBuilder addUnboundedFrom(String key, String from) {
addRange(new Range(key, from, null));
return this;
}
@ -112,14 +115,19 @@ public class IPv4RangeAggregatorFactory extends AbstractFactory<IPv4RangeAggrega
* Same as {@link #addUnboundedFrom(String, String)} but the key will be
* generated automatically.
*/
public IPv4RangeAggregatorFactory addUnboundedFrom(String from) {
public IPv4RangeAggregatorBuilder addUnboundedFrom(String from) {
return addUnboundedFrom(null, from);
}
@Override
protected IPv4RangeAggregatorFactory createFactoryFromStream(String name, StreamInput in) throws IOException {
protected Ipv4RangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new Ipv4RangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory);
}
@Override
protected IPv4RangeAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
int size = in.readVInt();
IPv4RangeAggregatorFactory factory = new IPv4RangeAggregatorFactory(name);
IPv4RangeAggregatorBuilder factory = new IPv4RangeAggregatorBuilder(name);
for (int i = 0; i < size; i++) {
factory.addRange(Range.PROTOTYPE.readFrom(in));
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.range.ipv4;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeParser;
@ -48,16 +48,16 @@ public class IpRangeParser extends RangeParser {
@Override
protected Range parseRange(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
return IPv4RangeAggregatorFactory.Range.PROTOTYPE.fromXContent(parser, parseFieldMatcher);
return IPv4RangeAggregatorBuilder.Range.PROTOTYPE.fromXContent(parser, parseFieldMatcher);
}
@Override
protected IPv4RangeAggregatorFactory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected IPv4RangeAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
IPv4RangeAggregatorFactory factory = new IPv4RangeAggregatorFactory(aggregationName);
List<IPv4RangeAggregatorFactory.Range> ranges = (List<IPv4RangeAggregatorFactory.Range>) otherOptions
IPv4RangeAggregatorBuilder factory = new IPv4RangeAggregatorBuilder(aggregationName);
List<IPv4RangeAggregatorBuilder.Range> ranges = (List<IPv4RangeAggregatorBuilder.Range>) otherOptions
.get(RangeAggregator.RANGES_FIELD);
for (IPv4RangeAggregatorFactory.Range range : ranges) {
for (IPv4RangeAggregatorBuilder.Range range : ranges) {
factory.addRange(range);
}
Boolean keyed = (Boolean) otherOptions.get(RangeAggregator.KEYED_FIELD);
@ -68,8 +68,8 @@ public class IpRangeParser extends RangeParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new IPv4RangeAggregatorFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new IPv4RangeAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,39 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.range.ipv4;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.util.List;
public class Ipv4RangeAggregatorFactory
extends AbstractRangeAggregatorFactory<Ipv4RangeAggregatorFactory, IPv4RangeAggregatorBuilder.Range> {
public Ipv4RangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config,
List<IPv4RangeAggregatorBuilder.Range> ranges, boolean keyed,
Factory<?, ?> rangeFactory) {
super(name, type, config, ranges, keyed, rangeFactory);
}
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
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 org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, DiversifiedAggregatorFactory> {
private final int shardSize;
private final int maxDocsPerValue;
private final String executionHint;
public DiversifiedAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
String executionHint) {
super(name, type, config);
this.shardSize = shardSize;
this.maxDocsPerValue = maxDocsPerValue;
this.executionHint = executionHint;
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (valuesSource instanceof ValuesSource.Numeric) {
return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData,
(Numeric) valuesSource, maxDocsPerValue);
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, context.searchContext().parseFieldMatcher());
}
// In some cases using ordinals is just not supported: override
// it
if (execution == null) {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
if ((execution.needsGlobalOrdinals()) && (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals))) {
execution = ExecutionMode.MAP;
}
return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, pipelineAggregators,
metaData);
}
throw new AggregationExecutionException("Sampler aggregation cannot be applied to field [" + config.fieldContext().field()
+ "]. It can only be applied to numeric or string fields.");
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final UnmappedSampler aggregation = new UnmappedSampler(name, pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, factories, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
}

View File

@ -22,7 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -41,13 +41,13 @@ public class DiversifiedSamplerParser extends AnyValuesSourceParser {
@Override
public String type() {
return SamplerAggregator.DiversifiedFactory.TYPE.name();
return SamplerAggregator.DiversifiedAggregatorBuilder.TYPE.name();
}
@Override
protected SamplerAggregator.DiversifiedFactory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected SamplerAggregator.DiversifiedAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
SamplerAggregator.DiversifiedFactory factory = new SamplerAggregator.DiversifiedFactory(aggregationName);
SamplerAggregator.DiversifiedAggregatorBuilder factory = new SamplerAggregator.DiversifiedAggregatorBuilder(aggregationName);
Integer shardSize = (Integer) otherOptions.get(SamplerAggregator.SHARD_SIZE_FIELD);
if (shardSize != null) {
factory.shardSize(shardSize);
@ -87,8 +87,8 @@ public class DiversifiedSamplerParser extends AnyValuesSourceParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new SamplerAggregator.DiversifiedFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new SamplerAggregator.DiversifiedAggregatorBuilder(null);
}
}

View File

@ -29,10 +29,10 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
@ -40,8 +40,9 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -190,20 +191,20 @@ public class SamplerAggregator extends SingleBucketAggregator {
return new InternalSampler(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
}
public static class Factory extends AggregatorFactory<Factory> {
public static class SamplerAggregatorBuilder extends AggregatorBuilder<SamplerAggregatorBuilder> {
public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
private int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
public Factory(String name) {
public SamplerAggregatorBuilder(String name) {
super(name, InternalSampler.TYPE);
}
/**
* Set the max num docs to be returned from each shard.
*/
public Factory shardSize(int shardSize) {
public SamplerAggregatorBuilder shardSize(int shardSize) {
this.shardSize = shardSize;
return this;
}
@ -216,9 +217,8 @@ public class SamplerAggregator extends SingleBucketAggregator {
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new SamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
protected AggregatorFactory<?> doBuild(AggregationContext context) {
return new SamplerAggregatorFactory(name, type, shardSize);
}
@Override
@ -230,8 +230,8 @@ public class SamplerAggregator extends SingleBucketAggregator {
}
@Override
protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory = new Factory(name);
protected SamplerAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
SamplerAggregatorBuilder factory = new SamplerAggregatorBuilder(name);
factory.shardSize = in.readVInt();
return factory;
}
@ -248,30 +248,30 @@ public class SamplerAggregator extends SingleBucketAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
SamplerAggregatorBuilder other = (SamplerAggregatorBuilder) obj;
return Objects.equals(shardSize, other.shardSize);
}
}
public static class DiversifiedFactory extends ValuesSourceAggregatorFactory<ValuesSource, DiversifiedFactory> {
public static class DiversifiedAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, DiversifiedAggregatorBuilder> {
public static final Type TYPE = new Type("diversified_sampler");
public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
private int shardSize = Factory.DEFAULT_SHARD_SAMPLE_SIZE;
private int shardSize = SamplerAggregatorBuilder.DEFAULT_SHARD_SAMPLE_SIZE;
private int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT;
private String executionHint = null;
public DiversifiedFactory(String name) {
public DiversifiedAggregatorBuilder(String name) {
super(name, TYPE, ValuesSourceType.ANY, null);
}
/**
* Set the max num docs to be returned from each shard.
*/
public DiversifiedFactory shardSize(int shardSize) {
public DiversifiedAggregatorBuilder shardSize(int shardSize) {
this.shardSize = shardSize;
return this;
}
@ -286,7 +286,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
/**
* Set the max num docs to be returned per value.
*/
public DiversifiedFactory maxDocsPerValue(int maxDocsPerValue) {
public DiversifiedAggregatorBuilder maxDocsPerValue(int maxDocsPerValue) {
this.maxDocsPerValue = maxDocsPerValue;
return this;
}
@ -301,7 +301,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
/**
* Set the execution hint.
*/
public DiversifiedFactory executionHint(String executionHint) {
public DiversifiedAggregatorBuilder executionHint(String executionHint) {
this.executionHint = executionHint;
return this;
}
@ -314,49 +314,9 @@ public class SamplerAggregator extends SingleBucketAggregator {
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (valuesSource instanceof ValuesSource.Numeric) {
return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData,
(Numeric) valuesSource, maxDocsPerValue);
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, context.searchContext().parseFieldMatcher());
}
// In some cases using ordinals is just not supported: override
// it
if(execution==null){
execution = ExecutionMode.GLOBAL_ORDINALS;
}
if ((execution.needsGlobalOrdinals()) && (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals))) {
execution = ExecutionMode.MAP;
}
return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, pipelineAggregators,
metaData);
}
throw new AggregationExecutionException("Sampler aggregation cannot be applied to field [" + config.fieldContext().field() +
"]. It can only be applied to numeric or string fields.");
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final UnmappedSampler aggregation = new UnmappedSampler(name, pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, factories, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource> config) {
return new DiversifiedAggregatorFactory(name, TYPE, config, shardSize, maxDocsPerValue, executionHint);
}
@Override
@ -370,9 +330,9 @@ public class SamplerAggregator extends SingleBucketAggregator {
}
@Override
protected DiversifiedFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected DiversifiedAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
DiversifiedFactory factory = new DiversifiedFactory(name);
DiversifiedAggregatorBuilder factory = new DiversifiedAggregatorBuilder(name);
factory.shardSize = in.readVInt();
factory.maxDocsPerValue = in.readVInt();
factory.executionHint = in.readOptionalString();
@ -393,7 +353,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
@Override
protected boolean innerEquals(Object obj) {
DiversifiedFactory other = (DiversifiedFactory) obj;
DiversifiedAggregatorBuilder other = (DiversifiedAggregatorBuilder) obj;
return Objects.equals(shardSize, other.shardSize)
&& Objects.equals(maxDocsPerValue, other.maxDocsPerValue)
&& Objects.equals(executionHint, other.executionHint);

View File

@ -0,0 +1,47 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class SamplerAggregatorFactory extends AggregatorFactory<SamplerAggregatorFactory> {
private final int shardSize;
public SamplerAggregatorFactory(String name, Type type, int shardSize) {
super(name, type);
this.shardSize = shardSize;
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new SamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
}
}

View File

@ -23,7 +23,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
@ -38,7 +38,7 @@ public class SamplerParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
XContentParser.Token token;
String currentFieldName = null;
@ -60,7 +60,7 @@ public class SamplerParser implements Aggregator.Parser {
}
}
SamplerAggregator.Factory factory = new SamplerAggregator.Factory(aggregationName);
SamplerAggregator.SamplerAggregatorBuilder factory = new SamplerAggregator.SamplerAggregatorBuilder(aggregationName);
if (shardSize != null) {
factory.shardSize(shardSize);
}
@ -68,8 +68,8 @@ public class SamplerParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new SamplerAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new SamplerAggregator.SamplerAggregatorBuilder(null);
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -47,15 +48,17 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
protected long numCollectedDocs;
protected final SignificantTermsAggregatorFactory termsAggFactory;
private final SignificanceHeuristic significanceHeuristic;
public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories,
ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds,
IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent,
SubAggCollectionMode.DEPTH_FIRST, false, pipelineAggregators, metaData);
this.significanceHeuristic = significanceHeuristic;
this.termsAggFactory = termsAggFactory;
}
@ -117,7 +120,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
// that are for this shard only
// Back at the central reducer these properties will be updated with
// global stats
spare.updateScore(termsAggFactory.getSignificanceHeuristic());
spare.updateScore(significanceHeuristic);
spare = (SignificantStringTerms.Bucket) ordered.insertWithOverflow(spare);
}
@ -131,7 +134,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
}
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), pipelineAggregators(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic, Arrays.asList(list), pipelineAggregators(),
metaData());
}
@ -142,7 +145,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic,
Collections.<InternalSignificantTerms.Bucket> emptyList(), pipelineAggregators(), metaData());
}
@ -157,10 +160,11 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory,
pipelineAggregators, metaData);
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, significanceHeuristic,
termsAggFactory, pipelineAggregators, metaData);
bucketOrds = new LongHash(1, aggregationContext.bigArrays());
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -46,16 +47,19 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource,
ValueFormat format, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude,
SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
IncludeExclude.LongFilter includeExclude,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, null, bucketCountThresholds, aggregationContext, parent,
SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, pipelineAggregators, metaData);
this.significanceHeuristic = significanceHeuristic;
this.termsAggFactory = termsAggFactory;
}
protected long numCollectedDocs;
private final SignificantTermsAggregatorFactory termsAggFactory;
private final SignificanceHeuristic significanceHeuristic;
@Override
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
@ -95,7 +99,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
spare.supersetSize = supersetSize;
// During shard-local down-selection we use subset/superset stats that are for this shard only
// Back at the central reducer these properties will be updated with global stats
spare.updateScore(termsAggFactory.getSignificanceHeuristic());
spare.updateScore(significanceHeuristic);
spare.bucketOrd = i;
spare = (SignificantLongTerms.Bucket) ordered.insertWithOverflow(spare);
@ -108,7 +112,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
list[i] = bucket;
}
return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), pipelineAggregators(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic, Arrays.asList(list), pipelineAggregators(),
metaData());
}
@ -119,7 +123,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic,
Collections.<InternalSignificantTerms.Bucket> emptyList(), pipelineAggregators(), metaData());
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.terms.StringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -46,16 +47,16 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
protected long numCollectedDocs;
protected final SignificantTermsAggregatorFactory termsAggFactory;
private final SignificanceHeuristic significanceHeuristic;
public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
BucketCountThresholds bucketCountThresholds,
IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
BucketCountThresholds bucketCountThresholds, IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext,
Aggregator parent, SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent,
SubAggCollectionMode.DEPTH_FIRST, false, pipelineAggregators, metaData);
this.significanceHeuristic = significanceHeuristic;
this.termsAggFactory = termsAggFactory;
}
@ -100,7 +101,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
// that are for this shard only
// Back at the central reducer these properties will be updated with
// global stats
spare.updateScore(termsAggFactory.getSignificanceHeuristic());
spare.updateScore(significanceHeuristic);
spare.bucketOrd = i;
spare = (SignificantStringTerms.Bucket) ordered.insertWithOverflow(spare);
@ -116,7 +117,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
}
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), pipelineAggregators(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic, Arrays.asList(list), pipelineAggregators(),
metaData());
}
@ -127,7 +128,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(),
bucketCountThresholds.getMinDocCount(), significanceHeuristic,
Collections.<InternalSignificantTerms.Bucket> emptyList(), pipelineAggregators(), metaData());
}

View File

@ -0,0 +1,239 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.significant;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.index.FilterableTermsEnum;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.JLHScore;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicStreams;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.Objects;
/**
*
*/
public class SignificantTermsAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, SignificantTermsAggregatorBuilder> {
static final ParseField BACKGROUND_FILTER = new ParseField("background_filter");
static final ParseField HEURISTIC = new ParseField("significance_heuristic");
static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(
3, 0, 10, -1);
private IncludeExclude includeExclude = null;
private String executionHint = null;
private String indexedFieldName;
private MappedFieldType fieldType;
private FilterableTermsEnum termsEnum;
private int numberOfAggregatorsCreated = 0;
private QueryBuilder<?> filterBuilder = null;
private TermsAggregator.BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(DEFAULT_BUCKET_COUNT_THRESHOLDS);
private SignificanceHeuristic significanceHeuristic = JLHScore.PROTOTYPE;
protected TermsAggregator.BucketCountThresholds getBucketCountThresholds() {
return new TermsAggregator.BucketCountThresholds(bucketCountThresholds);
}
public SignificantTermsAggregatorBuilder(String name, ValueType valueType) {
super(name, SignificantStringTerms.TYPE, ValuesSourceType.ANY, valueType);
}
public TermsAggregator.BucketCountThresholds bucketCountThresholds() {
return bucketCountThresholds;
}
public SignificantTermsAggregatorBuilder bucketCountThresholds(TermsAggregator.BucketCountThresholds bucketCountThresholds) {
this.bucketCountThresholds = bucketCountThresholds;
return this;
}
/**
* Sets the size - indicating how many term buckets should be returned
* (defaults to 10)
*/
public SignificantTermsAggregatorBuilder size(int size) {
bucketCountThresholds.setRequiredSize(size);
return this;
}
/**
* Sets the shard_size - indicating the number of term buckets each shard
* will return to the coordinating node (the node that coordinates the
* search execution). The higher the shard size is, the more accurate the
* results are.
*/
public SignificantTermsAggregatorBuilder shardSize(int shardSize) {
bucketCountThresholds.setShardSize(shardSize);
return this;
}
/**
* Set the minimum document count terms should have in order to appear in
* the response.
*/
public SignificantTermsAggregatorBuilder minDocCount(long minDocCount) {
bucketCountThresholds.setMinDocCount(minDocCount);
return this;
}
/**
* Set the minimum document count terms should have on the shard in order to
* appear in the response.
*/
public SignificantTermsAggregatorBuilder shardMinDocCount(long shardMinDocCount) {
bucketCountThresholds.setShardMinDocCount(shardMinDocCount);
return this;
}
/**
* Expert: sets an execution hint to the aggregation.
*/
public SignificantTermsAggregatorBuilder executionHint(String executionHint) {
this.executionHint = executionHint;
return this;
}
/**
* Expert: gets an execution hint to the aggregation.
*/
public String executionHint() {
return executionHint;
}
public SignificantTermsAggregatorBuilder backgroundFilter(QueryBuilder<?> filterBuilder) {
this.filterBuilder = filterBuilder;
return this;
}
public QueryBuilder<?> backgroundFilter() {
return filterBuilder;
}
/**
* Set terms to include and exclude from the aggregation results
*/
public SignificantTermsAggregatorBuilder includeExclude(IncludeExclude includeExclude) {
this.includeExclude = includeExclude;
return this;
}
/**
* Get terms to include and exclude from the aggregation results
*/
public IncludeExclude includeExclude() {
return includeExclude;
}
public SignificantTermsAggregatorBuilder significanceHeuristic(SignificanceHeuristic significanceHeuristic) {
this.significanceHeuristic = significanceHeuristic;
return this;
}
public SignificanceHeuristic significanceHeuristic() {
return significanceHeuristic;
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource> config) {
return new SignificantTermsAggregatorFactory(name, type, config, includeExclude, executionHint, filterBuilder,
bucketCountThresholds, significanceHeuristic, context);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
bucketCountThresholds.toXContent(builder, params);
if (executionHint != null) {
builder.field(TermsAggregatorBuilder.EXECUTION_HINT_FIELD_NAME.getPreferredName(), executionHint);
}
if (filterBuilder != null) {
builder.field(BACKGROUND_FILTER.getPreferredName(), filterBuilder);
}
if (includeExclude != null) {
includeExclude.toXContent(builder, params);
}
significanceHeuristic.toXContent(builder, params);
return builder;
}
@Override
protected SignificantTermsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
SignificantTermsAggregatorBuilder factory = new SignificantTermsAggregatorBuilder(name, targetValueType);
factory.bucketCountThresholds = BucketCountThresholds.readFromStream(in);
factory.executionHint = in.readOptionalString();
if (in.readBoolean()) {
factory.filterBuilder = in.readQuery();
}
if (in.readBoolean()) {
factory.includeExclude = IncludeExclude.readFromStream(in);
}
factory.significanceHeuristic = SignificanceHeuristicStreams.read(in);
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
bucketCountThresholds.writeTo(out);
out.writeOptionalString(executionHint);
boolean hasfilterBuilder = filterBuilder != null;
out.writeBoolean(hasfilterBuilder);
if (hasfilterBuilder) {
out.writeQuery(filterBuilder);
}
boolean hasIncExc = includeExclude != null;
out.writeBoolean(hasIncExc);
if (hasIncExc) {
includeExclude.writeTo(out);
}
SignificanceHeuristicStreams.writeTo(significanceHeuristic, out);
}
@Override
protected int innerHashCode() {
return Objects.hash(bucketCountThresholds, executionHint, filterBuilder, includeExclude, significanceHeuristic);
}
@Override
protected boolean innerEquals(Object obj) {
SignificantTermsAggregatorBuilder other = (SignificantTermsAggregatorBuilder) obj;
return Objects.equals(bucketCountThresholds, other.bucketCountThresholds)
&& Objects.equals(executionHint, other.executionHint)
&& Objects.equals(filterBuilder, other.filterBuilder)
&& Objects.equals(includeExclude, other.includeExclude)
&& Objects.equals(significanceHeuristic, other.significanceHeuristic);
}
}

View File

@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.index.IndexReader;
@ -26,12 +27,9 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lucene.index.FilterableTermsEnum;
import org.elasticsearch.common.lucene.index.FreqTermsEnum;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
@ -39,232 +37,48 @@ 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.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.JLHScore;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicStreams;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
*
*/
public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, SignificantTermsAggregatorFactory>
implements Releasable {
static final ParseField BACKGROUND_FILTER = new ParseField("background_filter");
static final ParseField HEURISTIC = new ParseField("significance_heuristic");
static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(
3, 0, 10, -1);
public SignificanceHeuristic getSignificanceHeuristic() {
return significanceHeuristic;
}
public enum ExecutionMode {
MAP(new ParseField("map")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter();
return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, filter,
aggregationContext, parent, termsAggregatorFactory, pipelineAggregators, metaData);
}
},
GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsSignificantTermsAggregator(name, factories,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext,
parent, termsAggregatorFactory, pipelineAggregators, metaData);
}
},
GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent,
termsAggregatorFactory, pipelineAggregators, metaData);
}
};
public static ExecutionMode fromString(String value, ParseFieldMatcher parseFieldMatcher) {
for (ExecutionMode mode : values()) {
if (parseFieldMatcher.match(value, mode.parseField)) {
return mode;
}
}
throw new IllegalArgumentException("Unknown `execution_hint`: [" + value + "], expected any of " + values());
}
private final ParseField parseField;
ExecutionMode(ParseField parseField) {
this.parseField = parseField;
}
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException;
@Override
public String toString() {
return parseField.getPreferredName();
}
}
private IncludeExclude includeExclude = null;
private String executionHint = null;
private final IncludeExclude includeExclude;
private final String executionHint;
private String indexedFieldName;
private MappedFieldType fieldType;
private FilterableTermsEnum termsEnum;
private int numberOfAggregatorsCreated = 0;
private QueryBuilder<?> filterBuilder = null;
private TermsAggregator.BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(DEFAULT_BUCKET_COUNT_THRESHOLDS);
private SignificanceHeuristic significanceHeuristic = JLHScore.PROTOTYPE;
private int numberOfAggregatorsCreated;
private final QueryBuilder<?> filterBuilder;
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private final SignificanceHeuristic significanceHeuristic;
protected TermsAggregator.BucketCountThresholds getBucketCountThresholds() {
return new TermsAggregator.BucketCountThresholds(bucketCountThresholds);
}
public SignificantTermsAggregatorFactory(String name, ValueType valueType) {
super(name, SignificantStringTerms.TYPE, ValuesSourceType.ANY, valueType);
}
public TermsAggregator.BucketCountThresholds bucketCountThresholds() {
return bucketCountThresholds;
}
public SignificantTermsAggregatorFactory bucketCountThresholds(TermsAggregator.BucketCountThresholds bucketCountThresholds) {
this.bucketCountThresholds = bucketCountThresholds;
return this;
}
/**
* Sets the size - indicating how many term buckets should be returned
* (defaults to 10)
*/
public SignificantTermsAggregatorFactory size(int size) {
bucketCountThresholds.setRequiredSize(size);
return this;
}
/**
* Sets the shard_size - indicating the number of term buckets each shard
* will return to the coordinating node (the node that coordinates the
* search execution). The higher the shard size is, the more accurate the
* results are.
*/
public SignificantTermsAggregatorFactory shardSize(int shardSize) {
bucketCountThresholds.setShardSize(shardSize);
return this;
}
/**
* Set the minimum document count terms should have in order to appear in
* the response.
*/
public SignificantTermsAggregatorFactory minDocCount(long minDocCount) {
bucketCountThresholds.setMinDocCount(minDocCount);
return this;
}
/**
* Set the minimum document count terms should have on the shard in order to
* appear in the response.
*/
public SignificantTermsAggregatorFactory shardMinDocCount(long shardMinDocCount) {
bucketCountThresholds.setShardMinDocCount(shardMinDocCount);
return this;
}
/**
* Expert: sets an execution hint to the aggregation.
*/
public SignificantTermsAggregatorFactory executionHint(String executionHint) {
this.executionHint = executionHint;
return this;
}
/**
* Expert: gets an execution hint to the aggregation.
*/
public String executionHint() {
return executionHint;
}
public SignificantTermsAggregatorFactory backgroundFilter(QueryBuilder<?> filterBuilder) {
this.filterBuilder = filterBuilder;
return this;
}
public QueryBuilder<?> backgroundFilter() {
return filterBuilder;
}
/**
* Set terms to include and exclude from the aggregation results
*/
public SignificantTermsAggregatorFactory includeExclude(IncludeExclude includeExclude) {
public SignificantTermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, IncludeExclude includeExclude,
String executionHint, QueryBuilder<?> filterBuilder, TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic, AggregationContext context) {
super(name, type, config);
this.includeExclude = includeExclude;
return this;
}
/**
* Get terms to include and exclude from the aggregation results
*/
public IncludeExclude includeExclude() {
return includeExclude;
}
public SignificantTermsAggregatorFactory significanceHeuristic(SignificanceHeuristic significanceHeuristic) {
this.executionHint = executionHint;
this.filterBuilder = filterBuilder;
this.bucketCountThresholds = bucketCountThresholds;
this.significanceHeuristic = significanceHeuristic;
return this;
}
public SignificanceHeuristic significanceHeuristic() {
return significanceHeuristic;
}
@Override
public void doInit(AggregationContext context) {
super.doInit(context);
this.significanceHeuristic.initialize(context.searchContext());
setFieldInfo();
significanceHeuristic.initialize(context.searchContext());
}
private void setFieldInfo() {
@ -274,89 +88,14 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
numberOfAggregatorsCreated++;
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
if (bucketCountThresholds.getShardSize() == DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
//The user has not made a shardSize selection .
//Use default heuristic to avoid any wrong-ranking caused by distributed counting
//but request double the usual amount.
//We typically need more than the number of "top" terms requested by other aggregations
//as the significance algorithm is in less of a position to down-select at shard-level -
//some of the things we want to find have only one occurrence on each shard and as
// such are impossible to differentiate from non-significant terms at that early stage.
bucketCountThresholds.setShardSize(2 * BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
aggregationContext.searchContext().numberOfShards()));
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, aggregationContext.searchContext().parseFieldMatcher());
}
if (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals)) {
execution = ExecutionMode.MAP;
}
if (execution == null) {
if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
}
assert execution != null;
return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this,
pipelineAggregators, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style include/exclude " +
"settings as they can only be applied to string fields. Use an array of numeric values for include/exclude clauses used to filter numeric fields");
}
if (valuesSource instanceof ValuesSource.Numeric) {
if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) {
throw new UnsupportedOperationException("No support for examining floating point numerics");
}
IncludeExclude.LongFilter longFilter = null;
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
bucketCountThresholds, aggregationContext, parent, this, longFilter, pipelineAggregators, metaData);
}
throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field [" + config.fieldContext().field() +
"]. It can only be applied to numeric or string fields.");
}
/**
* Creates the TermsEnum (if not already created) and must be called before any calls to getBackgroundFrequency
* @param context The aggregation context
* @return The number of documents in the index (after an optional filter might have been applied)
* Creates the TermsEnum (if not already created) and must be called before
* any calls to getBackgroundFrequency
*
* @param context
* The aggregation context
* @return The number of documents in the index (after an optional filter
* might have been applied)
*/
public long prepareBackground(AggregationContext context) {
if (termsEnum != null) {
@ -378,8 +117,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
// Setup a termsEnum for sole use by one aggregator
termsEnum = new FilterableTermsEnum(reader, indexedFieldName, PostingsEnum.NONE, filter);
} else {
// When we have > 1 agg we have possibility of duplicate term frequency lookups
// and so use a TermsEnum that caches results of all term lookups
// When we have > 1 agg we have possibility of duplicate term
// frequency lookups
// and so use a TermsEnum that caches results of all term
// lookups
termsEnum = new FreqTermsEnum(reader, indexedFieldName, true, false, filter, searchContext.bigArrays());
}
} catch (IOException e) {
@ -389,7 +130,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
public long getBackgroundFrequency(BytesRef termBytes) {
assert termsEnum != null; // having failed to find a field in the index we don't expect any calls for frequencies
assert termsEnum != null; // having failed to find a field in the index
// we don't expect any calls for frequencies
long result = 0;
try {
if (termsEnum.seekExact(termBytes)) {
@ -401,12 +143,169 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
return result;
}
public long getBackgroundFrequency(long term) {
BytesRef indexedVal = fieldType.indexedValueForSearch(term);
return getBackgroundFrequency(indexedVal);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
numberOfAggregatorsCreated++;
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
if (bucketCountThresholds.getShardSize() == SignificantTermsAggregatorBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
// The user has not made a shardSize selection .
// Use default heuristic to avoid any wrong-ranking caused by
// distributed counting
// but request double the usual amount.
// We typically need more than the number of "top" terms requested
// by other aggregations
// as the significance algorithm is in less of a position to
// down-select at shard-level -
// some of the things we want to find have only one occurrence on
// each shard and as
// such are impossible to differentiate from non-significant terms
// at that early stage.
bucketCountThresholds.setShardSize(2 * BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
aggregationContext.searchContext().numberOfShards()));
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, aggregationContext.searchContext().parseFieldMatcher());
}
if (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals)) {
execution = ExecutionMode.MAP;
}
if (execution == null) {
if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
}
assert execution != null;
return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent,
significanceHeuristic, this, pipelineAggregators, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style include/exclude "
+ "settings as they can only be applied to string fields. Use an array of numeric values for include/exclude clauses used to filter numeric fields");
}
if (valuesSource instanceof ValuesSource.Numeric) {
if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) {
throw new UnsupportedOperationException("No support for examining floating point numerics");
}
IncludeExclude.LongFilter longFilter = null;
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
bucketCountThresholds, aggregationContext, parent, significanceHeuristic, this, longFilter, pipelineAggregators,
metaData);
}
throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field ["
+ config.fieldContext().field() + "]. It can only be applied to numeric or string fields.");
}
public enum ExecutionMode {
MAP(new ParseField("map")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter();
return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, filter,
aggregationContext, parent, significanceHeuristic, termsAggregatorFactory, pipelineAggregators, metaData);
}
},
GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsSignificantTermsAggregator(name, factories,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent,
significanceHeuristic, termsAggregatorFactory, pipelineAggregators, metaData);
}
},
GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent,
significanceHeuristic, termsAggregatorFactory, pipelineAggregators, metaData);
}
};
public static ExecutionMode fromString(String value, ParseFieldMatcher parseFieldMatcher) {
for (ExecutionMode mode : values()) {
if (parseFieldMatcher.match(value, mode.parseField)) {
return mode;
}
}
throw new IllegalArgumentException("Unknown `execution_hint`: [" + value + "], expected any of " + values());
}
private final ParseField parseField;
ExecutionMode(ParseField parseField) {
this.parseField = parseField;
}
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
@Override
public String toString() {
return parseField.getPreferredName();
}
}
@Override
public void close() {
try {
@ -417,68 +316,4 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
termsEnum = null;
}
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
bucketCountThresholds.toXContent(builder, params);
if (executionHint != null) {
builder.field(TermsAggregatorFactory.EXECUTION_HINT_FIELD_NAME.getPreferredName(), executionHint);
}
if (filterBuilder != null) {
builder.field(BACKGROUND_FILTER.getPreferredName(), filterBuilder);
}
if (includeExclude != null) {
includeExclude.toXContent(builder, params);
}
significanceHeuristic.toXContent(builder, params);
return builder;
}
@Override
protected SignificantTermsAggregatorFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
SignificantTermsAggregatorFactory factory = new SignificantTermsAggregatorFactory(name, targetValueType);
factory.bucketCountThresholds = BucketCountThresholds.readFromStream(in);
factory.executionHint = in.readOptionalString();
if (in.readBoolean()) {
factory.filterBuilder = in.readQuery();
}
if (in.readBoolean()) {
factory.includeExclude = IncludeExclude.readFromStream(in);
}
factory.significanceHeuristic = SignificanceHeuristicStreams.read(in);
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
bucketCountThresholds.writeTo(out);
out.writeOptionalString(executionHint);
boolean hasfilterBuilder = filterBuilder != null;
out.writeBoolean(hasfilterBuilder);
if (hasfilterBuilder) {
out.writeQuery(filterBuilder);
}
boolean hasIncExc = includeExclude != null;
out.writeBoolean(hasIncExc);
if (hasIncExc) {
includeExclude.writeTo(out);
}
SignificanceHeuristicStreams.writeTo(significanceHeuristic, out);
}
@Override
protected int innerHashCode() {
return Objects.hash(bucketCountThresholds, executionHint, filterBuilder, includeExclude, significanceHeuristic);
}
@Override
protected boolean innerEquals(Object obj) {
SignificantTermsAggregatorFactory other = (SignificantTermsAggregatorFactory) obj;
return Objects.equals(bucketCountThresholds, other.bucketCountThresholds)
&& Objects.equals(executionHint, other.executionHint)
&& Objects.equals(filterBuilder, other.filterBuilder)
&& Objects.equals(includeExclude, other.includeExclude)
&& Objects.equals(significanceHeuristic, other.significanceHeuristic);
}
}

View File

@ -27,7 +27,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParserMapper;
@ -61,10 +61,10 @@ public class SignificantTermsParser extends AbstractTermsParser {
}
@Override
protected SignificantTermsAggregatorFactory doCreateFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected SignificantTermsAggregatorBuilder doCreateFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode collectMode, String executionHint,
IncludeExclude incExc, Map<ParseField, Object> otherOptions) {
SignificantTermsAggregatorFactory factory = new SignificantTermsAggregatorFactory(aggregationName, targetValueType);
SignificantTermsAggregatorBuilder factory = new SignificantTermsAggregatorBuilder(aggregationName, targetValueType);
if (bucketCountThresholds != null) {
factory.bucketCountThresholds(bucketCountThresholds);
}
@ -74,11 +74,11 @@ public class SignificantTermsParser extends AbstractTermsParser {
if (incExc != null) {
factory.includeExclude(incExc);
}
QueryBuilder<?> backgroundFilter = (QueryBuilder<?>) otherOptions.get(SignificantTermsAggregatorFactory.BACKGROUND_FILTER);
QueryBuilder<?> backgroundFilter = (QueryBuilder<?>) otherOptions.get(SignificantTermsAggregatorBuilder.BACKGROUND_FILTER);
if (backgroundFilter != null) {
factory.backgroundFilter(backgroundFilter);
}
SignificanceHeuristic significanceHeuristic = (SignificanceHeuristic) otherOptions.get(SignificantTermsAggregatorFactory.HEURISTIC);
SignificanceHeuristic significanceHeuristic = (SignificanceHeuristic) otherOptions.get(SignificantTermsAggregatorBuilder.HEURISTIC);
if (significanceHeuristic != null) {
factory.significanceHeuristic(significanceHeuristic);
}
@ -92,14 +92,14 @@ public class SignificantTermsParser extends AbstractTermsParser {
SignificanceHeuristicParser significanceHeuristicParser = significanceHeuristicParserMapper.get(currentFieldName);
if (significanceHeuristicParser != null) {
SignificanceHeuristic significanceHeuristic = significanceHeuristicParser.parse(parser, parseFieldMatcher);
otherOptions.put(SignificantTermsAggregatorFactory.HEURISTIC, significanceHeuristic);
otherOptions.put(SignificantTermsAggregatorBuilder.HEURISTIC, significanceHeuristic);
return true;
} else if (parseFieldMatcher.match(currentFieldName, SignificantTermsAggregatorFactory.BACKGROUND_FILTER)) {
} else if (parseFieldMatcher.match(currentFieldName, SignificantTermsAggregatorBuilder.BACKGROUND_FILTER)) {
QueryParseContext queryParseContext = new QueryParseContext(queriesRegistry);
queryParseContext.reset(parser);
queryParseContext.parseFieldMatcher(parseFieldMatcher);
QueryBuilder<?> filter = queryParseContext.parseInnerQueryBuilder();
otherOptions.put(SignificantTermsAggregatorFactory.BACKGROUND_FILTER, filter);
otherOptions.put(SignificantTermsAggregatorBuilder.BACKGROUND_FILTER, filter);
return true;
}
}
@ -107,12 +107,12 @@ public class SignificantTermsParser extends AbstractTermsParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new SignificantTermsAggregatorFactory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new SignificantTermsAggregatorBuilder(null, null);
}
@Override
protected BucketCountThresholds getDefaultBucketCountThresholds() {
return new TermsAggregator.BucketCountThresholds(SignificantTermsAggregatorFactory.DEFAULT_BUCKET_COUNT_THRESHOLDS);
return new TermsAggregator.BucketCountThresholds(SignificantTermsAggregatorBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS);
}
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -50,7 +50,7 @@ public abstract class AbstractTermsParser extends AnyValuesSourceParser {
}
@Override
protected final ValuesSourceAggregatorFactory<ValuesSource, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected final ValuesSourceAggregatorBuilder<ValuesSource, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
BucketCountThresholds bucketCountThresholds = getDefaultBucketCountThresholds();
Integer requiredSize = (Integer) otherOptions.get(REQUIRED_SIZE_FIELD_NAME);
@ -77,7 +77,7 @@ public abstract class AbstractTermsParser extends AnyValuesSourceParser {
otherOptions);
}
protected abstract ValuesSourceAggregatorFactory<ValuesSource, ?> doCreateFactory(String aggregationName,
protected abstract ValuesSourceAggregatorBuilder<ValuesSource, ?> doCreateFactory(String aggregationName,
ValuesSourceType valuesSourceType,
ValueType targetValueType, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode collectMode, String executionHint,
IncludeExclude incExc, Map<ParseField, Object> otherOptions);

View File

@ -127,10 +127,10 @@ public abstract class TermsAggregator extends BucketsAggregator {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field(TermsAggregatorFactory.REQUIRED_SIZE_FIELD_NAME.getPreferredName(), requiredSize);
builder.field(TermsAggregatorFactory.SHARD_SIZE_FIELD_NAME.getPreferredName(), shardSize);
builder.field(TermsAggregatorFactory.MIN_DOC_COUNT_FIELD_NAME.getPreferredName(), minDocCount);
builder.field(TermsAggregatorFactory.SHARD_MIN_DOC_COUNT_FIELD_NAME.getPreferredName(), shardMinDocCount);
builder.field(TermsAggregatorBuilder.REQUIRED_SIZE_FIELD_NAME.getPreferredName(), requiredSize);
builder.field(TermsAggregatorBuilder.SHARD_SIZE_FIELD_NAME.getPreferredName(), shardSize);
builder.field(TermsAggregatorBuilder.MIN_DOC_COUNT_FIELD_NAME.getPreferredName(), minDocCount);
builder.field(TermsAggregatorBuilder.SHARD_MIN_DOC_COUNT_FIELD_NAME.getPreferredName(), shardMinDocCount);
return builder;
}

View File

@ -0,0 +1,269 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
/**
*
*/
/**
*
*/
public class TermsAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, TermsAggregatorBuilder> {
public static final ParseField EXECUTION_HINT_FIELD_NAME = new ParseField("execution_hint");
public static final ParseField SHARD_SIZE_FIELD_NAME = new ParseField("shard_size");
public static final ParseField MIN_DOC_COUNT_FIELD_NAME = new ParseField("min_doc_count");
public static final ParseField SHARD_MIN_DOC_COUNT_FIELD_NAME = new ParseField("shard_min_doc_count");
public static final ParseField REQUIRED_SIZE_FIELD_NAME = new ParseField("size");
static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(1, 0, 10,
-1);
public static final ParseField SHOW_TERM_DOC_COUNT_ERROR = new ParseField("show_term_doc_count_error");
public static final ParseField ORDER_FIELD = new ParseField("order");
private Terms.Order order = Terms.Order.compound(Terms.Order.count(false), Terms.Order.term(true));
private IncludeExclude includeExclude = null;
private String executionHint = null;
private SubAggCollectionMode collectMode = SubAggCollectionMode.DEPTH_FIRST;
private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds(
DEFAULT_BUCKET_COUNT_THRESHOLDS);
private boolean showTermDocCountError = false;
public TermsAggregatorBuilder(String name, ValueType valueType) {
super(name, StringTerms.TYPE, ValuesSourceType.ANY, valueType);
}
public TermsAggregator.BucketCountThresholds bucketCountThresholds() {
return bucketCountThresholds;
}
public TermsAggregatorBuilder bucketCountThresholds(TermsAggregator.BucketCountThresholds bucketCountThresholds) {
this.bucketCountThresholds = bucketCountThresholds;
return this;
}
/**
* Sets the size - indicating how many term buckets should be returned
* (defaults to 10)
*/
public TermsAggregatorBuilder size(int size) {
bucketCountThresholds.setRequiredSize(size);
return this;
}
/**
* Sets the shard_size - indicating the number of term buckets each shard
* will return to the coordinating node (the node that coordinates the
* search execution). The higher the shard size is, the more accurate the
* results are.
*/
public TermsAggregatorBuilder shardSize(int shardSize) {
bucketCountThresholds.setShardSize(shardSize);
return this;
}
/**
* Set the minimum document count terms should have in order to appear in
* the response.
*/
public TermsAggregatorBuilder minDocCount(long minDocCount) {
bucketCountThresholds.setMinDocCount(minDocCount);
return this;
}
/**
* Set the minimum document count terms should have on the shard in order to
* appear in the response.
*/
public TermsAggregatorBuilder shardMinDocCount(long shardMinDocCount) {
bucketCountThresholds.setShardMinDocCount(shardMinDocCount);
return this;
}
/**
* Sets the order in which the buckets will be returned.
*/
public TermsAggregatorBuilder order(Terms.Order order) {
this.order = order;
return this;
}
/**
* Sets the order in which the buckets will be returned.
*/
public TermsAggregatorBuilder order(List<Terms.Order> orders) {
order(Terms.Order.compound(orders));
return this;
}
/**
* Gets the order in which the buckets will be returned.
*/
public Terms.Order order() {
return order;
}
/**
* Expert: sets an execution hint to the aggregation.
*/
public TermsAggregatorBuilder executionHint(String executionHint) {
this.executionHint = executionHint;
return this;
}
/**
* Expert: gets an execution hint to the aggregation.
*/
public String executionHint() {
return executionHint;
}
/**
* Expert: set the collection mode.
*/
public TermsAggregatorBuilder collectMode(SubAggCollectionMode mode) {
this.collectMode = mode;
return this;
}
/**
* Expert: get the collection mode.
*/
public SubAggCollectionMode collectMode() {
return collectMode;
}
/**
* Set terms to include and exclude from the aggregation results
*/
public TermsAggregatorBuilder includeExclude(IncludeExclude includeExclude) {
this.includeExclude = includeExclude;
return this;
}
/**
* Get terms to include and exclude from the aggregation results
*/
public IncludeExclude includeExclude() {
return includeExclude;
}
/**
* Get whether doc count error will be return for individual terms
*/
public boolean showTermDocCountError() {
return showTermDocCountError;
}
/**
* Set whether doc count error will be return for individual terms
*/
public TermsAggregatorBuilder showTermDocCountError(boolean showTermDocCountError) {
this.showTermDocCountError = showTermDocCountError;
return this;
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource> config) {
return new TermsAggregatorFactory(name, type, config, order, includeExclude, executionHint, collectMode,
bucketCountThresholds, showTermDocCountError);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
bucketCountThresholds.toXContent(builder, params);
builder.field(SHOW_TERM_DOC_COUNT_ERROR.getPreferredName(), showTermDocCountError);
if (executionHint != null) {
builder.field(TermsAggregatorBuilder.EXECUTION_HINT_FIELD_NAME.getPreferredName(), executionHint);
}
builder.field(ORDER_FIELD.getPreferredName());
order.toXContent(builder, params);
builder.field(SubAggCollectionMode.KEY.getPreferredName(), collectMode.parseField().getPreferredName());
if (includeExclude != null) {
includeExclude.toXContent(builder, params);
}
return builder;
}
@Override
protected TermsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
TermsAggregatorBuilder factory = new TermsAggregatorBuilder(name, targetValueType);
factory.bucketCountThresholds = BucketCountThresholds.readFromStream(in);
factory.collectMode = SubAggCollectionMode.BREADTH_FIRST.readFrom(in);
factory.executionHint = in.readOptionalString();
if (in.readBoolean()) {
factory.includeExclude = IncludeExclude.readFromStream(in);
}
factory.order = InternalOrder.Streams.readOrder(in);
factory.showTermDocCountError = in.readBoolean();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
bucketCountThresholds.writeTo(out);
collectMode.writeTo(out);
out.writeOptionalString(executionHint);
boolean hasIncExc = includeExclude != null;
out.writeBoolean(hasIncExc);
if (hasIncExc) {
includeExclude.writeTo(out);
}
InternalOrder.Streams.writeOrder(order, out);
out.writeBoolean(showTermDocCountError);
}
@Override
protected int innerHashCode() {
return Objects.hash(bucketCountThresholds, collectMode, executionHint, includeExclude, order, showTermDocCountError);
}
@Override
protected boolean innerEquals(Object obj) {
TermsAggregatorBuilder other = (TermsAggregatorBuilder) obj;
return Objects.equals(bucketCountThresholds, other.bucketCountThresholds)
&& Objects.equals(collectMode, other.collectMode)
&& Objects.equals(executionHint, other.executionHint)
&& Objects.equals(includeExclude, other.includeExclude)
&& Objects.equals(order, other.order)
&& Objects.equals(showTermDocCountError, other.showTermDocCountError);
}
}

View File

@ -16,54 +16,169 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket.terms;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
*
*/
/**
*
*/
public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, TermsAggregatorFactory> {
public static final ParseField EXECUTION_HINT_FIELD_NAME = new ParseField("execution_hint");
public static final ParseField SHARD_SIZE_FIELD_NAME = new ParseField("shard_size");
public static final ParseField MIN_DOC_COUNT_FIELD_NAME = new ParseField("min_doc_count");
public static final ParseField SHARD_MIN_DOC_COUNT_FIELD_NAME = new ParseField("shard_min_doc_count");
public static final ParseField REQUIRED_SIZE_FIELD_NAME = new ParseField("size");
private final Terms.Order order;
private final IncludeExclude includeExclude;
private final String executionHint;
private final SubAggCollectionMode collectMode;
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private boolean showTermDocCountError;
static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds(1, 0, 10,
-1);
public static final ParseField SHOW_TERM_DOC_COUNT_ERROR = new ParseField("show_term_doc_count_error");
public static final ParseField ORDER_FIELD = new ParseField("order");
public TermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
IncludeExclude includeExclude, String executionHint, SubAggCollectionMode collectMode,
TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError) {
super(name, type, config);
this.order = order;
this.includeExclude = includeExclude;
this.executionHint = executionHint;
this.collectMode = collectMode;
this.bucketCountThresholds = bucketCountThresholds;
this.showTermDocCountError = showTermDocCountError;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, factories, pipelineAggregators, metaData) {
{
// even in the case of an unmapped aggregator, validate the
// order
InternalOrder.validate(order, this);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
&& bucketCountThresholds.getShardSize() == TermsAggregatorBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
// The user has not made a shardSize selection. Use default
// heuristic to avoid any wrong-ranking caused by distributed
// counting
bucketCountThresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
aggregationContext.searchContext().numberOfShards()));
}
bucketCountThresholds.ensureValidity();
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, aggregationContext.searchContext().parseFieldMatcher());
}
// In some cases, using ordinals is just not supported: override it
if (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals)) {
execution = ExecutionMode.MAP;
}
final long maxOrd;
final double ratio;
if (execution == null || execution.needsGlobalOrdinals()) {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher);
ratio = maxOrd / ((double) indexSearcher.getIndexReader().numDocs());
} else {
maxOrd = -1;
ratio = -1;
}
// Let's try to use a good default
if (execution == null) {
// if there is a parent bucket aggregator the number of
// instances of this aggregator is going
// to be unbounded and most instances may only aggregate few
// documents, so use hashed based
// global ordinals to keep the bucket ords dense.
if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else {
if (factories == AggregatorFactories.EMPTY) {
if (ratio <= 0.5 && maxOrd <= 2048) {
// 0.5: At least we need reduce the number of global
// ordinals look-ups by half
// 2048: GLOBAL_ORDINALS_LOW_CARDINALITY has
// additional memory usage, which directly linked to
// maxOrd, so we need to limit.
execution = ExecutionMode.GLOBAL_ORDINALS_LOW_CARDINALITY;
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
}
}
return execution.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent,
collectMode, showTermDocCountError, pipelineAggregators, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style include/exclude "
+ "settings as they can only be applied to string fields. Use an array of numeric values for include/exclude clauses used to filter numeric fields");
}
if (valuesSource instanceof ValuesSource.Numeric) {
IncludeExclude.LongFilter longFilter = null;
if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) {
if (includeExclude != null) {
longFilter = includeExclude.convertToDoubleFilter();
}
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, aggregationContext, parent, collectMode, showTermDocCountError, longFilter,
pipelineAggregators, metaData);
}
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, aggregationContext, parent, collectMode, showTermDocCountError, longFilter, pipelineAggregators,
metaData);
}
throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field()
+ "]. It can only be applied to numeric or string fields.");
}
public enum ExecutionMode {
@ -74,7 +189,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
throws IOException {
final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter();
return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, filter, aggregationContext,
parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
@ -93,7 +208,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, order,
bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError,
@ -113,11 +228,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories,
(ValuesSource.Bytes.WithOrdinals) valuesSource, order, bucketCountThresholds, filter, aggregationContext,
parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource,
order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError,
pipelineAggregators, metaData);
}
@Override
@ -132,10 +247,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
throws IOException {
if (includeExclude != null || factories.countAggregators() > 0
// we need the FieldData impl to be able to extract the
// segment to global ord mapping
// we need the FieldData impl to be able to extract the
// segment to global ord mapping
|| valuesSource.getClass() != ValuesSource.Bytes.FieldData.class) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude,
aggregationContext, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
@ -170,7 +285,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException;
throws IOException;
abstract boolean needsGlobalOrdinals();
@ -180,335 +295,4 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
}
}
private Terms.Order order = Terms.Order.compound(Terms.Order.count(false), Terms.Order.term(true));
private IncludeExclude includeExclude = null;
private String executionHint = null;
private SubAggCollectionMode collectMode = SubAggCollectionMode.DEPTH_FIRST;
private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds(
DEFAULT_BUCKET_COUNT_THRESHOLDS);
private boolean showTermDocCountError = false;
public TermsAggregatorFactory(String name, ValueType valueType) {
super(name, StringTerms.TYPE, ValuesSourceType.ANY, valueType);
}
public TermsAggregator.BucketCountThresholds bucketCountThresholds() {
return bucketCountThresholds;
}
public TermsAggregatorFactory bucketCountThresholds(TermsAggregator.BucketCountThresholds bucketCountThresholds) {
this.bucketCountThresholds = bucketCountThresholds;
return this;
}
/**
* Sets the size - indicating how many term buckets should be returned
* (defaults to 10)
*/
public TermsAggregatorFactory size(int size) {
bucketCountThresholds.setRequiredSize(size);
return this;
}
/**
* Sets the shard_size - indicating the number of term buckets each shard
* will return to the coordinating node (the node that coordinates the
* search execution). The higher the shard size is, the more accurate the
* results are.
*/
public TermsAggregatorFactory shardSize(int shardSize) {
bucketCountThresholds.setShardSize(shardSize);
return this;
}
/**
* Set the minimum document count terms should have in order to appear in
* the response.
*/
public TermsAggregatorFactory minDocCount(long minDocCount) {
bucketCountThresholds.setMinDocCount(minDocCount);
return this;
}
/**
* Set the minimum document count terms should have on the shard in order to
* appear in the response.
*/
public TermsAggregatorFactory shardMinDocCount(long shardMinDocCount) {
bucketCountThresholds.setShardMinDocCount(shardMinDocCount);
return this;
}
/**
* Sets the order in which the buckets will be returned.
*/
public TermsAggregatorFactory order(Terms.Order order) {
this.order = order;
return this;
}
/**
* Sets the order in which the buckets will be returned.
*/
public TermsAggregatorFactory order(List<Terms.Order> orders) {
order(Terms.Order.compound(orders));
return this;
}
/**
* Gets the order in which the buckets will be returned.
*/
public Terms.Order order() {
return order;
}
/**
* Expert: sets an execution hint to the aggregation.
*/
public TermsAggregatorFactory executionHint(String executionHint) {
this.executionHint = executionHint;
return this;
}
/**
* Expert: gets an execution hint to the aggregation.
*/
public String executionHint() {
return executionHint;
}
/**
* Expert: set the collection mode.
*/
public TermsAggregatorFactory collectMode(SubAggCollectionMode mode) {
this.collectMode = mode;
return this;
}
/**
* Expert: get the collection mode.
*/
public SubAggCollectionMode collectMode() {
return collectMode;
}
/**
* Set terms to include and exclude from the aggregation results
*/
public TermsAggregatorFactory includeExclude(IncludeExclude includeExclude) {
this.includeExclude = includeExclude;
return this;
}
/**
* Get terms to include and exclude from the aggregation results
*/
public IncludeExclude includeExclude() {
return includeExclude;
}
/**
* Get whether doc count error will be return for individual terms
*/
public boolean showTermDocCountError() {
return showTermDocCountError;
}
/**
* Set whether doc count error will be return for individual terms
*/
public TermsAggregatorFactory showTermDocCountError(boolean showTermDocCountError) {
this.showTermDocCountError = showTermDocCountError;
return this;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, factories, pipelineAggregators, metaData) {
{
// even in the case of an unmapped aggregator, validate the order
InternalOrder.validate(order, this);
}
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
};
}
private Order resolveOrder(List<Order> orders) {
Terms.Order order;
if (orders.size() == 1 && (orders.get(0) == InternalOrder.TERM_ASC || orders.get(0) == InternalOrder.TERM_DESC)) {
// If order is only terms order then we don't need compound
// ordering
order = orders.get(0);
} else {
// for all other cases we need compound order so term order asc
// can be added to make the order deterministic
order = Order.compound(orders);
}
return order;
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
&& bucketCountThresholds.getShardSize() == DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
// The user has not made a shardSize selection. Use default
// heuristic to avoid any wrong-ranking caused by distributed
// counting
bucketCountThresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
aggregationContext.searchContext().numberOfShards()));
}
bucketCountThresholds.ensureValidity();
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, aggregationContext.searchContext().parseFieldMatcher());
}
// In some cases, using ordinals is just not supported: override it
if (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals)) {
execution = ExecutionMode.MAP;
}
final long maxOrd;
final double ratio;
if (execution == null || execution.needsGlobalOrdinals()) {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher);
ratio = maxOrd / ((double) indexSearcher.getIndexReader().numDocs());
} else {
maxOrd = -1;
ratio = -1;
}
// Let's try to use a good default
if (execution == null) {
// if there is a parent bucket aggregator the number of instances of this aggregator is going
// to be unbounded and most instances may only aggregate few documents, so use hashed based
// global ordinals to keep the bucket ords dense.
if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else {
if (factories == AggregatorFactories.EMPTY) {
if (ratio <= 0.5 && maxOrd <= 2048) {
// 0.5: At least we need reduce the number of global ordinals look-ups by half
// 2048: GLOBAL_ORDINALS_LOW_CARDINALITY has additional memory usage, which directly linked to maxOrd, so we need to limit.
execution = ExecutionMode.GLOBAL_ORDINALS_LOW_CARDINALITY;
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
} else {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
}
}
return execution.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext,
parent, collectMode, showTermDocCountError, pipelineAggregators, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
throw new AggregationExecutionException(
"Aggregation ["
+ name
+ "] cannot support regular expression style include/exclude "
+ "settings as they can only be applied to string fields. Use an array of numeric values for include/exclude clauses used to filter numeric fields");
}
if (valuesSource instanceof ValuesSource.Numeric) {
IncludeExclude.LongFilter longFilter = null;
if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) {
if (includeExclude != null) {
longFilter = includeExclude.convertToDoubleFilter();
}
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, aggregationContext, parent, collectMode, showTermDocCountError, longFilter,
pipelineAggregators, metaData);
}
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, aggregationContext, parent, collectMode, showTermDocCountError, longFilter, pipelineAggregators,
metaData);
}
throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field()
+ "]. It can only be applied to numeric or string fields.");
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
bucketCountThresholds.toXContent(builder, params);
builder.field(SHOW_TERM_DOC_COUNT_ERROR.getPreferredName(), showTermDocCountError);
if (executionHint != null) {
builder.field(TermsAggregatorFactory.EXECUTION_HINT_FIELD_NAME.getPreferredName(), executionHint);
}
builder.field(ORDER_FIELD.getPreferredName());
order.toXContent(builder, params);
builder.field(SubAggCollectionMode.KEY.getPreferredName(), collectMode.parseField().getPreferredName());
if (includeExclude != null) {
includeExclude.toXContent(builder, params);
}
return builder;
}
@Override
protected TermsAggregatorFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
TermsAggregatorFactory factory = new TermsAggregatorFactory(name, targetValueType);
factory.bucketCountThresholds = BucketCountThresholds.readFromStream(in);
factory.collectMode = SubAggCollectionMode.BREADTH_FIRST.readFrom(in);
factory.executionHint = in.readOptionalString();
if (in.readBoolean()) {
factory.includeExclude = IncludeExclude.readFromStream(in);
}
factory.order = InternalOrder.Streams.readOrder(in);
factory.showTermDocCountError = in.readBoolean();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
bucketCountThresholds.writeTo(out);
collectMode.writeTo(out);
out.writeOptionalString(executionHint);
boolean hasIncExc = includeExclude != null;
out.writeBoolean(hasIncExc);
if (hasIncExc) {
includeExclude.writeTo(out);
}
InternalOrder.Streams.writeOrder(order, out);
out.writeBoolean(showTermDocCountError);
}
@Override
protected int innerHashCode() {
return Objects.hash(bucketCountThresholds, collectMode, executionHint, includeExclude, order, showTermDocCountError);
}
@Override
protected boolean innerEquals(Object obj) {
TermsAggregatorFactory other = (TermsAggregatorFactory) obj;
return Objects.equals(bucketCountThresholds, other.bucketCountThresholds)
&& Objects.equals(collectMode, other.collectMode)
&& Objects.equals(executionHint, other.executionHint)
&& Objects.equals(includeExclude, other.includeExclude)
&& Objects.equals(order, other.order)
&& Objects.equals(showTermDocCountError, other.showTermDocCountError);
}
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
@ -49,11 +49,11 @@ public class TermsParser extends AbstractTermsParser {
}
@Override
protected TermsAggregatorFactory doCreateFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected TermsAggregatorBuilder doCreateFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode collectMode, String executionHint,
IncludeExclude incExc, Map<ParseField, Object> otherOptions) {
TermsAggregatorFactory factory = new TermsAggregatorFactory(aggregationName, targetValueType);
List<OrderElement> orderElements = (List<OrderElement>) otherOptions.get(TermsAggregatorFactory.ORDER_FIELD);
TermsAggregatorBuilder factory = new TermsAggregatorBuilder(aggregationName, targetValueType);
List<OrderElement> orderElements = (List<OrderElement>) otherOptions.get(TermsAggregatorBuilder.ORDER_FIELD);
if (orderElements != null) {
List<Terms.Order> orders = new ArrayList<>(orderElements.size());
for (OrderElement orderElement : orderElements) {
@ -73,7 +73,7 @@ public class TermsParser extends AbstractTermsParser {
if (incExc != null) {
factory.includeExclude(incExc);
}
Boolean showTermDocCountError = (Boolean) otherOptions.get(TermsAggregatorFactory.SHOW_TERM_DOC_COUNT_ERROR);
Boolean showTermDocCountError = (Boolean) otherOptions.get(TermsAggregatorBuilder.SHOW_TERM_DOC_COUNT_ERROR);
if (showTermDocCountError != null) {
factory.showTermDocCountError(showTermDocCountError);
}
@ -84,12 +84,12 @@ public class TermsParser extends AbstractTermsParser {
public boolean parseSpecial(String aggregationName, XContentParser parser, ParseFieldMatcher parseFieldMatcher, Token token,
String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException {
if (token == XContentParser.Token.START_OBJECT) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorFactory.ORDER_FIELD)) {
otherOptions.put(TermsAggregatorFactory.ORDER_FIELD, Collections.singletonList(parseOrderParam(aggregationName, parser)));
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorBuilder.ORDER_FIELD)) {
otherOptions.put(TermsAggregatorBuilder.ORDER_FIELD, Collections.singletonList(parseOrderParam(aggregationName, parser)));
return true;
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorFactory.ORDER_FIELD)) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorBuilder.ORDER_FIELD)) {
List<OrderElement> orderElements = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
@ -100,12 +100,12 @@ public class TermsParser extends AbstractTermsParser {
"Order elements must be of type object in [" + aggregationName + "] found token of type [" + token + "].");
}
}
otherOptions.put(TermsAggregatorFactory.ORDER_FIELD, orderElements);
otherOptions.put(TermsAggregatorBuilder.ORDER_FIELD, orderElements);
return true;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorFactory.SHOW_TERM_DOC_COUNT_ERROR)) {
otherOptions.put(TermsAggregatorFactory.SHOW_TERM_DOC_COUNT_ERROR, parser.booleanValue());
if (parseFieldMatcher.match(currentFieldName, TermsAggregatorBuilder.SHOW_TERM_DOC_COUNT_ERROR)) {
otherOptions.put(TermsAggregatorBuilder.SHOW_TERM_DOC_COUNT_ERROR, parser.booleanValue());
return true;
}
}
@ -165,7 +165,7 @@ public class TermsParser extends AbstractTermsParser {
@Override
public TermsAggregator.BucketCountThresholds getDefaultBucketCountThresholds() {
return new TermsAggregator.BucketCountThresholds(TermsAggregatorFactory.DEFAULT_BUCKET_COUNT_THRESHOLDS);
return new TermsAggregator.BucketCountThresholds(TermsAggregatorBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS);
}
static Terms.Order resolveOrder(String key, boolean asc) {
@ -179,8 +179,8 @@ public class TermsParser extends AbstractTermsParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new TermsAggregatorFactory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new TermsAggregatorBuilder(null, null);
}
}

View File

@ -36,8 +36,10 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
@ -117,30 +119,21 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
return new InternalAvg(name, 0.0, 0L, formatter, pipelineAggregators(), metaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Factory> {
public static class AvgAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, AvgAggregatorBuilder> {
public Factory(String name) {
public AvgAggregatorBuilder(String name) {
super(name, InternalAvg.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
protected AvgAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new AvgAggregatorFactory(name, type, config);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected AvgAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) {
return new AvgAggregator.Factory(name);
return new AvgAggregator.AvgAggregatorBuilder(name);
}
@Override

View File

@ -0,0 +1,53 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.avg;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class AvgAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AvgAggregatorFactory> {
public AvgAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config) {
super(name, type, config);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.avg;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -50,14 +50,14 @@ public class AvgParser extends NumericValuesSourceParser {
}
@Override
protected AvgAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected AvgAggregator.AvgAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
return new AvgAggregator.Factory(aggregationName);
return new AvgAggregator.AvgAggregatorBuilder(aggregationName);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new AvgAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new AvgAggregator.AvgAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,114 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.cardinality;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.Objects;
public final class CardinalityAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource, CardinalityAggregatorBuilder> {
public static final ParseField PRECISION_THRESHOLD_FIELD = new ParseField("precision_threshold");
private Long precisionThreshold = null;
public CardinalityAggregatorBuilder(String name, ValueType targetValueType) {
super(name, InternalCardinality.TYPE, ValuesSourceType.ANY, targetValueType);
}
/**
* Set a precision threshold. Higher values improve accuracy but also
* increase memory usage.
*/
public CardinalityAggregatorBuilder precisionThreshold(long precisionThreshold) {
this.precisionThreshold = precisionThreshold;
return this;
}
/**
* Get the precision threshold. Higher values improve accuracy but also
* increase memory usage. Will return <code>null</code> if the
* precisionThreshold has not been set yet.
*/
public Long precisionThreshold() {
return precisionThreshold;
}
/**
* @deprecated no replacement - values will always be rehashed
*/
@Deprecated
public void rehash(boolean rehash) {
// Deprecated all values are already rehashed so do nothing
}
@Override
protected CardinalityAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config) {
return new CardinalityAggregatorFactory(name, type, config, precisionThreshold);
}
@Override
protected CardinalityAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
CardinalityAggregatorBuilder factory = new CardinalityAggregatorBuilder(name, targetValueType);
if (in.readBoolean()) {
factory.precisionThreshold = in.readLong();
}
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
boolean hasPrecisionThreshold = precisionThreshold != null;
out.writeBoolean(hasPrecisionThreshold);
if (hasPrecisionThreshold) {
out.writeLong(precisionThreshold);
}
}
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
if (precisionThreshold != null) {
builder.field(PRECISION_THRESHOLD_FIELD.getPreferredName(), precisionThreshold);
}
return builder;
}
@Override
protected int innerHashCode() {
return Objects.hash(precisionThreshold);
}
@Override
protected boolean innerEquals(Object obj) {
CardinalityAggregatorBuilder other = (CardinalityAggregatorBuilder) obj;
return Objects.equals(precisionThreshold, other.precisionThreshold);
}
}

View File

@ -19,113 +19,45 @@
package org.elasticsearch.search.aggregations.metrics.cardinality;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource, CardinalityAggregatorFactory> {
public class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, CardinalityAggregatorFactory> {
public static final ParseField PRECISION_THRESHOLD_FIELD = new ParseField("precision_threshold");
private final Long precisionThreshold;
private Long precisionThreshold = null;
public CardinalityAggregatorFactory(String name, ValueType targetValueType) {
super(name, InternalCardinality.TYPE, ValuesSourceType.ANY, targetValueType);
}
/**
* Set a precision threshold. Higher values improve accuracy but also
* increase memory usage.
*/
public CardinalityAggregatorFactory precisionThreshold(long precisionThreshold) {
public CardinalityAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Long precisionThreshold) {
super(name, type, config);
this.precisionThreshold = precisionThreshold;
return this;
}
/**
* Get the precision threshold. Higher values improve accuracy but also
* increase memory usage. Will return <code>null</code> if the
* precisionThreshold has not been set yet.
*/
public Long precisionThreshold() {
return precisionThreshold;
}
/**
* @deprecated no replacement - values will always be rehashed
*/
@Deprecated
public void rehash(boolean rehash) {
// Deprecated all values are already rehashed so do nothing
}
private int precision(Aggregator parent) {
return precisionThreshold == null ? defaultPrecision(parent) : HyperLogLogPlusPlus.precisionFromThreshold(precisionThreshold);
}
@Override
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new CardinalityAggregator(name, null, precision(parent), config.formatter(), context, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, valuesSource, precision(parent), config.formatter(), context, parent, pipelineAggregators,
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, null, precision(parent), config.formatter(), context, parent, pipelineAggregators,
metaData);
}
@Override
protected CardinalityAggregatorFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory(name, targetValueType);
if (in.readBoolean()) {
factory.precisionThreshold = in.readLong();
}
return factory;
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new CardinalityAggregator(name, valuesSource, precision(parent), config.formatter(), aggregationContext, parent,
pipelineAggregators, metaData);
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
boolean hasPrecisionThreshold = precisionThreshold != null;
out.writeBoolean(hasPrecisionThreshold);
if (hasPrecisionThreshold) {
out.writeLong(precisionThreshold);
}
}
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
if (precisionThreshold != null) {
builder.field(PRECISION_THRESHOLD_FIELD.getPreferredName(), precisionThreshold);
}
return builder;
}
@Override
protected int innerHashCode() {
return Objects.hash(precisionThreshold);
}
@Override
protected boolean innerEquals(Object obj) {
CardinalityAggregatorFactory other = (CardinalityAggregatorFactory) obj;
return Objects.equals(precisionThreshold, other.precisionThreshold);
private int precision(Aggregator parent) {
return precisionThreshold == null ? defaultPrecision(parent) : HyperLogLogPlusPlus.precisionFromThreshold(precisionThreshold);
}
/*
@ -146,5 +78,4 @@ public final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFa
return Math.max(precision, HyperLogLogPlusPlus.MIN_PRECISION);
}
}

View File

@ -23,7 +23,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -46,10 +46,10 @@ public class CardinalityParser extends AnyValuesSourceParser {
}
@Override
protected CardinalityAggregatorFactory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected CardinalityAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory(aggregationName, targetValueType);
Long precisionThreshold = (Long) otherOptions.get(CardinalityAggregatorFactory.PRECISION_THRESHOLD_FIELD);
CardinalityAggregatorBuilder factory = new CardinalityAggregatorBuilder(aggregationName, targetValueType);
Long precisionThreshold = (Long) otherOptions.get(CardinalityAggregatorBuilder.PRECISION_THRESHOLD_FIELD);
if (precisionThreshold != null) {
factory.precisionThreshold(precisionThreshold);
}
@ -60,8 +60,8 @@ public class CardinalityParser extends AnyValuesSourceParser {
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
if (token.isValue()) {
if (parseFieldMatcher.match(currentFieldName, CardinalityAggregatorFactory.PRECISION_THRESHOLD_FIELD)) {
otherOptions.put(CardinalityAggregatorFactory.PRECISION_THRESHOLD_FIELD, parser.longValue());
if (parseFieldMatcher.match(currentFieldName, CardinalityAggregatorBuilder.PRECISION_THRESHOLD_FIELD)) {
otherOptions.put(CardinalityAggregatorBuilder.PRECISION_THRESHOLD_FIELD, parser.longValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, REHASH)) {
// ignore
@ -72,7 +72,7 @@ public class CardinalityParser extends AnyValuesSourceParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new CardinalityAggregatorFactory(null, null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new CardinalityAggregatorBuilder(null, null);
}
}

View File

@ -38,7 +38,8 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -174,18 +175,18 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights);
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Factory> {
public static class GeoBoundsAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoBoundsAggregatorBuilder> {
private boolean wrapLongitude = true;
public Factory(String name) {
public GeoBoundsAggregatorBuilder(String name) {
super(name, InternalGeoBounds.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
/**
* Set whether to wrap longitudes. Defaults to true.
*/
public Factory wrapLongitude(boolean wrapLongitude) {
public GeoBoundsAggregatorBuilder wrapLongitude(boolean wrapLongitude) {
this.wrapLongitude = wrapLongitude;
return this;
}
@ -198,22 +199,15 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, pipelineAggregators, metaData);
protected GeoBoundsAggregatorFactory innerBuild(AggregationContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config) {
return new GeoBoundsAggregatorFactory(name, type, config, wrapLongitude);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected GeoBoundsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
Factory factory = new Factory(name);
GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregatorBuilder(name);
factory.wrapLongitude = in.readBoolean();
return factory;
}
@ -236,7 +230,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
GeoBoundsAggregatorBuilder other = (GeoBoundsAggregatorBuilder) obj;
return Objects.equals(wrapLongitude, other.wrapLongitude);
}

View File

@ -0,0 +1,55 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.geobounds;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
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.List;
import java.util.Map;
public class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoBoundsAggregatorFactory> {
private final boolean wrapLongitude;
public GeoBoundsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude) {
super(name, type, config);
this.wrapLongitude = wrapLongitude;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, pipelineAggregators, metaData);
}
}

View File

@ -23,7 +23,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -43,9 +43,9 @@ public class GeoBoundsParser extends GeoPointValuesSourceParser {
}
@Override
protected GeoBoundsAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected GeoBoundsAggregator.GeoBoundsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
GeoBoundsAggregator.Factory factory = new GeoBoundsAggregator.Factory(aggregationName);
GeoBoundsAggregator.GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregator.GeoBoundsAggregatorBuilder(aggregationName);
Boolean wrapLongitude = (Boolean) otherOptions.get(GeoBoundsAggregator.WRAP_LONGITUDE_FIELD);
if (wrapLongitude != null) {
factory.wrapLongitude(wrapLongitude);
@ -66,8 +66,8 @@ public class GeoBoundsParser extends GeoPointValuesSourceParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new GeoBoundsAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new GeoBoundsAggregator.GeoBoundsAggregatorBuilder(null);
}
}

View File

@ -38,9 +38,9 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -125,29 +125,22 @@ public final class GeoCentroidAggregator extends MetricsAggregator {
Releasables.close(centroids, counts);
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.GeoPoint, Factory> {
public static class GeoCentroidAggregatorBuilder
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.GeoPoint, GeoCentroidAggregatorBuilder> {
public Factory(String name) {
public GeoCentroidAggregatorBuilder(String name) {
super(name, InternalGeoCentroid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoCentroidAggregator(name, aggregationContext, parent, null, pipelineAggregators, metaData);
protected GeoCentroidAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config) {
return new GeoCentroidAggregatorFactory(name, type, config);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoCentroidAggregator(name, aggregationContext, parent, valuesSource, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected GeoCentroidAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
return new Factory(name);
return new GeoCentroidAggregatorBuilder(name);
}
@Override

View File

@ -0,0 +1,52 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.geocentroid;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
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.List;
import java.util.Map;
public class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoCentroidAggregatorFactory> {
public GeoCentroidAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config) {
super(name, type, config);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoCentroidAggregator(name, aggregationContext, parent, null, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoCentroidAggregator(name, aggregationContext, parent, valuesSource, pipelineAggregators, metaData);
}
}

View File

@ -23,7 +23,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -52,13 +52,13 @@ public class GeoCentroidParser extends GeoPointValuesSourceParser {
}
@Override
protected GeoCentroidAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected GeoCentroidAggregator.GeoCentroidAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
return new GeoCentroidAggregator.Factory(aggregationName);
return new GeoCentroidAggregator.GeoCentroidAggregatorBuilder(aggregationName);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new GeoCentroidAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new GeoCentroidAggregator.GeoCentroidAggregatorBuilder(null);
}
}

View File

@ -37,7 +37,9 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
@ -118,29 +120,21 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, pipelineAggregators(), metaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Factory> {
public static class MaxAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MaxAggregatorBuilder> {
public Factory(String name) {
public MaxAggregatorBuilder(String name) {
super(name, InternalMax.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
protected MaxAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new MaxAggregatorFactory(name, type, config);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected MaxAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) {
return new MaxAggregator.Factory(name);
return new MaxAggregator.MaxAggregatorBuilder(name);
}
@Override

View File

@ -0,0 +1,53 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.max;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class MaxAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, MaxAggregatorFactory> {
public MaxAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config) {
super(name, type, config);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.max;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -50,14 +50,14 @@ public class MaxParser extends NumericValuesSourceParser {
}
@Override
protected MaxAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected MaxAggregator.MaxAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
return new MaxAggregator.Factory(aggregationName);
return new MaxAggregator.MaxAggregatorBuilder(aggregationName);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new MaxAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new MaxAggregator.MaxAggregatorBuilder(null);
}
}

View File

@ -37,8 +37,10 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
@ -117,29 +119,21 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, pipelineAggregators(), metaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Factory> {
public static class MinAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MinAggregatorBuilder> {
public Factory(String name) {
public MinAggregatorBuilder(String name) {
super(name, InternalMin.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
protected MinAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
return new MinAggregatorFactory(name, type, config);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Factory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected MinAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) {
return new MinAggregator.Factory(name);
return new MinAggregator.MinAggregatorBuilder(name);
}
@Override

View File

@ -0,0 +1,53 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.min;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class MinAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, MinAggregatorFactory> {
public MinAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config) {
super(name, type, config);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -22,7 +22,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -51,13 +51,13 @@ public class MinParser extends NumericValuesSourceParser {
}
@Override
protected MinAggregator.Factory createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected MinAggregator.MinAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
return new MinAggregator.Factory(aggregationName);
return new MinAggregator.MinAggregatorBuilder(aggregationName);
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new MinAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new MinAggregator.MinAggregatorBuilder(null);
}
}

View File

@ -27,7 +27,7 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
@ -117,7 +117,7 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
protected ValuesSourceAggregatorBuilder<Numeric, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
PercentilesMethod method = (PercentilesMethod) otherOptions.getOrDefault(METHOD_FIELD, PercentilesMethod.TDIGEST);
@ -128,7 +128,7 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
return buildFactory(aggregationName, cdfValues, method, compression, numberOfSignificantValueDigits, keyed);
}
protected abstract ValuesSourceAggregatorFactory<Numeric, ?> buildFactory(String aggregationName, double[] cdfValues,
protected abstract ValuesSourceAggregatorBuilder<Numeric, ?> buildFactory(String aggregationName, double[] cdfValues,
PercentilesMethod method,
Double compression,
Integer numberOfSignificantValueDigits, Boolean keyed);

View File

@ -22,24 +22,23 @@ package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory.LeafOnly;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder.LeafOnly;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Numeric, PercentileRanksAggregatorFactory> {
public class PercentileRanksAggregatorBuilder extends LeafOnly<ValuesSource.Numeric, PercentileRanksAggregatorBuilder> {
private double[] values;
private PercentilesMethod method = PercentilesMethod.TDIGEST;
@ -47,14 +46,14 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
private double compression = 100.0;
private boolean keyed = false;
public PercentileRanksAggregatorFactory(String name) {
public PercentileRanksAggregatorBuilder(String name) {
super(name, InternalTDigestPercentileRanks.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* Set the values to compute percentiles from.
*/
public PercentileRanksAggregatorFactory values(double... values) {
public PercentileRanksAggregatorBuilder values(double... values) {
double[] sortedValues = Arrays.copyOf(values, values.length);
Arrays.sort(sortedValues);
this.values = sortedValues;
@ -71,7 +70,7 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
/**
* Set whether the XContent response should be keyed
*/
public PercentileRanksAggregatorFactory keyed(boolean keyed) {
public PercentileRanksAggregatorBuilder keyed(boolean keyed) {
this.keyed = keyed;
return this;
}
@ -87,7 +86,7 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
* Expert: set the number of significant digits in the values. Only relevant
* when using {@link PercentilesMethod#HDR}.
*/
public PercentileRanksAggregatorFactory numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
public PercentileRanksAggregatorBuilder numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
return this;
}
@ -104,7 +103,7 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
* Expert: set the compression. Higher values improve accuracy but also
* memory usage. Only relevant when using {@link PercentilesMethod#TDIGEST}.
*/
public PercentileRanksAggregatorFactory compression(double compression) {
public PercentileRanksAggregatorBuilder compression(double compression) {
this.compression = compression;
return this;
}
@ -117,7 +116,7 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
return compression;
}
public PercentileRanksAggregatorFactory method(PercentilesMethod method) {
public PercentileRanksAggregatorBuilder method(PercentilesMethod method) {
this.method = method;
return this;
}
@ -127,41 +126,21 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
switch (method) {
case TDIGEST:
return new TDigestPercentileRanksAggregator(name, null, aggregationContext, parent, values, compression, keyed,
config.formatter(),
pipelineAggregators, metaData);
return new TDigestPercentileRanksAggregatorFactory(name, type, config, values, compression, keyed);
case HDR:
return new HDRPercentileRanksAggregator(name, null, aggregationContext, parent, values, numberOfSignificantValueDigits, keyed,
config.formatter(), pipelineAggregators, metaData);
return new HDRPercentileRanksAggregatorFactory(name, type, config, values, numberOfSignificantValueDigits, keyed);
default:
throw new IllegalStateException("Illegal method [" + method.getName() + "]");
}
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
switch (method) {
case TDIGEST:
return new TDigestPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
case HDR:
return new HDRPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, numberOfSignificantValueDigits,
keyed, config.formatter(), pipelineAggregators, metaData);
default:
throw new IllegalStateException("Illegal method [" + method.getName() + "]");
}
}
@Override
protected PercentileRanksAggregatorFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected PercentileRanksAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
PercentileRanksAggregatorFactory factory = new PercentileRanksAggregatorFactory(name);
PercentileRanksAggregatorBuilder factory = new PercentileRanksAggregatorBuilder(name);
factory.values = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.numberOfSignificantValueDigits = in.readVInt();
@ -195,7 +174,7 @@ public class PercentileRanksAggregatorFactory extends LeafOnly<ValuesSource.Nume
@Override
protected boolean innerEquals(Object obj) {
PercentileRanksAggregatorFactory other = (PercentileRanksAggregatorFactory) obj;
PercentileRanksAggregatorBuilder other = (PercentileRanksAggregatorBuilder) obj;
if (!Objects.equals(method, other.method)) {
return false;
}

View File

@ -19,10 +19,10 @@
package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentileRanks;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
/**
*
@ -46,9 +46,9 @@ public class PercentileRanksParser extends AbstractPercentilesParser {
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
protected ValuesSourceAggregatorBuilder<Numeric, ?> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
PercentileRanksAggregatorFactory factory = new PercentileRanksAggregatorFactory(aggregationName);
PercentileRanksAggregatorBuilder factory = new PercentileRanksAggregatorBuilder(aggregationName);
if (keys != null) {
factory.values(keys);
}
@ -68,8 +68,8 @@ public class PercentileRanksParser extends AbstractPercentilesParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new PercentileRanksAggregatorFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new PercentileRanksAggregatorBuilder(null);
}
}

View File

@ -22,24 +22,23 @@ package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentilesAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentilesAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory.LeafOnly;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder.LeafOnly;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric, PercentilesAggregatorFactory> {
public class PercentilesAggregatorBuilder extends LeafOnly<ValuesSource.Numeric, PercentilesAggregatorBuilder> {
private double[] percents = PercentilesParser.DEFAULT_PERCENTS;
private PercentilesMethod method = PercentilesMethod.TDIGEST;
@ -47,14 +46,14 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
private double compression = 100.0;
private boolean keyed = false;
public PercentilesAggregatorFactory(String name) {
public PercentilesAggregatorBuilder(String name) {
super(name, InternalTDigestPercentiles.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* Set the values to compute percentiles from.
*/
public PercentilesAggregatorFactory percentiles(double... percents) {
public PercentilesAggregatorBuilder percentiles(double... percents) {
double[] sortedPercents = Arrays.copyOf(percents, percents.length);
Arrays.sort(sortedPercents);
this.percents = sortedPercents;
@ -71,7 +70,7 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
/**
* Set whether the XContent response should be keyed
*/
public PercentilesAggregatorFactory keyed(boolean keyed) {
public PercentilesAggregatorBuilder keyed(boolean keyed) {
this.keyed = keyed;
return this;
}
@ -87,7 +86,7 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
* Expert: set the number of significant digits in the values. Only relevant
* when using {@link PercentilesMethod#HDR}.
*/
public PercentilesAggregatorFactory numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
public PercentilesAggregatorBuilder numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
return this;
}
@ -104,7 +103,7 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
* Expert: set the compression. Higher values improve accuracy but also
* memory usage. Only relevant when using {@link PercentilesMethod#TDIGEST}.
*/
public PercentilesAggregatorFactory compression(double compression) {
public PercentilesAggregatorBuilder compression(double compression) {
this.compression = compression;
return this;
}
@ -117,7 +116,7 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
return compression;
}
public PercentilesAggregatorFactory method(PercentilesMethod method) {
public PercentilesAggregatorBuilder method(PercentilesMethod method) {
this.method = method;
return this;
}
@ -127,41 +126,21 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config) {
switch (method) {
case TDIGEST:
return new TDigestPercentilesAggregator(name, null, aggregationContext, parent, percents, compression, keyed,
config.formatter(),
pipelineAggregators, metaData);
return new TDigestPercentilesAggregatorFactory(name, type, config, percents, compression, keyed);
case HDR:
return new HDRPercentilesAggregator(name, null, aggregationContext, parent, percents, numberOfSignificantValueDigits, keyed,
config.formatter(), pipelineAggregators, metaData);
return new HDRPercentilesAggregatorFactory(name, type, config, percents, numberOfSignificantValueDigits, keyed);
default:
throw new IllegalStateException("Illegal method [" + method.getName() + "]");
}
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
switch (method) {
case TDIGEST:
return new TDigestPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
case HDR:
return new HDRPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, numberOfSignificantValueDigits,
keyed, config.formatter(), pipelineAggregators, metaData);
default:
throw new IllegalStateException("Illegal method [" + method.getName() + "]");
}
}
@Override
protected PercentilesAggregatorFactory innerReadFrom(String name, ValuesSourceType valuesSourceType,
protected PercentilesAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
PercentilesAggregatorFactory factory = new PercentilesAggregatorFactory(name);
PercentilesAggregatorBuilder factory = new PercentilesAggregatorBuilder(name);
factory.percents = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.numberOfSignificantValueDigits = in.readVInt();
@ -195,7 +174,7 @@ public class PercentilesAggregatorFactory extends LeafOnly<ValuesSource.Numeric,
@Override
protected boolean innerEquals(Object obj) {
PercentilesAggregatorFactory other = (PercentilesAggregatorFactory) obj;
PercentilesAggregatorBuilder other = (PercentilesAggregatorBuilder) obj;
if (!Objects.equals(method, other.method)) {
return false;
}

View File

@ -19,10 +19,10 @@
package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
/**
*
@ -48,9 +48,9 @@ public class PercentilesParser extends AbstractPercentilesParser {
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
protected ValuesSourceAggregatorBuilder<Numeric, ?> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
PercentilesAggregatorFactory factory = new PercentilesAggregatorFactory(aggregationName);
PercentilesAggregatorBuilder factory = new PercentilesAggregatorBuilder(aggregationName);
if (keys != null) {
factory.percentiles(keys);
}
@ -70,8 +70,8 @@ public class PercentilesParser extends AbstractPercentilesParser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new PercentilesAggregatorFactory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new PercentilesAggregatorBuilder(null);
}
}

View File

@ -0,0 +1,65 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class HDRPercentileRanksAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, HDRPercentileRanksAggregatorFactory> {
private final double[] values;
private final int numberOfSignificantValueDigits;
private final boolean keyed;
public HDRPercentileRanksAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double[] values,
int numberOfSignificantValueDigits, boolean keyed) {
super(name, type, config);
this.values = values;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new HDRPercentileRanksAggregator(name, null, aggregationContext, parent, values, numberOfSignificantValueDigits, keyed,
config.formatter(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new HDRPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, numberOfSignificantValueDigits,
keyed, config.formatter(), pipelineAggregators, metaData);
}
}

View File

@ -0,0 +1,64 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class HDRPercentilesAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, HDRPercentilesAggregatorFactory> {
private final double[] percents;
private final int numberOfSignificantValueDigits;
private final boolean keyed;
public HDRPercentilesAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double[] percents,
int numberOfSignificantValueDigits, boolean keyed) {
super(name, type, config);
this.percents = percents;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new HDRPercentilesAggregator(name, null, aggregationContext, parent, percents, numberOfSignificantValueDigits, keyed,
config.formatter(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new HDRPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, numberOfSignificantValueDigits, keyed,
config.formatter(), pipelineAggregators, metaData);
}
}

View File

@ -0,0 +1,65 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class TDigestPercentileRanksAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, TDigestPercentileRanksAggregatorFactory> {
private final double[] percents;
private final double compression;
private final boolean keyed;
public TDigestPercentileRanksAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double[] percents,
double compression, boolean keyed) {
super(name, type, config);
this.percents = percents;
this.compression = compression;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new TDigestPercentileRanksAggregator(name, null, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new TDigestPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
}
}

View File

@ -0,0 +1,65 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class TDigestPercentilesAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, TDigestPercentilesAggregatorFactory> {
private final double[] percents;
private final double compression;
private final boolean keyed;
public TDigestPercentilesAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double[] percents,
double compression, boolean keyed) {
super(name, type, config);
this.percents = percents;
this.compression = compression;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new TDigestPercentilesAggregator(name, null, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new TDigestPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData);
}
}

View File

@ -29,8 +29,8 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
@ -38,15 +38,10 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
public class ScriptedMetricAggregator extends MetricsAggregator {
@ -110,7 +105,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
return new InternalScriptedMetric(name, null, reduceScript, pipelineAggregators(), metaData());
}
public static class Factory extends AggregatorFactory<Factory> {
public static class ScriptedMetricAggregatorBuilder extends AggregatorBuilder<ScriptedMetricAggregatorBuilder> {
private Script initScript;
private Script mapScript;
@ -118,14 +113,14 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
private Script reduceScript;
private Map<String, Object> params;
public Factory(String name) {
public ScriptedMetricAggregatorBuilder(String name) {
super(name, InternalScriptedMetric.TYPE);
}
/**
* Set the <tt>init</tt> script.
*/
public Factory initScript(Script initScript) {
public ScriptedMetricAggregatorBuilder initScript(Script initScript) {
this.initScript = initScript;
return this;
}
@ -140,7 +135,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
/**
* Set the <tt>map</tt> script.
*/
public Factory mapScript(Script mapScript) {
public ScriptedMetricAggregatorBuilder mapScript(Script mapScript) {
this.mapScript = mapScript;
return this;
}
@ -155,7 +150,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
/**
* Set the <tt>combine</tt> script.
*/
public Factory combineScript(Script combineScript) {
public ScriptedMetricAggregatorBuilder combineScript(Script combineScript) {
this.combineScript = combineScript;
return this;
}
@ -170,7 +165,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
/**
* Set the <tt>reduce</tt> script.
*/
public Factory reduceScript(Script reduceScript) {
public ScriptedMetricAggregatorBuilder reduceScript(Script reduceScript) {
this.reduceScript = reduceScript;
return this;
}
@ -186,7 +181,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
* Set parameters that will be available in the <tt>init</tt>,
* <tt>map</tt> and <tt>combine</tt> phases.
*/
public Factory params(Map<String, Object> params) {
public ScriptedMetricAggregatorBuilder params(Map<String, Object> params) {
this.params = params;
return this;
}
@ -200,68 +195,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
}
Map<String, Object> params = this.params;
if (params != null) {
params = deepCopyParams(params, context.searchContext());
} else {
params = new HashMap<>();
params.put("_agg", new HashMap<String, Object>());
}
return new ScriptedMetricAggregator(name, insertParams(initScript, params), insertParams(mapScript, params), insertParams(
combineScript, params), deepCopyScript(reduceScript, context.searchContext()), params, context, parent, pipelineAggregators,
metaData);
}
private static Script insertParams(Script script, Map<String, Object> params) {
if (script == null) {
return null;
}
return new Script(script.getScript(), script.getType(), script.getLang(), params);
}
private static Script deepCopyScript(Script script, SearchContext context) {
if (script != null) {
Map<String, Object> params = script.getParams();
if (params != null) {
params = deepCopyParams(params, context);
}
return new Script(script.getScript(), script.getType(), script.getLang(), params);
} else {
return null;
}
}
@SuppressWarnings({ "unchecked" })
private static <T> T deepCopyParams(T original, SearchContext context) {
T clone;
if (original instanceof Map) {
Map<?, ?> originalMap = (Map<?, ?>) original;
Map<Object, Object> clonedMap = new HashMap<>();
for (Entry<?, ?> e : originalMap.entrySet()) {
clonedMap.put(deepCopyParams(e.getKey(), context), deepCopyParams(e.getValue(), context));
}
clone = (T) clonedMap;
} else if (original instanceof List) {
List<?> originalList = (List<?>) original;
List<Object> clonedList = new ArrayList<Object>();
for (Object o : originalList) {
clonedList.add(deepCopyParams(o, context));
}
clone = (T) clonedList;
} else if (original instanceof String || original instanceof Integer || original instanceof Long || original instanceof Short
|| original instanceof Byte || original instanceof Float || original instanceof Double || original instanceof Character
|| original instanceof Boolean) {
clone = original;
} else {
throw new SearchParseException(context, "Can only clone primitives, String, ArrayList, and HashMap. Found: "
+ original.getClass().getCanonicalName(), null);
}
return clone;
protected AggregatorFactory<?> doBuild(AggregationContext context) {
return new ScriptedMetricAggregatorFactory(name, type, initScript, mapScript, combineScript, reduceScript, params);
}
@Override
@ -291,8 +226,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
}
@Override
protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory = new Factory(name);
protected ScriptedMetricAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregatorBuilder(name);
factory.initScript = in.readOptionalStreamable(Script.SUPPLIER);
factory.mapScript = in.readOptionalStreamable(Script.SUPPLIER);
factory.combineScript = in.readOptionalStreamable(Script.SUPPLIER);
@ -323,7 +258,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
ScriptedMetricAggregatorBuilder other = (ScriptedMetricAggregatorBuilder) obj;
return Objects.equals(initScript, other.initScript)
&& Objects.equals(mapScript, other.mapScript)
&& Objects.equals(combineScript, other.combineScript)

View File

@ -0,0 +1,121 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics.scripted;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedMetricAggregatorFactory> {
private final Script initScript;
private final Script mapScript;
private final Script combineScript;
private final Script reduceScript;
private final Map<String, Object> params;
public ScriptedMetricAggregatorFactory(String name, Type type, Script initScript, Script mapScript, Script combineScript,
Script reduceScript, Map<String, Object> params) {
super(name, type);
this.initScript = initScript;
this.mapScript = mapScript;
this.combineScript = combineScript;
this.reduceScript = reduceScript;
this.params = params;
}
@Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
}
Map<String, Object> params = this.params;
if (params != null) {
params = deepCopyParams(params, context.searchContext());
} else {
params = new HashMap<>();
params.put("_agg", new HashMap<String, Object>());
}
return new ScriptedMetricAggregator(name, insertParams(initScript, params), insertParams(mapScript, params),
insertParams(combineScript, params), deepCopyScript(reduceScript, context.searchContext()), params, context, parent,
pipelineAggregators, metaData);
}
private static Script insertParams(Script script, Map<String, Object> params) {
if (script == null) {
return null;
}
return new Script(script.getScript(), script.getType(), script.getLang(), params);
}
private static Script deepCopyScript(Script script, SearchContext context) {
if (script != null) {
Map<String, Object> params = script.getParams();
if (params != null) {
params = deepCopyParams(params, context);
}
return new Script(script.getScript(), script.getType(), script.getLang(), params);
} else {
return null;
}
}
@SuppressWarnings({ "unchecked" })
private static <T> T deepCopyParams(T original, SearchContext context) {
T clone;
if (original instanceof Map) {
Map<?, ?> originalMap = (Map<?, ?>) original;
Map<Object, Object> clonedMap = new HashMap<>();
for (Entry<?, ?> e : originalMap.entrySet()) {
clonedMap.put(deepCopyParams(e.getKey(), context), deepCopyParams(e.getValue(), context));
}
clone = (T) clonedMap;
} else if (original instanceof List) {
List<?> originalList = (List<?>) original;
List<Object> clonedList = new ArrayList<Object>();
for (Object o : originalList) {
clonedList.add(deepCopyParams(o, context));
}
clone = (T) clonedList;
} else if (original instanceof String || original instanceof Integer || original instanceof Long || original instanceof Short
|| original instanceof Byte || original instanceof Float || original instanceof Double || original instanceof Character
|| original instanceof Boolean) {
clone = original;
} else {
throw new SearchParseException(context,
"Can only clone primitives, String, ArrayList, and HashMap. Found: " + original.getClass().getCanonicalName(), null);
}
return clone;
}
}

View File

@ -27,7 +27,7 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptParameterParser;
import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import java.io.IOException;
import java.util.HashSet;
@ -54,7 +54,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
public AggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
Script initScript = null;
Script mapScript = null;
Script combineScript = null;
@ -141,7 +141,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
throw new ParsingException(parser.getTokenLocation(), "map_script field is required in [" + aggregationName + "].");
}
ScriptedMetricAggregator.Factory factory = new ScriptedMetricAggregator.Factory(aggregationName);
ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder(aggregationName);
factory.initScript(initScript);
factory.mapScript(mapScript);
factory.combineScript(combineScript);
@ -151,8 +151,8 @@ public class ScriptedMetricParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory<?> getFactoryPrototypes() {
return new ScriptedMetricAggregator.Factory(null);
public AggregatorBuilder<?> getFactoryPrototypes() {
return new ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder(null);
}
}

Some files were not shown because too many files have changed in this diff Show More