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
* 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.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.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
@ -44,7 +45,7 @@ import java.util.Objects;
public abstract class AggregatorFactory extends ToXContentToBytes implements NamedWriteable<AggregatorFactory> {
protected String name;
protected String type;
protected Type type;
protected AggregatorFactory parent;
protected AggregatorFactories factories = AggregatorFactories.EMPTY;
protected Map<String, Object> metaData;
@ -56,7 +57,7 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
* @param name The aggregation name
* @param type The aggregation type
*/
public AggregatorFactory(String name, String type) {
public AggregatorFactory(String name, Type type) {
this.name = name;
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
* using it to create {@link Aggregator}s.
*
*
* @param context
* the {@link AggregationContext} to use during initialization.
*/
@ -170,7 +171,7 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
if (this.metaData != null) {
builder.field("meta", this.metaData);
}
builder.field(type);
builder.field(type.name());
internalXContent(builder, params);
if (factories != null && factories.count() > 0) {
@ -189,7 +190,11 @@ public abstract class AggregatorFactory extends ToXContentToBytes implements Nam
@Override
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());
for (Aggregator.Parser parser : aggParsers) {
aggParsersBuilder.put(parser.type(), parser);
AggregatorFactory factoryPrototype = parser.getFactoryPrototype();
AggregatorFactory[] factoryPrototypes = parser.getFactoryPrototypes();
// NORELEASE remove this check when agg refactoring complete
if (factoryPrototype != null) {
namedWriteableRegistry.registerPrototype(AggregatorFactory.class, factoryPrototype);
if (factoryPrototypes != null) {
for (AggregatorFactory factoryPrototype : factoryPrototypes) {
namedWriteableRegistry.registerPrototype(AggregatorFactory.class, factoryPrototype);
}
}
}
this.aggParsers = unmodifiableMap(aggParsersBuilder);

View File

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

View File

@ -189,7 +189,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
private Query childFilter;
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;
}

View File

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

View File

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

View File

@ -150,7 +150,7 @@ public class FiltersAggregator extends BucketsAggregator {
private 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.keyed = keyed;
this.otherBucketKey = otherBucketKey;

View File

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

View File

@ -72,8 +72,8 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
return InternalGeoHashGrid.TYPE.name();
}
@Override
public AggregatorFactory getFactoryPrototype() {
return new GeoGridFactory(null);
public AggregatorFactory[] getFactoryPrototypes() {
return new AggregatorFactory[] { new GeoGridFactory(null) };
}
@Override
@ -121,7 +121,7 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
private int shardSize = -1;
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) {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -63,7 +63,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
public AggregatorFactory[] getFactoryPrototypes() {
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 String type() {
return TYPE.name();
public Type type() {
return TYPE;
}
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
@Override
public AggregatorFactory getFactoryPrototype() {
public AggregatorFactory[] getFactoryPrototypes() {
return null;
}
}

View File

@ -120,7 +120,7 @@ public class DateRangeParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
public AggregatorFactory[] getFactoryPrototypes() {
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> {
@Override
public String type() {
return TYPE.name();
public Type type() {
return TYPE;
}
@Override

View File

@ -246,7 +246,7 @@ public class GeoDistanceParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
public AggregatorFactory[] getFactoryPrototypes() {
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> {
@Override
public String type() {
return TYPE.name();
public Type type() {
return TYPE;
}
@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> {
@Override
public String type() {
return TYPE.name();
public Type type() {
return TYPE;
}
@Override

View File

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

View File

@ -185,7 +185,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
private int shardSize;
public Factory(String name, int shardSize) {
super(name, InternalSampler.TYPE.name());
super(name, InternalSampler.TYPE);
this.shardSize = shardSize;
}
@ -204,7 +204,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
private String executionHint;
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.maxDocsPerValue = maxDocsPerValue;
this.executionHint = executionHint;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -36,7 +36,7 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory.L
private final 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;
}

View File

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

View File

@ -171,7 +171,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
private final 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;
}

View File

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

View File

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

View File

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

View File

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

View File

@ -58,8 +58,8 @@ public class MaxParser extends NumericValuesSourceParser {
}
@Override
public AggregatorFactory getFactoryPrototype() {
return new MaxAggregator.Factory(null);
public AggregatorFactory[] getFactoryPrototypes() {
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 Factory(String name) {
super(name, InternalMin.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalMin.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override

View File

@ -59,7 +59,7 @@ public class MinParser extends NumericValuesSourceParser {
}
@Override
public AggregatorFactory getFactoryPrototype() {
return new MinAggregator.Factory(null);
public AggregatorFactory[] getFactoryPrototypes() {
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 org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.support.ValuesSource;
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.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
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 METHOD_FIELD = new ParseField("method");
@ -44,139 +43,95 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
private boolean formattable;
public AbstractPercentilesParser(boolean formattable) {
this.formattable = formattable;
super(true, formattable, false);
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
ValuesSourceParser<ValuesSource.Numeric> vsParser = ValuesSourceParser.numeric(aggregationName, InternalTDigestPercentiles.TYPE, context)
.formattable(formattable).build();
double[] keys = null;
boolean keyed = true;
Double compression = null;
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;
}
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, keysField())) {
DoubleArrayList values = new DoubleArrayList(10);
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
double value = parser.doubleValue();
values.add(value);
}
double[] keys = values.toArray();
otherOptions.put(keysField(), keys);
return true;
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
return false;
}
} 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;
}
}
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);
return false;
}
protected abstract AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> config,
double[] cdfValues, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed);
@Override
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();

View File

@ -19,14 +19,12 @@
package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.TDigestPercentileRanksAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
/**
*
@ -50,25 +48,40 @@ public class PercentileRanksParser extends AbstractPercentilesParser {
}
@Override
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> valuesSourceInput,
double[] keys, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed) {
if (keys == null) {
throw new SearchParseException(context, "Missing token values in [" + aggregationName + "].", null);
}
protected ValuesSourceAggregatorFactory<Numeric> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
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) {
return new HDRPercentileRanksAggregator.Factory(aggregationName, valuesSourceInput, keys, numberOfSignificantValueDigits,
keyed);
HDRPercentileRanksAggregator.Factory factory = new HDRPercentileRanksAggregator.Factory(aggregationName);
if (keys != null) {
factory.values(keys);
}
if (numberOfSignificantValueDigits != null) {
factory.numberOfSignificantValueDigits(numberOfSignificantValueDigits);
}
if (keyed != null) {
factory.keyed(keyed);
}
return factory;
} else {
throw new AssertionError();
}
}
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
return null;
public AggregatorFactory[] getFactoryPrototypes() {
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.TDigestPercentilesAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
/**
*
@ -38,7 +37,7 @@ public class PercentilesParser extends AbstractPercentilesParser {
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
public String type() {
@ -51,24 +50,40 @@ public class PercentilesParser extends AbstractPercentilesParser {
}
@Override
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> valuesSourceInput,
double[] keys, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed) {
if (keys == null) {
keys = DEFAULT_PERCENTS;
}
protected ValuesSourceAggregatorFactory<Numeric> buildFactory(String aggregationName, double[] keys, PercentilesMethod method,
Double compression, Integer numberOfSignificantValueDigits, Boolean keyed) {
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) {
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 {
throw new AssertionError();
}
}
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
return null;
public AggregatorFactory[] getFactoryPrototypes() {
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;
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.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.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.ValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
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> {
private final double[] values;
private final int numberOfSignificantValueDigits;
private final boolean keyed;
private double[] values;
private int numberOfSignificantValueDigits = 3;
private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, double[] values,
int numberOfSignificantValueDigits, boolean keyed) {
super(name, InternalHDRPercentiles.TYPE.name(), valuesSourceInput);
this.values = values;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
public Factory(String name) {
super(name, InternalHDRPercentileRanks.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* 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;
}
/**
* 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
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
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,
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;
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.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.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.ValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
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> {
private final double[] percents;
private final int numberOfSignificantValueDigits;
private final boolean keyed;
private double[] percents = PercentilesParser.DEFAULT_PERCENTS;
private int numberOfSignificantValueDigits = 3;
private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, double[] percents,
int numberOfSignificantValueDigits, boolean keyed) {
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput);
this.percents = percents;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
public Factory(String name) {
super(name, InternalHDRPercentiles.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* 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;
}
/**
* 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
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
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,
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;
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;
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.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.ValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
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> {
private final double[] values;
private final double compression;
private final boolean keyed;
private double[] values;
private double compression = 100.0;
private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput,
double[] values, double compression, boolean keyed) {
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput);
this.values = values;
this.compression = compression;
public Factory(String name) {
super(name, InternalTDigestPercentileRanks.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* 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;
}
/**
* 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
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
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,
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;
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;
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.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.ValuesSourceParser;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
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> {
private final double[] percents;
private final double compression;
private final boolean keyed;
private double[] percents = PercentilesParser.DEFAULT_PERCENTS;
private double compression = 100.0;
private boolean keyed = false;
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput,
double[] percents, double compression, boolean keyed) {
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput);
this.percents = percents;
this.compression = compression;
public Factory(String name) {
super(name, InternalTDigestPercentiles.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* 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;
}
/**
* 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
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
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,
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,
Map<String, Object> params) {
super(name, InternalScriptedMetric.TYPE.name());
super(name, InternalScriptedMetric.TYPE);
this.initScript = initScript;
this.mapScript = mapScript;
this.combineScript = combineScript;

View File

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

View File

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

View File

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

View File

@ -202,7 +202,7 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
private double sigma = 2.0;
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) {

View File

@ -69,7 +69,7 @@ public class ExtendedStatsParser extends NumericValuesSourceParser {
}
@Override
public AggregatorFactory getFactoryPrototype() {
return new ExtendedStatsAggregator.Factory(null);
public AggregatorFactory[] getFactoryPrototypes() {
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 Factory(String name) {
super(name, InternalSum.TYPE.name(), ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, InternalSum.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override

View File

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

View File

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

View File

@ -140,7 +140,7 @@ public class TopHitsParser implements Aggregator.Parser {
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory getFactoryPrototype() {
public AggregatorFactory[] getFactoryPrototypes() {
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 Factory(String name, ValuesSourceType valuesSourceType, ValueType valueType) {
super(name, InternalValueCount.TYPE.name(), valuesSourceType, valueType);
super(name, InternalValueCount.TYPE, valuesSourceType, valueType);
}
@Override

View File

@ -58,7 +58,7 @@ public class ValueCountParser extends AnyValuesSourceParser {
}
@Override
public AggregatorFactory getFactoryPrototype() {
return new ValueCountAggregator.Factory<ValuesSource>(null, null, null);
public AggregatorFactory[] getFactoryPrototypes() {
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.AggregatorFactories;
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.format.ValueFormat;
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> {
protected LeafOnly(String name, String type, ValuesSourceParser.Input<VS> input) {
protected LeafOnly(String name, Type type, ValuesSourceParser.Input<VS> 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);
}
@ -86,7 +87,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
* {@link ValuesSourceParser} to using {@link AbstractValuesSourceParser}.
*/
@Deprecated
protected ValuesSourceAggregatorFactory(String name, String type, ValuesSourceParser.Input<VS> input) {
protected ValuesSourceAggregatorFactory(String name, Type type, ValuesSourceParser.Input<VS> input) {
super(name, type);
this.valuesSourceType = input.valuesSourceType;
this.targetValueType = input.targetValueType;
@ -98,7 +99,7 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
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);
this.valuesSourceType = valuesSourceType;
this.targetValueType = targetValueType;

View File

@ -178,9 +178,9 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
assertEquals(testAgg.name, parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, 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());
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.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()) {
testAgg.writeTo(output);
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);
assertEquals(deserializedQuery, testAgg);
assertEquals(deserializedQuery.hashCode(), testAgg.hashCode());
@ -241,7 +241,7 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
try (BytesStreamOutput output = new BytesStreamOutput()) {
agg.writeTo(output);
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")
AF secondAgg = (AF) prototype.readFrom(in);
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;
}
}