Aggregations Refactor: Refactor Percentiles and Percentile Ranks Aggregation

This commit is contained in:
Colin Goodheart-Smithe 2015-11-18 15:07:50 +00:00
parent 2772e88447
commit 113df32b3a
73 changed files with 904 additions and 293 deletions

View File

@ -65,7 +65,7 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
* @return an empty {@link AggregatorFactory} instance for this parser * @return an empty {@link AggregatorFactory} instance for this parser
* that can be used for deserialization * that can be used for deserialization
*/ */
AggregatorFactory getFactoryPrototype(); AggregatorFactory[] getFactoryPrototypes();
} }
/** /**

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.common.xcontent.XContentBuilder; 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.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime; import org.elasticsearch.search.internal.SearchContext.Lifetime;
@ -44,7 +45,7 @@ import java.util.Objects;
public abstract class AggregatorFactory extends ToXContentToBytes implements NamedWriteable<AggregatorFactory> { public abstract class AggregatorFactory extends ToXContentToBytes implements NamedWriteable<AggregatorFactory> {
protected String name; protected String name;
protected String type; protected Type type;
protected AggregatorFactory parent; protected AggregatorFactory parent;
protected AggregatorFactories factories = AggregatorFactories.EMPTY; protected AggregatorFactories factories = AggregatorFactories.EMPTY;
protected Map<String, Object> metaData; protected Map<String, Object> metaData;
@ -56,7 +57,7 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
* @param name The aggregation name * @param name The aggregation name
* @param type The aggregation type * @param type The aggregation type
*/ */
public AggregatorFactory(String name, String type) { public AggregatorFactory(String name, Type type) {
this.name = name; this.name = name;
this.type = type; this.type = type;
} }
@ -74,7 +75,7 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
/** /**
* Allows the {@link AggregatorFactory} to initialize any state prior to * Allows the {@link AggregatorFactory} to initialize any state prior to
* using it to create {@link Aggregator}s. * using it to create {@link Aggregator}s.
* *
* @param context * @param context
* the {@link AggregationContext} to use during initialization. * the {@link AggregationContext} to use during initialization.
*/ */
@ -170,7 +171,7 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
if (this.metaData != null) { if (this.metaData != null) {
builder.field("meta", this.metaData); builder.field("meta", this.metaData);
} }
builder.field(type); builder.field(type.name());
internalXContent(builder, params); internalXContent(builder, params);
if (factories != null && factories.count() > 0) { if (factories != null && factories.count() > 0) {
@ -189,7 +190,11 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
@Override @Override
public String getWriteableName() { public String getWriteableName() {
return type; return type.stream().toUtf8();
}
public String getType() {
return type.name();
} }
/** /**

View File

@ -60,10 +60,12 @@ public class AggregatorParsers {
Map<String, Aggregator.Parser> aggParsersBuilder = new HashMap<>(aggParsers.size()); Map<String, Aggregator.Parser> aggParsersBuilder = new HashMap<>(aggParsers.size());
for (Aggregator.Parser parser : aggParsers) { for (Aggregator.Parser parser : aggParsers) {
aggParsersBuilder.put(parser.type(), parser); aggParsersBuilder.put(parser.type(), parser);
AggregatorFactory factoryPrototype = parser.getFactoryPrototype(); AggregatorFactory[] factoryPrototypes = parser.getFactoryPrototypes();
// NORELEASE remove this check when agg refactoring complete // NORELEASE remove this check when agg refactoring complete
if (factoryPrototype != null) { if (factoryPrototypes != null) {
namedWriteableRegistry.registerPrototype(AggregatorFactory.class, factoryPrototype); for (AggregatorFactory factoryPrototype : factoryPrototypes) {
namedWriteableRegistry.registerPrototype(AggregatorFactory.class, factoryPrototype);
}
} }
} }
this.aggParsers = unmodifiableMap(aggParsersBuilder); this.aggParsers = unmodifiableMap(aggParsersBuilder);

View File

@ -18,7 +18,6 @@
*/ */
package org.elasticsearch.search.aggregations.bucket.children; package org.elasticsearch.search.aggregations.bucket.children;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
@ -70,7 +69,7 @@ public class ChildrenParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -189,7 +189,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
private Query childFilter; private Query childFilter;
public Factory(String name, String childType) { public Factory(String name, String childType) {
super(name, InternalChildren.TYPE.name(), new ValuesSourceParser.Input<ValuesSource.Bytes.WithOrdinals.ParentChild>()); super(name, InternalChildren.TYPE, new ValuesSourceParser.Input<ValuesSource.Bytes.WithOrdinals.ParentChild>());
this.childType = childType; this.childType = childType;
} }

View File

@ -85,7 +85,7 @@ public class FilterAggregator extends SingleBucketAggregator {
private final Query filter; private final Query filter;
public Factory(String name, Query filter) { public Factory(String name, Query filter) {
super(name, InternalFilter.TYPE.name()); super(name, InternalFilter.TYPE);
this.filter = filter; this.filter = filter;
} }

View File

@ -46,7 +46,7 @@ public class FilterParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -150,7 +150,7 @@ public class FiltersAggregator extends BucketsAggregator {
private String otherBucketKey; private String otherBucketKey;
public Factory(String name, List<KeyedFilter> filters, boolean keyed, String otherBucketKey) { public Factory(String name, List<KeyedFilter> filters, boolean keyed, String otherBucketKey) {
super(name, InternalFilters.TYPE.name()); super(name, InternalFilters.TYPE);
this.filters = filters; this.filters = filters;
this.keyed = keyed; this.keyed = keyed;
this.otherBucketKey = otherBucketKey; this.otherBucketKey = otherBucketKey;

View File

@ -119,7 +119,7 @@ public class FiltersParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -72,8 +72,8 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return InternalGeoHashGrid.TYPE.name(); return InternalGeoHashGrid.TYPE.name();
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new GeoGridFactory(null); return new AggregatorFactory[] { new GeoGridFactory(null) };
} }
@Override @Override
@ -121,7 +121,7 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
private int shardSize = -1; private int shardSize = -1;
public GeoGridFactory(String name) { public GeoGridFactory(String name) {
super(name, InternalGeoHashGrid.TYPE.name(), ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); super(name, InternalGeoHashGrid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
} }
public void precision(int precision) { public void precision(int precision) {

View File

@ -74,7 +74,7 @@ public class GlobalAggregator extends SingleBucketAggregator {
public static class Factory extends AggregatorFactory { public static class Factory extends AggregatorFactory {
public Factory(String name) { public Factory(String name) {
super(name, InternalGlobal.TYPE.name()); super(name, InternalGlobal.TYPE);
} }
@Override @Override

View File

@ -42,8 +42,8 @@ public class GlobalParser implements Aggregator.Parser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new GlobalAggregator.Factory(null); return new AggregatorFactory[] { new GlobalAggregator.Factory(null) };
} }
} }

View File

@ -106,7 +106,7 @@ public class DateHistogramParser extends HistogramParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return HistogramAggregator.DateHistogramFactory.PROTOTYPE; return new AggregatorFactory[] { HistogramAggregator.DateHistogramFactory.PROTOTYPE };
} }
} }

View File

@ -165,7 +165,7 @@ public class HistogramParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return HistogramAggregator.Factory.PROTOTYPE; return new AggregatorFactory[] { HistogramAggregator.Factory.PROTOTYPE };
} }
} }

View File

@ -67,8 +67,8 @@ public class InternalDateHistogram {
} }
@Override @Override
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
@Override @Override

View File

@ -237,8 +237,8 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
protected Factory() { protected Factory() {
} }
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
public ValueType valueType() { public ValueType valueType() {
@ -523,7 +523,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
@Override @Override
protected void doWriteTo(StreamOutput out) throws IOException { protected void doWriteTo(StreamOutput out) throws IOException {
out.writeString(factory.type()); out.writeString(factory.type().name());
InternalOrder.Streams.writeOrder(order, out); InternalOrder.Streams.writeOrder(order, out);
out.writeVLong(minDocCount); out.writeVLong(minDocCount);
if (minDocCount == 0) { if (minDocCount == 0) {

View File

@ -84,7 +84,7 @@ public class MissingAggregator extends SingleBucketAggregator {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource> {
public Factory(String name, ValuesSourceParser.Input valueSourceInput) { public Factory(String name, ValuesSourceParser.Input valueSourceInput) {
super(name, InternalMissing.TYPE.name(), valueSourceInput); super(name, InternalMissing.TYPE, valueSourceInput);
} }
@Override @Override

View File

@ -62,7 +62,7 @@ public class MissingParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -114,7 +114,7 @@ public class NestedAggregator extends SingleBucketAggregator {
} }
}; };
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), pipelineAggregators(), return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), pipelineAggregators(),
@ -142,7 +142,7 @@ public class NestedAggregator extends SingleBucketAggregator {
private final String path; private final String path;
public Factory(String name, String path) { public Factory(String name, String path) {
super(name, InternalNested.TYPE.name()); super(name, InternalNested.TYPE);
this.path = path; this.path = path;
} }

View File

@ -69,7 +69,7 @@ public class NestedParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -123,7 +123,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
private final String path; private final String path;
public Factory(String name, String path) { public Factory(String name, String path) {
super(name, InternalReverseNested.TYPE.name()); super(name, InternalReverseNested.TYPE);
this.path = path; this.path = path;
} }

View File

@ -63,7 +63,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -221,8 +221,8 @@ public class InternalRange<B extends InternalRange.Bucket, R extends InternalRan
public static class Factory<B extends Bucket, R extends InternalRange<B, R>> { public static class Factory<B extends Bucket, R extends InternalRange<B, R>> {
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
public R create(String name, List<B> ranges, ValueFormatter formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators, public R create(String name, List<B> ranges, ValueFormatter formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,

View File

@ -115,7 +115,7 @@ public class RangeParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -120,7 +120,7 @@ public class DateRangeParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -115,8 +115,8 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket, I
public static class Factory extends InternalRange.Factory<InternalDateRange.Bucket, InternalDateRange> { public static class Factory extends InternalRange.Factory<InternalDateRange.Bucket, InternalDateRange> {
@Override @Override
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
@Override @Override

View File

@ -246,7 +246,7 @@ public class GeoDistanceParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -104,8 +104,8 @@ public class InternalGeoDistance extends InternalRange<InternalGeoDistance.Bucke
public static class Factory extends InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> { public static class Factory extends InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> {
@Override @Override
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
@Override @Override

View File

@ -111,8 +111,8 @@ public class InternalIPv4Range extends InternalRange<InternalIPv4Range.Bucket, I
public static class Factory extends InternalRange.Factory<InternalIPv4Range.Bucket, InternalIPv4Range> { public static class Factory extends InternalRange.Factory<InternalIPv4Range.Bucket, InternalIPv4Range> {
@Override @Override
public String type() { public Type type() {
return TYPE.name(); return TYPE;
} }
@Override @Override

View File

@ -141,7 +141,7 @@ public class IpRangeParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -185,7 +185,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
private int shardSize; private int shardSize;
public Factory(String name, int shardSize) { public Factory(String name, int shardSize) {
super(name, InternalSampler.TYPE.name()); super(name, InternalSampler.TYPE);
this.shardSize = shardSize; this.shardSize = shardSize;
} }
@ -204,7 +204,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
private String executionHint; private String executionHint;
public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceParser.Input vsInput, int maxDocsPerValue) { public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceParser.Input vsInput, int maxDocsPerValue) {
super(name, InternalSampler.TYPE.name(), vsInput); super(name, InternalSampler.TYPE, vsInput);
this.shardSize = shardSize; this.shardSize = shardSize;
this.maxDocsPerValue = maxDocsPerValue; this.maxDocsPerValue = maxDocsPerValue;
this.executionHint = executionHint; this.executionHint = executionHint;

View File

@ -103,7 +103,7 @@ public class SamplerParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -147,7 +147,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
String executionHint, Query filter, SignificanceHeuristic significanceHeuristic) { String executionHint, Query filter, SignificanceHeuristic significanceHeuristic) {
super(name, SignificantStringTerms.TYPE.name(), valueSourceInput); super(name, SignificantStringTerms.TYPE, valueSourceInput);
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
this.executionHint = executionHint; this.executionHint = executionHint;

View File

@ -83,7 +83,7 @@ public class SignificantTermsParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -165,7 +165,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
public TermsAggregatorFactory(String name, ValuesSourceParser.Input input, Terms.Order order, public TermsAggregatorFactory(String name, ValuesSourceParser.Input input, Terms.Order order,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint,
SubAggCollectionMode executionMode, boolean showTermDocCountError) { SubAggCollectionMode executionMode, boolean showTermDocCountError) {
super(name, StringTerms.TYPE.name(), input); super(name, StringTerms.TYPE, input);
this.order = order; this.order = order;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
this.executionHint = executionHint; this.executionHint = executionHint;

View File

@ -89,7 +89,7 @@ public class TermsParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -121,7 +121,7 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name) { public Factory(String name) {
super(name, InternalAvg.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalAvg.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
@Override @Override

View File

@ -58,8 +58,8 @@ public class AvgParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new AvgAggregator.Factory(null); return new AggregatorFactory[] { new AvgAggregator.Factory(null) };
} }
} }

View File

@ -36,7 +36,7 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory.L
private final long precisionThreshold; private final long precisionThreshold;
CardinalityAggregatorFactory(String name, ValuesSourceParser.Input<ValuesSource> input, long precisionThreshold) { CardinalityAggregatorFactory(String name, ValuesSourceParser.Input<ValuesSource> input, long precisionThreshold) {
super(name, InternalCardinality.TYPE.name(), input); super(name, InternalCardinality.TYPE, input);
this.precisionThreshold = precisionThreshold; this.precisionThreshold = precisionThreshold;
} }

View File

@ -78,7 +78,7 @@ public class CardinalityParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -171,7 +171,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
private final boolean wrapLongitude; private final boolean wrapLongitude;
protected Factory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> input, boolean wrapLongitude) { protected Factory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> input, boolean wrapLongitude) {
super(name, InternalGeoBounds.TYPE.name(), input); super(name, InternalGeoBounds.TYPE, input);
this.wrapLongitude = wrapLongitude; this.wrapLongitude = wrapLongitude;
} }

View File

@ -69,7 +69,7 @@ public class GeoBoundsParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -124,7 +124,7 @@ public final class GeoCentroidAggregator extends MetricsAggregator {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.GeoPoint> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.GeoPoint> {
protected Factory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> config) { protected Factory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> config) {
super(name, InternalGeoBounds.TYPE.name(), config); super(name, InternalGeoBounds.TYPE, config);
} }
@Override @Override

View File

@ -63,7 +63,7 @@ public class GeoCentroidParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }
} }

View File

@ -122,7 +122,7 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name) { public Factory(String name) {
super(name, InternalMax.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalMax.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
@Override @Override

View File

@ -58,8 +58,8 @@ public class MaxParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new MaxAggregator.Factory(null); return new AggregatorFactory[] { new MaxAggregator.Factory(null) };
} }
} }

View File

@ -121,7 +121,7 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name) { public Factory(String name) {
super(name, InternalMin.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalMin.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
@Override @Override

View File

@ -59,7 +59,7 @@ public class MinParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new MinAggregator.Factory(null); return new AggregatorFactory[] { new MinAggregator.Factory(null) };
} }
} }

View File

@ -21,20 +21,19 @@ package org.elasticsearch.search.aggregations.metrics.percentiles;
import com.carrotsearch.hppc.DoubleArrayList; import com.carrotsearch.hppc.DoubleArrayList;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Map;
public abstract class AbstractPercentilesParser implements Aggregator.Parser { public abstract class AbstractPercentilesParser extends NumericValuesSourceParser {
public static final ParseField KEYED_FIELD = new ParseField("keyed"); public static final ParseField KEYED_FIELD = new ParseField("keyed");
public static final ParseField METHOD_FIELD = new ParseField("method"); public static final ParseField METHOD_FIELD = new ParseField("method");
@ -44,139 +43,95 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
private boolean formattable; private boolean formattable;
public AbstractPercentilesParser(boolean formattable) { public AbstractPercentilesParser(boolean formattable) {
this.formattable = formattable; super(true, formattable, false);
} }
@Override @Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
ValuesSourceParser<ValuesSource.Numeric> vsParser = ValuesSourceParser.numeric(aggregationName, InternalTDigestPercentiles.TYPE, context) if (token == XContentParser.Token.START_ARRAY) {
.formattable(formattable).build(); if (parseFieldMatcher.match(currentFieldName, keysField())) {
DoubleArrayList values = new DoubleArrayList(10);
double[] keys = null; while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
boolean keyed = true; double value = parser.doubleValue();
Double compression = null; values.add(value);
Integer numberOfSignificantValueDigits = null;
PercentilesMethod method = null;
XContentParser.Token token;
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (vsParser.token(currentFieldName, token, parser)) {
continue;
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, keysField())) {
DoubleArrayList values = new DoubleArrayList(10);
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
double value = parser.doubleValue();
values.add(value);
}
keys = values.toArray();
Arrays.sort(keys);
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (context.parseFieldMatcher().match(currentFieldName, KEYED_FIELD)) {
keyed = parser.booleanValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (method != null) {
throw new SearchParseException(context, "Found multiple methods in [" + aggregationName + "]: [" + currentFieldName
+ "]. only one of [" + PercentilesMethod.TDIGEST.getName() + "] and [" + PercentilesMethod.HDR.getName()
+ "] may be used.", parser.getTokenLocation());
}
method = PercentilesMethod.resolveFromName(currentFieldName);
if (method == null) {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
} else {
switch (method) {
case TDIGEST:
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (context.parseFieldMatcher().match(currentFieldName, COMPRESSION_FIELD)) {
compression = parser.doubleValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName
+ "]: [" + currentFieldName + "].", parser.getTokenLocation());
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
}
break;
case HDR:
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (context.parseFieldMatcher().match(currentFieldName, NUMBER_SIGNIFICANT_DIGITS_FIELD)) {
numberOfSignificantValueDigits = parser.intValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName
+ "]: [" + currentFieldName + "].", parser.getTokenLocation());
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
}
break;
}
} }
double[] keys = values.toArray();
otherOptions.put(keysField(), keys);
return true;
} else { } else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", return false;
parser.getTokenLocation()); }
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, KEYED_FIELD)) {
boolean keyed = parser.booleanValue();
otherOptions.put(KEYED_FIELD, keyed);
return true;
} else {
return false;
}
} else if (token == XContentParser.Token.START_OBJECT) {
PercentilesMethod method = PercentilesMethod.resolveFromName(currentFieldName);
if (method == null) {
return false;
} else {
otherOptions.put(METHOD_FIELD, method);
switch (method) {
case TDIGEST:
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, COMPRESSION_FIELD)) {
double compression = parser.doubleValue();
otherOptions.put(COMPRESSION_FIELD, compression);
} else {
return false;
}
} else {
return false;
}
}
break;
case HDR:
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, NUMBER_SIGNIFICANT_DIGITS_FIELD)) {
int numberOfSignificantValueDigits = parser.intValue();
otherOptions.put(NUMBER_SIGNIFICANT_DIGITS_FIELD, numberOfSignificantValueDigits);
} else {
return false;
}
} else {
return false;
}
}
break;
}
return true;
} }
} }
return false;
if (method == null) {
method = PercentilesMethod.TDIGEST;
}
switch (method) {
case TDIGEST:
if (numberOfSignificantValueDigits != null) {
throw new SearchParseException(context, "[number_of_significant_value_digits] cannot be used with method [tdigest] in ["
+ aggregationName + "].", parser.getTokenLocation());
}
if (compression == null) {
compression = 100.0;
}
break;
case HDR:
if (compression != null) {
throw new SearchParseException(context, "[compression] cannot be used with method [hdr] in [" + aggregationName + "].",
parser.getTokenLocation());
}
if (numberOfSignificantValueDigits == null) {
numberOfSignificantValueDigits = 3;
}
break;
default:
// Shouldn't get here but if we do, throw a parse exception for
// invalid method
throw new SearchParseException(context, "Unknown value for [" + currentFieldName + "] in [" + aggregationName + "]: [" + method
+ "].", parser.getTokenLocation());
}
return buildFactory(context, aggregationName, vsParser.input(), keys, method, compression,
numberOfSignificantValueDigits, keyed);
} }
protected abstract AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> config, @Override
double[] cdfValues, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed); protected ValuesSourceAggregatorFactory<Numeric> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
PercentilesMethod method = (PercentilesMethod) otherOptions.getOrDefault(METHOD_FIELD, PercentilesMethod.TDIGEST);
double[] cdfValues = (double[]) otherOptions.get(keysField());
Double compression = (Double) otherOptions.get(COMPRESSION_FIELD);
Integer numberOfSignificantValueDigits = (Integer) otherOptions.get(NUMBER_SIGNIFICANT_DIGITS_FIELD);
Boolean keyed = (Boolean) otherOptions.get(KEYED_FIELD);
return buildFactory(aggregationName, cdfValues, method, compression, numberOfSignificantValueDigits, keyed);
}
protected abstract ValuesSourceAggregatorFactory<Numeric> buildFactory(String aggregationName, double[] cdfValues,
PercentilesMethod method,
Double compression,
Integer numberOfSignificantValueDigits, Boolean keyed);
protected abstract ParseField keysField(); protected abstract ParseField keysField();

View File

@ -19,14 +19,12 @@
package org.elasticsearch.search.aggregations.metrics.percentiles; package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregator; import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentileRanks; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregator; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
/** /**
* *
@ -50,25 +48,40 @@ public class PercentileRanksParser extends AbstractPercentilesParser {
} }
@Override @Override
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> valuesSourceInput, protected ValuesSourceAggregatorFactory<Numeric> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
double[] keys, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed) { Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
if (keys == null) {
throw new SearchParseException(context, "Missing token values in [" + aggregationName + "].", null);
}
if (method == PercentilesMethod.TDIGEST) { if (method == PercentilesMethod.TDIGEST) {
return new TDigestPercentileRanksAggregator.Factory(aggregationName, valuesSourceInput, keys, compression, keyed); TDigestPercentileRanksAggregator.Factory factory = new TDigestPercentileRanksAggregator.Factory(aggregationName);
if (keys != null) {
factory.values(keys);
}
if (compression != null) {
factory.compression(compression);
}
if (keyed != null) {
factory.keyed(keyed);
}
return factory;
} else if (method == PercentilesMethod.HDR) { } else if (method == PercentilesMethod.HDR) {
return new HDRPercentileRanksAggregator.Factory(aggregationName, valuesSourceInput, keys, numberOfSignificantValueDigits, HDRPercentileRanksAggregator.Factory factory = new HDRPercentileRanksAggregator.Factory(aggregationName);
keyed); if (keys != null) {
factory.values(keys);
}
if (numberOfSignificantValueDigits != null) {
factory.numberOfSignificantValueDigits(numberOfSignificantValueDigits);
}
if (keyed != null) {
factory.keyed(keyed);
}
return factory;
} else { } else {
throw new AssertionError(); throw new AssertionError();
} }
} }
// NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return new AggregatorFactory[] { new TDigestPercentileRanksAggregator.Factory(null), new HDRPercentileRanksAggregator.Factory(null) };
} }
} }

View File

@ -24,8 +24,7 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercenti
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregator; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
/** /**
* *
@ -38,7 +37,7 @@ public class PercentilesParser extends AbstractPercentilesParser {
super(true); super(true);
} }
private final static double[] DEFAULT_PERCENTS = new double[] { 1, 5, 25, 50, 75, 95, 99 }; public final static double[] DEFAULT_PERCENTS = new double[] { 1, 5, 25, 50, 75, 95, 99 };
@Override @Override
public String type() { public String type() {
@ -51,24 +50,40 @@ public class PercentilesParser extends AbstractPercentilesParser {
} }
@Override @Override
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> valuesSourceInput, protected ValuesSourceAggregatorFactory<Numeric> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
double[] keys, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed) { Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
if (keys == null) {
keys = DEFAULT_PERCENTS;
}
if (method == PercentilesMethod.TDIGEST) { if (method == PercentilesMethod.TDIGEST) {
return new TDigestPercentilesAggregator.Factory(aggregationName, valuesSourceInput, keys, compression, keyed); TDigestPercentilesAggregator.Factory factory = new TDigestPercentilesAggregator.Factory(aggregationName);
if (keys != null) {
factory.percents(keys);
}
if (compression != null) {
factory.compression(compression);
}
if (keyed != null) {
factory.keyed(keyed);
}
return factory;
} else if (method == PercentilesMethod.HDR) { } else if (method == PercentilesMethod.HDR) {
return new HDRPercentilesAggregator.Factory(aggregationName, valuesSourceInput, keys, numberOfSignificantValueDigits, keyed); HDRPercentilesAggregator.Factory factory = new HDRPercentilesAggregator.Factory(aggregationName);
if (keys != null) {
factory.percents(keys);
}
if (numberOfSignificantValueDigits != null) {
factory.numberOfSignificantValueDigits(numberOfSignificantValueDigits);
}
if (keyed != null) {
factory.keyed(keyed);
}
return factory;
} else { } else {
throw new AssertionError(); throw new AssertionError();
} }
} }
// NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return new AggregatorFactory[] { new TDigestPercentilesAggregator.Factory(null), new HDRPercentilesAggregator.Factory(null) };
} }
} }

View File

@ -19,19 +19,28 @@
package org.elasticsearch.search.aggregations.metrics.percentiles.hdr; package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
import org.HdrHistogram.DoubleHistogram; import org.HdrHistogram.DoubleHistogram;
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.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.percentiles.AbstractPercentilesParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; 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;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
/** /**
* *
@ -76,18 +85,58 @@ public class HDRPercentileRanksAggregator extends AbstractHDRPercentilesAggregat
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] values; private double[] values;
private final int numberOfSignificantValueDigits; private int numberOfSignificantValueDigits = 3;
private final boolean keyed; private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, double[] values, public Factory(String name) {
int numberOfSignificantValueDigits, boolean keyed) { super(name, InternalHDRPercentileRanks.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalHDRPercentiles.TYPE.name(), valuesSourceInput); }
this.values = values;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits; /**
* Set the values to compute percentiles from.
*/
public void values(double[] values) {
double[] sortedValues = Arrays.copyOf(values, values.length);
Arrays.sort(sortedValues);
this.values = sortedValues;
}
/**
* Get the values to compute percentiles from.
*/
public double[] values() {
return values;
}
/**
* Set whether the XContent response should be keyed
*/
public void keyed(boolean keyed) {
this.keyed = keyed; this.keyed = keyed;
} }
/**
* Get whether the XContent response should be keyed
*/
public boolean keyed() {
return keyed;
}
/**
* Expert: set the number of significant digits in the values.
*/
public void numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
}
/**
* Expert: set the number of significant digits in the values.
*/
public int numberOfSignificantValueDigits() {
return numberOfSignificantValueDigits;
}
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException { List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -102,5 +151,44 @@ public class HDRPercentileRanksAggregator extends AbstractHDRPercentilesAggregat
return new HDRPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, numberOfSignificantValueDigits, return new HDRPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, numberOfSignificantValueDigits,
keyed, config.formatter(), pipelineAggregators, metaData); keyed, config.formatter(), pipelineAggregators, metaData);
} }
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
Factory factory = new Factory(name);
factory.values = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.numberOfSignificantValueDigits = in.readVInt();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
out.writeDoubleArray(values);
out.writeBoolean(keyed);
out.writeVInt(numberOfSignificantValueDigits);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(PercentileRanksParser.VALUES_FIELD.getPreferredName(), values);
builder.field(AbstractPercentilesParser.KEYED_FIELD.getPreferredName(), keyed);
builder.startObject(PercentilesMethod.HDR.getName());
builder.field(AbstractPercentilesParser.NUMBER_SIGNIFICANT_DIGITS_FIELD.getPreferredName(), numberOfSignificantValueDigits);
builder.endObject();
return builder;
}
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.deepEquals(values, other.values) && Objects.equals(keyed, other.keyed)
&& Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits);
}
@Override
protected int innerHashCode() {
return Objects.hash(Arrays.hashCode(values), keyed, numberOfSignificantValueDigits);
}
} }
} }

View File

@ -19,20 +19,28 @@
package org.elasticsearch.search.aggregations.metrics.percentiles.hdr; package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
import org.HdrHistogram.DoubleHistogram; import org.HdrHistogram.DoubleHistogram;
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.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles; import org.elasticsearch.search.aggregations.metrics.percentiles.AbstractPercentilesParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesParser;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; 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;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
/** /**
* *
@ -78,18 +86,58 @@ public class HDRPercentilesAggregator extends AbstractHDRPercentilesAggregator {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] percents; private double[] percents = PercentilesParser.DEFAULT_PERCENTS;
private final int numberOfSignificantValueDigits; private int numberOfSignificantValueDigits = 3;
private final boolean keyed; private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, double[] percents, public Factory(String name) {
int numberOfSignificantValueDigits, boolean keyed) { super(name, InternalHDRPercentiles.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput); }
this.percents = percents;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits; /**
* Set the percentiles to compute.
*/
public void percents(double[] percents) {
double[] sortedPercents = Arrays.copyOf(percents, percents.length);
Arrays.sort(sortedPercents);
this.percents = sortedPercents;
}
/**
* Get the percentiles to compute.
*/
public double[] percents() {
return percents;
}
/**
* Set whether the XContent response should be keyed
*/
public void keyed(boolean keyed) {
this.keyed = keyed; this.keyed = keyed;
} }
/**
* Get whether the XContent response should be keyed
*/
public boolean keyed() {
return keyed;
}
/**
* Expert: set the number of significant digits in the values.
*/
public void numberOfSignificantValueDigits(int numberOfSignificantValueDigits) {
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
}
/**
* Expert: set the number of significant digits in the values.
*/
public int numberOfSignificantValueDigits() {
return numberOfSignificantValueDigits;
}
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException { List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -104,5 +152,44 @@ public class HDRPercentilesAggregator extends AbstractHDRPercentilesAggregator {
return new HDRPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, numberOfSignificantValueDigits, return new HDRPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, numberOfSignificantValueDigits,
keyed, config.formatter(), pipelineAggregators, metaData); keyed, config.formatter(), pipelineAggregators, metaData);
} }
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
Factory factory = new Factory(name);
factory.percents = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.numberOfSignificantValueDigits = in.readVInt();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
out.writeDoubleArray(percents);
out.writeBoolean(keyed);
out.writeVInt(numberOfSignificantValueDigits);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(PercentilesParser.PERCENTS_FIELD.getPreferredName(), percents);
builder.field(AbstractPercentilesParser.KEYED_FIELD.getPreferredName(), keyed);
builder.startObject(PercentilesMethod.HDR.getName());
builder.field(AbstractPercentilesParser.NUMBER_SIGNIFICANT_DIGITS_FIELD.getPreferredName(), numberOfSignificantValueDigits);
builder.endObject();
return builder;
}
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.deepEquals(percents, other.percents) && Objects.equals(keyed, other.keyed)
&& Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits);
}
@Override
protected int innerHashCode() {
return Objects.hash(Arrays.hashCode(percents), keyed, numberOfSignificantValueDigits);
}
} }
} }

View File

@ -18,19 +18,28 @@
*/ */
package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest; package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
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.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.percentiles.AbstractPercentilesParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; 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;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
/** /**
* *
@ -71,18 +80,60 @@ public class TDigestPercentileRanksAggregator extends AbstractTDigestPercentiles
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] values; private double[] values;
private final double compression; private double compression = 100.0;
private final boolean keyed; private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, public Factory(String name) {
double[] values, double compression, boolean keyed) { super(name, InternalTDigestPercentileRanks.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput); }
this.values = values;
this.compression = compression; /**
* Set the values to compute percentiles from.
*/
public void values(double[] values) {
double[] sortedValues = Arrays.copyOf(values, values.length);
Arrays.sort(sortedValues);
this.values = sortedValues;
}
/**
* Get the values to compute percentiles from.
*/
public double[] values() {
return values;
}
/**
* Set whether the XContent response should be keyed
*/
public void keyed(boolean keyed) {
this.keyed = keyed; this.keyed = keyed;
} }
/**
* Get whether the XContent response should be keyed
*/
public boolean keyed() {
return keyed;
}
/**
* Expert: set the compression. Higher values improve accuracy but also
* memory usage.
*/
public void compression(double compression) {
this.compression = compression;
}
/**
* Expert: set the compression. Higher values improve accuracy but also
* memory usage.
*/
public double compression() {
return compression;
}
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException { List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -97,5 +148,44 @@ public class TDigestPercentileRanksAggregator extends AbstractTDigestPercentiles
return new TDigestPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression, keyed, return new TDigestPercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression, keyed,
config.formatter(), pipelineAggregators, metaData); config.formatter(), pipelineAggregators, metaData);
} }
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
Factory factory = new Factory(name);
factory.values = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.compression = in.readDouble();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
out.writeDoubleArray(values);
out.writeBoolean(keyed);
out.writeDouble(compression);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(PercentileRanksParser.VALUES_FIELD.getPreferredName(), values);
builder.field(AbstractPercentilesParser.KEYED_FIELD.getPreferredName(), keyed);
builder.startObject(PercentilesMethod.TDIGEST.getName());
builder.field(AbstractPercentilesParser.COMPRESSION_FIELD.getPreferredName(), compression);
builder.endObject();
return builder;
}
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.deepEquals(values, other.values) && Objects.equals(keyed, other.keyed)
&& Objects.equals(compression, other.compression);
}
@Override
protected int innerHashCode() {
return Objects.hash(Arrays.hashCode(values), keyed, compression);
}
} }
} }

View File

@ -18,19 +18,28 @@
*/ */
package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest; package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
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.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.percentiles.AbstractPercentilesParser;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesParser;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; 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;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
/** /**
* *
@ -71,18 +80,60 @@ public class TDigestPercentilesAggregator extends AbstractTDigestPercentilesAggr
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] percents; private double[] percents = PercentilesParser.DEFAULT_PERCENTS;
private final double compression; private double compression = 100.0;
private final boolean keyed; private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, public Factory(String name) {
double[] percents, double compression, boolean keyed) { super(name, InternalTDigestPercentiles.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput); }
this.percents = percents;
this.compression = compression; /**
* Set the percentiles to compute.
*/
public void percents(double[] percents) {
double[] sortedPercents = Arrays.copyOf(percents, percents.length);
Arrays.sort(sortedPercents);
this.percents = sortedPercents;
}
/**
* Get the percentiles to compute.
*/
public double[] percents() {
return percents;
}
/**
* Set whether the XContent response should be keyed
*/
public void keyed(boolean keyed) {
this.keyed = keyed; this.keyed = keyed;
} }
/**
* Get whether the XContent response should be keyed
*/
public boolean keyed() {
return keyed;
}
/**
* Expert: set the compression. Higher values improve accuracy but also
* memory usage.
*/
public void compression(double compression) {
this.compression = compression;
}
/**
* Expert: set the compression. Higher values improve accuracy but also
* memory usage.
*/
public double compression() {
return compression;
}
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException { List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -97,5 +148,44 @@ public class TDigestPercentilesAggregator extends AbstractTDigestPercentilesAggr
return new TDigestPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed, return new TDigestPercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed,
config.formatter(), pipelineAggregators, metaData); config.formatter(), pipelineAggregators, metaData);
} }
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerReadFrom(String name, ValuesSourceType valuesSourceType,
ValueType targetValueType, StreamInput in) throws IOException {
Factory factory = new Factory(name);
factory.percents = in.readDoubleArray();
factory.keyed = in.readBoolean();
factory.compression = in.readDouble();
return factory;
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
out.writeDoubleArray(percents);
out.writeBoolean(keyed);
out.writeDouble(compression);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(PercentilesParser.PERCENTS_FIELD.getPreferredName(), percents);
builder.field(AbstractPercentilesParser.KEYED_FIELD.getPreferredName(), keyed);
builder.startObject(PercentilesMethod.TDIGEST.getName());
builder.field(AbstractPercentilesParser.COMPRESSION_FIELD.getPreferredName(), compression);
builder.endObject();
return builder;
}
@Override
protected boolean innerEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.deepEquals(percents, other.percents) && Objects.equals(keyed, other.keyed)
&& Objects.equals(compression, other.compression);
}
@Override
protected int innerHashCode() {
return Objects.hash(Arrays.hashCode(percents), keyed, compression);
}
} }
} }

View File

@ -116,7 +116,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
public Factory(String name, Script initScript, Script mapScript, Script combineScript, Script reduceScript, public Factory(String name, Script initScript, Script mapScript, Script combineScript, Script reduceScript,
Map<String, Object> params) { Map<String, Object> params) {
super(name, InternalScriptedMetric.TYPE.name()); super(name, InternalScriptedMetric.TYPE);
this.initScript = initScript; this.initScript = initScript;
this.mapScript = mapScript; this.mapScript = mapScript;
this.combineScript = combineScript; this.combineScript = combineScript;

View File

@ -152,7 +152,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -163,7 +163,7 @@ public class StatsAggregator extends NumericMetricsAggregator.MultiValue {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name) { public Factory(String name) {
super(name, InternalStats.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
@Override @Override

View File

@ -58,7 +58,7 @@ public class StatsParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new StatsAggregator.Factory(null); return new AggregatorFactory[] { new StatsAggregator.Factory(null) };
} }
} }

View File

@ -202,7 +202,7 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
private double sigma = 2.0; private double sigma = 2.0;
public Factory(String name) { public Factory(String name) {
super(name, InternalExtendedStats.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalExtendedStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
public void sigma(double sigma) { public void sigma(double sigma) {

View File

@ -69,7 +69,7 @@ public class ExtendedStatsParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new ExtendedStatsAggregator.Factory(null); return new AggregatorFactory[] { new ExtendedStatsAggregator.Factory(null) };
} }
} }

View File

@ -113,7 +113,7 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name) { public Factory(String name) {
super(name, InternalSum.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC); super(name, InternalSum.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
} }
@Override @Override

View File

@ -58,7 +58,7 @@ public class SumParser extends NumericValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new SumAggregator.Factory(null); return new AggregatorFactory[] { new SumAggregator.Factory(null) };
} }
} }

View File

@ -187,7 +187,7 @@ public class TopHitsAggregator extends MetricsAggregator {
private final SubSearchContext subSearchContext; private final SubSearchContext subSearchContext;
public Factory(String name, FetchPhase fetchPhase, SubSearchContext subSearchContext) { public Factory(String name, FetchPhase fetchPhase, SubSearchContext subSearchContext) {
super(name, InternalTopHits.TYPE.name()); super(name, InternalTopHits.TYPE);
this.fetchPhase = fetchPhase; this.fetchPhase = fetchPhase;
this.subSearchContext = subSearchContext; this.subSearchContext = subSearchContext;
} }

View File

@ -140,7 +140,7 @@ public class TopHitsParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation // NORELEASE implement this method when refactoring this aggregation
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return null; return null;
} }

View File

@ -115,7 +115,7 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> { public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> {
public Factory(String name, ValuesSourceType valuesSourceType, ValueType valueType) { public Factory(String name, ValuesSourceType valuesSourceType, ValueType valueType) {
super(name, InternalValueCount.TYPE.name(), valuesSourceType, valueType); super(name, InternalValueCount.TYPE, valuesSourceType, valueType);
} }
@Override @Override

View File

@ -58,7 +58,7 @@ public class ValueCountParser extends AnyValuesSourceParser {
} }
@Override @Override
public AggregatorFactory getFactoryPrototype() { public AggregatorFactory[] getFactoryPrototypes() {
return new ValueCountAggregator.Factory<ValuesSource>(null, null, null); return new AggregatorFactory[] { new ValueCountAggregator.Factory<ValuesSource>(null, null, null) };
} }
} }

View File

@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; 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.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.format.ValueFormat; import org.elasticsearch.search.aggregations.support.format.ValueFormat;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
@ -55,11 +56,11 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
public static abstract class LeafOnly<VS extends ValuesSource> extends ValuesSourceAggregatorFactory<VS> { public static abstract class LeafOnly<VS extends ValuesSource> extends ValuesSourceAggregatorFactory<VS> {
protected LeafOnly(String name, String type, ValuesSourceParser.Input<VS> input) { protected LeafOnly(String name, Type type, ValuesSourceParser.Input<VS> input) {
super(name, type, input); super(name, type, input);
} }
protected LeafOnly(String name, String type, ValuesSourceType valuesSourceType, ValueType targetValueType) { protected LeafOnly(String name, Type type, ValuesSourceType valuesSourceType, ValueType targetValueType) {
super(name, type, valuesSourceType, targetValueType); super(name, type, valuesSourceType, targetValueType);
} }
@ -86,7 +87,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
* {@link ValuesSourceParser} to using {@link AbstractValuesSourceParser}. * {@link ValuesSourceParser} to using {@link AbstractValuesSourceParser}.
*/ */
@Deprecated @Deprecated
protected ValuesSourceAggregatorFactory(String name, String type, ValuesSourceParser.Input<VS> input) { protected ValuesSourceAggregatorFactory(String name, Type type, ValuesSourceParser.Input<VS> input) {
super(name, type); super(name, type);
this.valuesSourceType = input.valuesSourceType; this.valuesSourceType = input.valuesSourceType;
this.targetValueType = input.targetValueType; this.targetValueType = input.targetValueType;
@ -98,7 +99,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
this.timeZone = input.timezone; this.timeZone = input.timezone;
} }
protected ValuesSourceAggregatorFactory(String name, String type, ValuesSourceType valuesSourceType, ValueType targetValueType) { protected ValuesSourceAggregatorFactory(String name, Type type, ValuesSourceType valuesSourceType, ValueType targetValueType) {
super(name, type); super(name, type);
this.valuesSourceType = valuesSourceType; this.valuesSourceType = valuesSourceType;
this.targetValueType = targetValueType; this.targetValueType = targetValueType;

View File

@ -178,9 +178,9 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
assertEquals(testAgg.name, parser.currentName()); assertEquals(testAgg.name, parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken()); assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken()); assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.type, parser.currentName()); assertEquals(testAgg.type.name(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken()); assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
AggregatorFactory newAgg = aggParsers.parser(testAgg.getWriteableName()).parse(testAgg.name, parser, SearchContext.current()); AggregatorFactory newAgg = aggParsers.parser(testAgg.getType()).parse(testAgg.name, parser, SearchContext.current());
assertSame(XContentParser.Token.END_OBJECT, parser.currentToken()); assertSame(XContentParser.Token.END_OBJECT, parser.currentToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken()); assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken()); assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());
@ -200,7 +200,7 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
try (BytesStreamOutput output = new BytesStreamOutput()) { try (BytesStreamOutput output = new BytesStreamOutput()) {
testAgg.writeTo(output); testAgg.writeTo(output);
try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) { try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) {
AggregatorFactory prototype = aggParsers.parser(testAgg.getWriteableName()).getFactoryPrototype(); AggregatorFactory prototype = (AggregatorFactory) namedWriteableRegistry.getPrototype(AggregatorFactory.class, testAgg.getWriteableName());
AggregatorFactory deserializedQuery = prototype.readFrom(in); AggregatorFactory deserializedQuery = prototype.readFrom(in);
assertEquals(deserializedQuery, testAgg); assertEquals(deserializedQuery, testAgg);
assertEquals(deserializedQuery.hashCode(), testAgg.hashCode()); assertEquals(deserializedQuery.hashCode(), testAgg.hashCode());
@ -241,7 +241,7 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
try (BytesStreamOutput output = new BytesStreamOutput()) { try (BytesStreamOutput output = new BytesStreamOutput()) {
agg.writeTo(output); agg.writeTo(output);
try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) { try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) {
AggregatorFactory prototype = aggParsers.parser(agg.getWriteableName()).getFactoryPrototype(); AggregatorFactory prototype = (AggregatorFactory) namedWriteableRegistry.getPrototype(AggregatorFactory.class, agg.getWriteableName());
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
AF secondAgg = (AF) prototype.readFrom(in); AF secondAgg = (AF) prototype.readFrom(in);
return secondAgg; return secondAgg;

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;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregator.Factory;
public class HDRPercentileRanksTests extends BaseAggregationTestCase<HDRPercentileRanksAggregator.Factory> {
@Override
protected Factory createTestAggregatorFactory() {
Factory factory = new Factory(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
factory.keyed(randomBoolean());
}
int valuesSize = randomIntBetween(1, 20);
double[] values = new double[valuesSize];
for (int i = 0; i < valuesSize; i++) {
values[i] = randomDouble() * 100;
}
factory.values(values);
if (randomBoolean()) {
factory.numberOfSignificantValueDigits(randomIntBetween(0, 5));
}
String field = randomNumericField();
int randomFieldBranch = randomInt(3);
switch (randomFieldBranch) {
case 0:
factory.field(field);
break;
case 1:
factory.field(field);
factory.script(new Script("_value + 1"));
break;
case 2:
factory.script(new Script("doc[" + field + "] + 1"));
break;
}
if (randomBoolean()) {
factory.missing("MISSING");
}
return factory;
}
}

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.metrics;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentilesAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentilesAggregator.Factory;
public class HDRPercentilesTests extends BaseAggregationTestCase<HDRPercentilesAggregator.Factory> {
@Override
protected Factory createTestAggregatorFactory() {
Factory factory = new Factory(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
factory.keyed(randomBoolean());
}
if (randomBoolean()) {
int percentsSize = randomIntBetween(1, 20);
double[] percents = new double[percentsSize];
for (int i = 0; i < percentsSize; i++) {
percents[i] = randomDouble() * 100;
}
factory.percents(percents);
}
if (randomBoolean()) {
factory.numberOfSignificantValueDigits(randomIntBetween(0, 5));
}
String field = randomNumericField();
int randomFieldBranch = randomInt(3);
switch (randomFieldBranch) {
case 0:
factory.field(field);
break;
case 1:
factory.field(field);
factory.script(new Script("_value + 1"));
break;
case 2:
factory.script(new Script("doc[" + field + "] + 1"));
break;
}
if (randomBoolean()) {
factory.missing("MISSING");
}
if (randomBoolean()) {
factory.format("###.00");
}
return factory;
}
}

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;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregator.Factory;
public class TDigestPercentileRanksTests extends BaseAggregationTestCase<TDigestPercentileRanksAggregator.Factory> {
@Override
protected Factory createTestAggregatorFactory() {
Factory factory = new Factory(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
factory.keyed(randomBoolean());
}
int valuesSize = randomIntBetween(1, 20);
double[] values = new double[valuesSize];
for (int i = 0; i < valuesSize; i++) {
values[i] = randomDouble() * 100;
}
factory.values(values);
if (randomBoolean()) {
factory.compression(randomDoubleBetween(10, 40000, true));
}
String field = randomNumericField();
int randomFieldBranch = randomInt(3);
switch (randomFieldBranch) {
case 0:
factory.field(field);
break;
case 1:
factory.field(field);
factory.script(new Script("_value + 1"));
break;
case 2:
factory.script(new Script("doc[" + field + "] + 1"));
break;
}
if (randomBoolean()) {
factory.missing("MISSING");
}
return factory;
}
}

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.metrics;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregator;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregator.Factory;
public class TDigestPercentilesTests extends BaseAggregationTestCase<TDigestPercentilesAggregator.Factory> {
@Override
protected Factory createTestAggregatorFactory() {
Factory factory = new Factory(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
factory.keyed(randomBoolean());
}
if (randomBoolean()) {
int percentsSize = randomIntBetween(1, 20);
double[] percents = new double[percentsSize];
for (int i = 0; i < percentsSize; i++) {
percents[i] = randomDouble() * 100;
}
factory.percents(percents);
}
if (randomBoolean()) {
factory.compression(randomDoubleBetween(10, 40000, true));
}
String field = randomNumericField();
int randomFieldBranch = randomInt(3);
switch (randomFieldBranch) {
case 0:
factory.field(field);
break;
case 1:
factory.field(field);
factory.script(new Script("_value + 1"));
break;
case 2:
factory.script(new Script("doc[" + field + "] + 1"));
break;
}
if (randomBoolean()) {
factory.missing("MISSING");
}
if (randomBoolean()) {
factory.format("###.00");
}
return factory;
}
}