Refactor ValuesSource to separate Parsing from Factory logic
ValuesSourceConfig is now evaluated in the ValuesSourceAggregatorFactory instead of ValueSourceParser. This means that the ValueSourceParser purely deals with parsing the XContent and the logic requiring access to the mappings etc. on the shard is left to the ValuesSourceAggregatorFactory. This means that, in the future, the parsing logic can be moved to the coordinating node.
This commit is contained in:
parent
1ec44dcdda
commit
7a3f6fc1ba
|
@ -82,8 +82,9 @@ abstract class QueryCollector extends SimpleCollector {
|
|||
if (context.aggregations() != null) {
|
||||
AggregationContext aggregationContext = new AggregationContext(context);
|
||||
context.aggregations().aggregationContext(aggregationContext);
|
||||
context.aggregations().factories().init(aggregationContext);
|
||||
|
||||
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
|
||||
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators();
|
||||
for (int i = 0; i < aggregators.length; i++) {
|
||||
if (!(aggregators[i] instanceof GlobalAggregator)) {
|
||||
Aggregator aggregator = aggregators[i];
|
||||
|
|
|
@ -72,12 +72,13 @@ public class AggregationPhase implements SearchPhase {
|
|||
if (context.aggregations() != null) {
|
||||
AggregationContext aggregationContext = new AggregationContext(context);
|
||||
context.aggregations().aggregationContext(aggregationContext);
|
||||
context.aggregations().factories().init(aggregationContext);
|
||||
|
||||
List<Aggregator> collectors = new ArrayList<>();
|
||||
Aggregator[] aggregators;
|
||||
try {
|
||||
AggregatorFactories factories = context.aggregations().factories();
|
||||
aggregators = factories.createTopLevelAggregators(aggregationContext);
|
||||
aggregators = factories.createTopLevelAggregators();
|
||||
for (int i = 0; i < aggregators.length; i++) {
|
||||
if (aggregators[i] instanceof GlobalAggregator == false) {
|
||||
collectors.add(aggregators[i]);
|
||||
|
|
|
@ -53,6 +53,12 @@ public class AggregatorFactories {
|
|||
this.pipelineAggregatorFactories = pipelineAggregators;
|
||||
}
|
||||
|
||||
public void init(AggregationContext context) {
|
||||
for (AggregatorFactory factory : factories) {
|
||||
factory.init(context);
|
||||
}
|
||||
}
|
||||
|
||||
public List<PipelineAggregator> createPipelineAggregators() throws IOException {
|
||||
List<PipelineAggregator> pipelineAggregators = new ArrayList<>();
|
||||
for (PipelineAggregatorFactory factory : this.pipelineAggregatorFactories) {
|
||||
|
@ -73,18 +79,18 @@ public class AggregatorFactories {
|
|||
// propagate the fact that only bucket 0 will be collected with single-bucket
|
||||
// aggs
|
||||
final boolean collectsFromSingleBucket = false;
|
||||
aggregators[i] = factories[i].create(parent.context(), parent, collectsFromSingleBucket);
|
||||
aggregators[i] = factories[i].create(parent, collectsFromSingleBucket);
|
||||
}
|
||||
return aggregators;
|
||||
}
|
||||
|
||||
public Aggregator[] createTopLevelAggregators(AggregationContext ctx) throws IOException {
|
||||
public Aggregator[] createTopLevelAggregators() throws IOException {
|
||||
// These aggregators are going to be used with a single bucket ordinal, no need to wrap the PER_BUCKET ones
|
||||
Aggregator[] aggregators = new Aggregator[factories.length];
|
||||
for (int i = 0; i < factories.length; i++) {
|
||||
// top-level aggs only get called with bucket 0
|
||||
final boolean collectsFromSingleBucket = true;
|
||||
aggregators[i] = factories[i].create(ctx, null, collectsFromSingleBucket);
|
||||
aggregators[i] = factories[i].create(null, collectsFromSingleBucket);
|
||||
}
|
||||
return aggregators;
|
||||
}
|
||||
|
@ -125,7 +131,7 @@ public class AggregatorFactories {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator[] createTopLevelAggregators(AggregationContext ctx) {
|
||||
public Aggregator[] createTopLevelAggregators() {
|
||||
return EMPTY_AGGREGATORS;
|
||||
}
|
||||
|
||||
|
|
|
@ -41,6 +41,7 @@ public abstract class AggregatorFactory {
|
|||
protected AggregatorFactory parent;
|
||||
protected AggregatorFactories factories = AggregatorFactories.EMPTY;
|
||||
protected Map<String, Object> metaData;
|
||||
private AggregationContext context;
|
||||
|
||||
/**
|
||||
* Constructs a new aggregator factory.
|
||||
|
@ -53,6 +54,19 @@ public abstract class AggregatorFactory {
|
|||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes this factory with the given {@link AggregationContext} ready
|
||||
* to create {@link Aggregator}s
|
||||
*/
|
||||
public final void init(AggregationContext context) {
|
||||
this.context = context;
|
||||
doInit(context);
|
||||
this.factories.init(context);
|
||||
}
|
||||
|
||||
protected void doInit(AggregationContext context) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers sub-factories with this factory. The sub-factory will be responsible for the creation of sub-aggregators under the
|
||||
* aggregator created by this factory.
|
||||
|
@ -98,7 +112,7 @@ public abstract class AggregatorFactory {
|
|||
*
|
||||
* @return The created aggregator
|
||||
*/
|
||||
public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
|
||||
public final Aggregator create(Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
|
||||
return createInternal(context, parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData);
|
||||
}
|
||||
|
||||
|
@ -116,7 +130,7 @@ public abstract class AggregatorFactory {
|
|||
* to collect bucket <tt>0</tt>, this returns an aggregator that can collect any bucket.
|
||||
*/
|
||||
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final AggregationContext context, final Aggregator parent) throws IOException {
|
||||
final Aggregator first = factory.create(context, parent, true);
|
||||
final Aggregator first = factory.create(parent, true);
|
||||
final BigArrays bigArrays = context.bigArrays();
|
||||
return new Aggregator() {
|
||||
|
||||
|
@ -197,7 +211,7 @@ public abstract class AggregatorFactory {
|
|||
if (collector == null) {
|
||||
Aggregator aggregator = aggregators.get(bucket);
|
||||
if (aggregator == null) {
|
||||
aggregator = factory.create(context, parent, true);
|
||||
aggregator = factory.create(parent, true);
|
||||
aggregator.preCollection();
|
||||
aggregators.set(bucket, aggregator);
|
||||
}
|
||||
|
|
|
@ -20,15 +20,9 @@ package org.elasticsearch.search.aggregations.bucket.children;
|
|||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
|
||||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -70,31 +64,7 @@ public class ChildrenParser implements Aggregator.Parser {
|
|||
parser.getTokenLocation());
|
||||
}
|
||||
|
||||
ValuesSourceConfig<ValuesSource.Bytes.WithOrdinals.ParentChild> config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class);
|
||||
DocumentMapper childDocMapper = context.mapperService().documentMapper(childType);
|
||||
|
||||
String parentType = null;
|
||||
Query parentFilter = null;
|
||||
Query childFilter = null;
|
||||
if (childDocMapper != null) {
|
||||
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
|
||||
if (!parentFieldMapper.active()) {
|
||||
throw new SearchParseException(context, "[children] no [_parent] field not configured that points to a parent type", parser.getTokenLocation());
|
||||
}
|
||||
parentType = parentFieldMapper.type();
|
||||
DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
|
||||
if (parentDocMapper != null) {
|
||||
// TODO: use the query API
|
||||
parentFilter = parentDocMapper.typeFilter();
|
||||
childFilter = childDocMapper.typeFilter();
|
||||
ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData().getForField(parentFieldMapper.fieldType());
|
||||
config.fieldContext(new FieldContext(parentFieldMapper.fieldType().names().indexName(), parentChildIndexFieldData, parentFieldMapper.fieldType()));
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
return new ParentToChildrenAggregator.Factory(aggregationName, config, parentType, parentFilter, childFilter);
|
||||
return new ParentToChildrenAggregator.Factory(aggregationName, childType);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,11 @@ import org.elasticsearch.common.lease.Releasables;
|
|||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.util.LongObjectPagedHashMap;
|
||||
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
|
||||
import org.elasticsearch.index.search.child.ConstantScorer;
|
||||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
|
@ -39,9 +44,11 @@ import org.elasticsearch.search.aggregations.NonCollectingAggregator;
|
|||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
@ -176,15 +183,19 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild> {
|
||||
|
||||
private final String parentType;
|
||||
private final Query parentFilter;
|
||||
private final Query childFilter;
|
||||
private String parentType;
|
||||
private final String childType;
|
||||
private Filter parentFilter;
|
||||
private Filter childFilter;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Bytes.WithOrdinals.ParentChild> config, String parentType, Query parentFilter, Query childFilter) {
|
||||
super(name, InternalChildren.TYPE.name(), config);
|
||||
this.parentType = parentType;
|
||||
this.parentFilter = parentFilter;
|
||||
this.childFilter = childFilter;
|
||||
public Factory(String name, String childType) {
|
||||
super(name, InternalChildren.TYPE.name(), new ValuesSourceParser.Input<ValuesSource.Bytes.WithOrdinals.ParentChild>());
|
||||
this.childType = childType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInit(AggregationContext context) {
|
||||
resolveConfig(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -209,5 +220,33 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
|
|||
valuesSource, maxOrd, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
private void resolveConfig(AggregationContext aggregationContext) {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class);
|
||||
DocumentMapper childDocMapper = aggregationContext.searchContext().mapperService().documentMapper(childType);
|
||||
|
||||
if (childDocMapper != null) {
|
||||
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
|
||||
if (!parentFieldMapper.active()) {
|
||||
throw new SearchParseException(aggregationContext.searchContext(),
|
||||
"[children] no [_parent] field not configured that points to a parent type", null); // NOCOMMIT fix exception args
|
||||
}
|
||||
parentType = parentFieldMapper.type();
|
||||
DocumentMapper parentDocMapper = aggregationContext.searchContext().mapperService().documentMapper(parentType);
|
||||
if (parentDocMapper != null) {
|
||||
// TODO: use the query API
|
||||
parentFilter = new QueryWrapperFilter(parentDocMapper.typeFilter());
|
||||
childFilter = new QueryWrapperFilter(childDocMapper.typeFilter());
|
||||
ParentChildIndexFieldData parentChildIndexFieldData = aggregationContext.searchContext().fieldData()
|
||||
.getForField(parentFieldMapper.fieldType());
|
||||
config.fieldContext(new FieldContext(parentFieldMapper.fieldType().names().indexName(), parentChildIndexFieldData,
|
||||
parentFieldMapper.fieldType()));
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -62,7 +61,8 @@ public class GeoHashGridParser implements Aggregator.Parser {
|
|||
@Override
|
||||
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||
|
||||
ValuesSourceParser vsParser = ValuesSourceParser.geoPoint(aggregationName, InternalGeoHashGrid.TYPE, context).build();
|
||||
ValuesSourceParser<ValuesSource.GeoPoint> vsParser = ValuesSourceParser
|
||||
.geoPoint(aggregationName, InternalGeoHashGrid.TYPE, context).build();
|
||||
|
||||
int precision = GeoHashGridParams.DEFAULT_PRECISION;
|
||||
int requiredSize = GeoHashGridParams.DEFAULT_MAX_NUM_CELLS;
|
||||
|
@ -107,7 +107,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
|
|||
shardSize = requiredSize;
|
||||
}
|
||||
|
||||
return new GeoGridFactory(aggregationName, vsParser.config(), precision, requiredSize, shardSize);
|
||||
return new GeoGridFactory(aggregationName, vsParser.input(), precision, requiredSize, shardSize);
|
||||
|
||||
}
|
||||
|
||||
|
@ -118,8 +118,9 @@ public class GeoHashGridParser implements Aggregator.Parser {
|
|||
private final int requiredSize;
|
||||
private final int shardSize;
|
||||
|
||||
public GeoGridFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, int precision, int requiredSize, int shardSize) {
|
||||
super(name, InternalGeoHashGrid.TYPE.name(), config);
|
||||
public GeoGridFactory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> input, int precision, int requiredSize,
|
||||
int shardSize) {
|
||||
super(name, InternalGeoHashGrid.TYPE.name(), input);
|
||||
this.precision = precision;
|
||||
this.requiredSize = requiredSize;
|
||||
this.shardSize = shardSize;
|
||||
|
@ -131,6 +132,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
|
|||
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize,
|
||||
Collections.<InternalGeoHashGrid.Bucket> emptyList(), pipelineAggregators, metaData);
|
||||
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
|
||||
@Override
|
||||
public InternalAggregation buildEmptyAggregation() {
|
||||
return aggregation;
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.elasticsearch.search.SearchParseException;
|
|||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -191,7 +190,11 @@ public class DateHistogramParser implements Aggregator.Parser {
|
|||
.offset(offset).build();
|
||||
|
||||
ValuesSourceConfig config = vsParser.config();
|
||||
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds,
|
||||
if (config.formatter()!=null) {
|
||||
((DateTime) config.formatter()).setTimeZone(timeZone);
|
||||
}
|
||||
ValuesSourceParser.Input input = vsParser.input();
|
||||
return new HistogramAggregator.DateHistogramFactory(aggregationName, input, rounding, order, keyed, minDocCount, extendedBounds,
|
||||
new InternalDateHistogram.Factory());
|
||||
|
||||
}
|
||||
|
|
|
@ -34,9 +34,12 @@ import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
|||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -153,11 +156,11 @@ public class HistogramAggregator extends BucketsAggregator {
|
|||
private final ExtendedBounds extendedBounds;
|
||||
private final InternalHistogram.Factory<?> histogramFactory;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> config,
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> input,
|
||||
Rounding rounding, InternalOrder order, boolean keyed, long minDocCount,
|
||||
ExtendedBounds extendedBounds, InternalHistogram.Factory<?> histogramFactory) {
|
||||
|
||||
super(name, histogramFactory.type(), config);
|
||||
super(name, histogramFactory.type(), input);
|
||||
this.rounding = rounding;
|
||||
this.order = order;
|
||||
this.keyed = keyed;
|
||||
|
@ -197,4 +200,38 @@ public class HistogramAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
public static class DateHistogramFactory extends Factory {
|
||||
|
||||
private DateTimeZone timeZone;
|
||||
|
||||
public DateHistogramFactory(String name, ValuesSourceParser.Input<Numeric> input, Rounding rounding, InternalOrder order,
|
||||
boolean keyed, long minDocCount, ExtendedBounds extendedBounds,
|
||||
org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Factory<?> histogramFactory, DateTimeZone timeZone) {
|
||||
super(name, input, rounding, order, keyed, minDocCount, extendedBounds, histogramFactory);
|
||||
this.timeZone = timeZone;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
setFormatterTimeZone();
|
||||
return super.createUnmapped(aggregationContext, parent, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator doCreateInternal(Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent,
|
||||
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
throws IOException {
|
||||
setFormatterTimeZone();
|
||||
return super
|
||||
.doCreateInternal(valuesSource, aggregationContext, parent, collectsFromSingleBucket, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
private void setFormatterTimeZone() {
|
||||
if (config.formatter() instanceof ValueFormatter.DateTime) {
|
||||
((DateTime) config.formatter()).setTimeZone(timeZone);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -127,7 +127,7 @@ public class HistogramParser implements Aggregator.Parser {
|
|||
|
||||
Rounding rounding = new Rounding.Interval(interval);
|
||||
if (offset != 0) {
|
||||
rounding = new Rounding.OffsetRounding((Rounding.Interval) rounding, offset);
|
||||
rounding = new Rounding.OffsetRounding(rounding, offset);
|
||||
}
|
||||
|
||||
if (extendedBounds != null) {
|
||||
|
@ -135,7 +135,7 @@ public class HistogramParser implements Aggregator.Parser {
|
|||
extendedBounds.processAndValidate(aggregationName, context, ValueParser.RAW);
|
||||
}
|
||||
|
||||
return new HistogramAggregator.Factory(aggregationName, vsParser.config(), rounding, order, keyed, minDocCount, extendedBounds,
|
||||
return new HistogramAggregator.Factory(aggregationName, vsParser.input(), rounding, order, keyed, minDocCount, extendedBounds,
|
||||
new InternalHistogram.Factory());
|
||||
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -83,8 +83,8 @@ public class MissingAggregator extends SingleBucketAggregator {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig valueSourceConfig) {
|
||||
super(name, InternalMissing.TYPE.name(), valueSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input valueSourceInput) {
|
||||
super(name, InternalMissing.TYPE.name(), valueSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -57,6 +57,6 @@ public class MissingParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
return new MissingAggregator.Factory(aggregationName, vsParser.config());
|
||||
return new MissingAggregator.Factory(aggregationName, vsParser.input());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueParser;
|
||||
|
@ -282,8 +282,9 @@ public class RangeAggregator extends BucketsAggregator {
|
|||
private final List<Range> ranges;
|
||||
private final boolean keyed;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valueSourceConfig, InternalRange.Factory rangeFactory, List<Range> ranges, boolean keyed) {
|
||||
super(name, rangeFactory.type(), valueSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valueSourceInput, InternalRange.Factory rangeFactory,
|
||||
List<Range> ranges, boolean keyed) {
|
||||
super(name, rangeFactory.type(), valueSourceInput);
|
||||
this.rangeFactory = rangeFactory;
|
||||
this.ranges = ranges;
|
||||
this.keyed = keyed;
|
||||
|
|
|
@ -110,6 +110,6 @@ public class RangeParser implements Aggregator.Parser {
|
|||
parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalRange.FACTORY, ranges, keyed);
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.input(), InternalRange.FACTORY, ranges, keyed);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -115,6 +115,6 @@ public class DateRangeParser implements Aggregator.Parser {
|
|||
parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalDateRange.FACTORY, ranges, keyed);
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.input(), InternalDateRange.FACTORY, ranges, keyed);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import org.elasticsearch.search.aggregations.support.GeoPointParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -161,7 +160,8 @@ public class GeoDistanceParser implements Aggregator.Parser {
|
|||
parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed);
|
||||
return new GeoDistanceFactory(aggregationName, vsParser.input(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges,
|
||||
keyed);
|
||||
}
|
||||
|
||||
private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
|
||||
|
@ -173,10 +173,10 @@ public class GeoDistanceParser implements Aggregator.Parser {
|
|||
private final List<RangeAggregator.Range> ranges;
|
||||
private final boolean keyed;
|
||||
|
||||
public GeoDistanceFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> valueSourceConfig,
|
||||
public GeoDistanceFactory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> valueSourceInput,
|
||||
InternalRange.Factory rangeFactory, GeoPoint origin, DistanceUnit unit, GeoDistance distanceType,
|
||||
List<RangeAggregator.Range> ranges, boolean keyed) {
|
||||
super(name, rangeFactory.type(), valueSourceConfig);
|
||||
super(name, rangeFactory.type(), valueSourceInput);
|
||||
this.origin = origin;
|
||||
this.unit = unit;
|
||||
this.distanceType = distanceType;
|
||||
|
|
|
@ -121,7 +121,7 @@ public class IpRangeParser implements Aggregator.Parser {
|
|||
parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalIPv4Range.FACTORY, ranges, keyed);
|
||||
return new RangeAggregator.Factory(aggregationName, vsParser.input(), InternalIPv4Range.FACTORY, ranges, keyed);
|
||||
}
|
||||
|
||||
private static void parseMaskRange(String cidr, RangeAggregator.Range range, String aggregationName, SearchContext ctx) {
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -203,8 +203,8 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
private int maxDocsPerValue;
|
||||
private String executionHint;
|
||||
|
||||
public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceConfig vsConfig, int maxDocsPerValue) {
|
||||
super(name, InternalSampler.TYPE.name(), vsConfig);
|
||||
public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceParser.Input vsInput, int maxDocsPerValue) {
|
||||
super(name, InternalSampler.TYPE.name(), vsInput);
|
||||
this.shardSize = shardSize;
|
||||
this.maxDocsPerValue = maxDocsPerValue;
|
||||
this.executionHint = executionHint;
|
||||
|
|
|
@ -23,7 +23,6 @@ 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.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -88,9 +87,9 @@ public class SamplerParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
ValuesSourceConfig vsConfig = vsParser.config();
|
||||
if (vsConfig.valid()) {
|
||||
return new SamplerAggregator.DiversifiedFactory(aggregationName, shardSize, executionHint, vsConfig, maxDocsPerValue);
|
||||
ValuesSourceParser.Input vsInput = vsParser.input();
|
||||
if (vsInput.valid()) {
|
||||
return new SamplerAggregator.DiversifiedFactory(aggregationName, shardSize, executionHint, vsInput, maxDocsPerValue);
|
||||
} else {
|
||||
if (diversityChoiceMade) {
|
||||
throw new SearchParseException(context, "Sampler aggregation has " + MAX_DOCS_PER_VALUE_FIELD.getPreferredName()
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -143,25 +143,30 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
|
|||
return new TermsAggregator.BucketCountThresholds(bucketCountThresholds);
|
||||
}
|
||||
|
||||
public SignificantTermsAggregatorFactory(String name, ValuesSourceConfig valueSourceConfig, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
public SignificantTermsAggregatorFactory(String name, ValuesSourceParser.Input valueSourceInput,
|
||||
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
String executionHint, Query filter, SignificanceHeuristic significanceHeuristic) {
|
||||
|
||||
super(name, SignificantStringTerms.TYPE.name(), valueSourceConfig);
|
||||
super(name, SignificantStringTerms.TYPE.name(), valueSourceInput);
|
||||
this.bucketCountThresholds = bucketCountThresholds;
|
||||
this.includeExclude = includeExclude;
|
||||
this.executionHint = executionHint;
|
||||
this.significanceHeuristic = significanceHeuristic;
|
||||
if (!valueSourceConfig.unmapped()) {
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
private void setFieldInfo() {
|
||||
if (!config.unmapped()) {
|
||||
this.indexedFieldName = config.fieldContext().field();
|
||||
fieldType = SearchContext.current().smartNameFieldType(indexedFieldName);
|
||||
}
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
|
||||
List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
setFieldInfo();
|
||||
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(),
|
||||
bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
|
||||
return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) {
|
||||
|
@ -176,6 +181,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
|
|||
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent,
|
||||
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
throws IOException {
|
||||
setFieldInfo();
|
||||
if (collectsFromSingleBucket == false) {
|
||||
return asMultiBucketAggregator(this, aggregationContext, parent);
|
||||
}
|
||||
|
|
|
@ -77,6 +77,7 @@ public class SignificantTermsParser implements Aggregator.Parser {
|
|||
if (significanceHeuristic == null) {
|
||||
significanceHeuristic = JLHScore.INSTANCE;
|
||||
}
|
||||
return new SignificantTermsAggregatorFactory(aggregationName, vsParser.config(), bucketCountThresholds, aggParser.getIncludeExclude(), aggParser.getExecutionHint(), aggParser.getFilter(), significanceHeuristic);
|
||||
return new SignificantTermsAggregatorFactory(aggregationName, vsParser.input(), bucketCountThresholds,
|
||||
aggParser.getIncludeExclude(), aggParser.getExecutionHint(), aggParser.getFilter(), significanceHeuristic);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -162,10 +162,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
|
||||
private final boolean showTermDocCountError;
|
||||
|
||||
public TermsAggregatorFactory(String name, ValuesSourceConfig config, Terms.Order order,
|
||||
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(), config);
|
||||
super(name, StringTerms.TYPE.name(), input);
|
||||
this.order = order;
|
||||
this.includeExclude = includeExclude;
|
||||
this.executionHint = executionHint;
|
||||
|
|
|
@ -73,7 +73,8 @@ public class TermsParser implements Aggregator.Parser {
|
|||
context.numberOfShards()));
|
||||
}
|
||||
bucketCountThresholds.ensureValidity();
|
||||
return new TermsAggregatorFactory(aggregationName, vsParser.config(), order, bucketCountThresholds, aggParser.getIncludeExclude(), aggParser.getExecutionHint(), aggParser.getCollectionMode(), aggParser.showTermDocCountError());
|
||||
return new TermsAggregatorFactory(aggregationName, vsParser.input(), order, bucketCountThresholds, aggParser.getIncludeExclude(),
|
||||
aggParser.getExecutionHint(), aggParser.getCollectionMode(), aggParser.showTermDocCountError());
|
||||
}
|
||||
|
||||
static Terms.Order resolveOrder(String key, boolean asc) {
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -63,8 +62,8 @@ public abstract class NumericValuesSourceMetricsAggregatorParser<S extends Inter
|
|||
}
|
||||
}
|
||||
|
||||
return createFactory(aggregationName, vsParser.config());
|
||||
return createFactory(aggregationName, vsParser.input());
|
||||
}
|
||||
|
||||
protected abstract AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config);
|
||||
protected abstract AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> config);
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -115,8 +115,8 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
|
||||
|
||||
public Factory(String name, String type, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
|
||||
super(name, type, valuesSourceConfig);
|
||||
public Factory(String name, String type, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput) {
|
||||
super(name, type, valuesSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.avg;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +33,8 @@ public class AvgParser extends NumericValuesSourceMetricsAggregatorParser<Intern
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config) {
|
||||
return new AvgAggregator.Factory(aggregationName, type(), config);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input) {
|
||||
return new AvgAggregator.Factory(aggregationName, type(), input);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,13 +19,15 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.metrics.cardinality;
|
||||
|
||||
import org.elasticsearch.index.mapper.core.Murmur3FieldMapper;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -35,8 +37,8 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory.L
|
|||
|
||||
private final long precisionThreshold;
|
||||
|
||||
CardinalityAggregatorFactory(String name, ValuesSourceConfig config, long precisionThreshold) {
|
||||
super(name, InternalCardinality.TYPE.name(), config);
|
||||
CardinalityAggregatorFactory(String name, ValuesSourceParser.Input<ValuesSource> input, long precisionThreshold) {
|
||||
super(name, InternalCardinality.TYPE.name(), input);
|
||||
this.precisionThreshold = precisionThreshold;
|
||||
}
|
||||
|
||||
|
@ -47,9 +49,19 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory.L
|
|||
@Override
|
||||
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
throws IOException {
|
||||
resolveRehash();
|
||||
return new CardinalityAggregator(name, null, precision(parent), config.formatter(), context, parent, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
private void resolveRehash() {
|
||||
if (rehash == null && config.fieldContext() != null
|
||||
&& config.fieldContext().fieldType() instanceof Murmur3FieldMapper.Murmur3FieldType) {
|
||||
rehash = false;
|
||||
} else if (rehash == null) {
|
||||
rehash = true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent,
|
||||
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
|
|
|
@ -43,7 +43,8 @@ public class CardinalityParser implements Aggregator.Parser {
|
|||
@Override
|
||||
public AggregatorFactory parse(String name, XContentParser parser, SearchContext context) throws IOException {
|
||||
|
||||
ValuesSourceParser<?> vsParser = ValuesSourceParser.any(name, InternalCardinality.TYPE, context).formattable(false).build();
|
||||
ValuesSourceParser<ValuesSource> vsParser = ValuesSourceParser.any(name, InternalCardinality.TYPE, context).formattable(false)
|
||||
.build();
|
||||
|
||||
long precisionThreshold = -1;
|
||||
|
||||
|
@ -68,7 +69,9 @@ public class CardinalityParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
return new CardinalityAggregatorFactory(name, vsParser.config(), precisionThreshold);
|
||||
ValuesSourceParser.Input<ValuesSource> input = vsParser.input();
|
||||
|
||||
return new CardinalityAggregatorFactory(name, input, precisionThreshold);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -160,7 +160,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
|
|||
return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY,
|
||||
Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights);
|
||||
|
@ -170,8 +170,8 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
|
|||
|
||||
private final boolean wrapLongitude;
|
||||
|
||||
protected Factory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude) {
|
||||
super(name, InternalGeoBounds.TYPE.name(), config);
|
||||
protected Factory(String name, ValuesSourceParser.Input<ValuesSource.GeoPoint> input, boolean wrapLongitude) {
|
||||
super(name, InternalGeoBounds.TYPE.name(), input);
|
||||
this.wrapLongitude = wrapLongitude;
|
||||
}
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ public class GeoBoundsParser implements Aggregator.Parser {
|
|||
currentFieldName = parser.currentName();
|
||||
} else if (vsParser.token(currentFieldName, token, parser)) {
|
||||
continue;
|
||||
|
||||
|
||||
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
|
||||
if ("wrap_longitude".equals(currentFieldName) || "wrapLongitude".equals(currentFieldName)) {
|
||||
wrapLongitude = parser.booleanValue();
|
||||
|
@ -64,7 +64,7 @@ public class GeoBoundsParser implements Aggregator.Parser {
|
|||
+ currentFieldName + "].", parser.getTokenLocation());
|
||||
}
|
||||
}
|
||||
return new GeoBoundsAggregator.Factory(aggregationName, vsParser.config(), wrapLongitude);
|
||||
return new GeoBoundsAggregator.Factory(aggregationName, vsParser.input(), wrapLongitude);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -116,8 +116,8 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
|
||||
super(name, InternalMax.TYPE.name(), valuesSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput) {
|
||||
super(name, InternalMax.TYPE.name(), valuesSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.max;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,8 +33,8 @@ public class MaxParser extends NumericValuesSourceMetricsAggregatorParser<Intern
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config) {
|
||||
return new MaxAggregator.Factory(aggregationName, config);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input) {
|
||||
return new MaxAggregator.Factory(aggregationName, input);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -115,8 +115,8 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
|
||||
super(name, InternalMin.TYPE.name(), valuesSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput) {
|
||||
super(name, InternalMin.TYPE.name(), valuesSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.min;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,7 +33,7 @@ public class MinParser extends NumericValuesSourceMetricsAggregatorParser<Intern
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config) {
|
||||
return new MinAggregator.Factory(aggregationName, config);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input) {
|
||||
return new MinAggregator.Factory(aggregationName, input);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ 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.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -50,16 +49,16 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
|
|||
|
||||
@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) {
|
||||
|
@ -103,17 +102,17 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
|
|||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
} else if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||
} else if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||
if (context.parseFieldMatcher().match(currentFieldName, COMPRESSION_FIELD)) {
|
||||
compression = parser.doubleValue();
|
||||
} else {
|
||||
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());
|
||||
}
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
|
||||
+ currentFieldName + "].", parser.getTokenLocation());
|
||||
}
|
||||
}
|
||||
break;
|
||||
case HDR:
|
||||
|
@ -123,7 +122,7 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
|
|||
} else if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||
if (context.parseFieldMatcher().match(currentFieldName, NUMBER_SIGNIFICANT_DIGITS_FIELD)) {
|
||||
numberOfSignificantValueDigits = parser.intValue();
|
||||
} else {
|
||||
} else {
|
||||
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName
|
||||
+ "]: [" + currentFieldName + "].", parser.getTokenLocation());
|
||||
}
|
||||
|
@ -172,11 +171,11 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser {
|
|||
+ "].", parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return buildFactory(context, aggregationName, vsParser.config(), keys, method, compression,
|
||||
return buildFactory(context, aggregationName, vsParser.input(), keys, method, compression,
|
||||
numberOfSignificantValueDigits, keyed);
|
||||
}
|
||||
|
||||
protected abstract AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceConfig<Numeric> config,
|
||||
protected abstract AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceParser.Input<Numeric> config,
|
||||
double[] cdfValues, PercentilesMethod method, Double compression, Integer numberOfSignificantValueDigits, boolean keyed);
|
||||
|
||||
protected abstract ParseField keysField();
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercenti
|
|||
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.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
/**
|
||||
|
@ -48,17 +48,17 @@ public class PercentileRanksParser extends AbstractPercentilesParser {
|
|||
protected ParseField keysField() {
|
||||
return VALUES_FIELD;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceConfig<Numeric> valuesSourceConfig,
|
||||
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);
|
||||
}
|
||||
if (method == PercentilesMethod.TDIGEST) {
|
||||
return new TDigestPercentileRanksAggregator.Factory(aggregationName, valuesSourceConfig, keys, compression, keyed);
|
||||
return new TDigestPercentileRanksAggregator.Factory(aggregationName, valuesSourceInput, keys, compression, keyed);
|
||||
} else if (method == PercentilesMethod.HDR) {
|
||||
return new HDRPercentileRanksAggregator.Factory(aggregationName, valuesSourceConfig, keys, numberOfSignificantValueDigits,
|
||||
return new HDRPercentileRanksAggregator.Factory(aggregationName, valuesSourceInput, keys, numberOfSignificantValueDigits,
|
||||
keyed);
|
||||
} else {
|
||||
throw new AssertionError();
|
||||
|
|
|
@ -24,7 +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.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
/**
|
||||
|
@ -49,15 +49,15 @@ public class PercentilesParser extends AbstractPercentilesParser {
|
|||
protected ParseField keysField() {
|
||||
return PERCENTS_FIELD;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceConfig<Numeric> valuesSourceConfig,
|
||||
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;
|
||||
}
|
||||
if (method == PercentilesMethod.TDIGEST) {
|
||||
return new TDigestPercentilesAggregator.Factory(aggregationName, valuesSourceConfig, keys, compression, keyed);
|
||||
return new TDigestPercentilesAggregator.Factory(aggregationName, valuesSourceInput, keys, compression, keyed);
|
||||
} else if (method == PercentilesMethod.HDR) {
|
||||
return new HDRPercentilesAggregator.Factory(aggregationName, valuesSourceConfig, keys, numberOfSignificantValueDigits, keyed);
|
||||
} else {
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -75,9 +75,9 @@ public class TDigestPercentileRanksAggregator extends AbstractTDigestPercentiles
|
|||
private final double compression;
|
||||
private final boolean keyed;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig,
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput,
|
||||
double[] values, double compression, boolean keyed) {
|
||||
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceConfig);
|
||||
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput);
|
||||
this.values = values;
|
||||
this.compression = compression;
|
||||
this.keyed = keyed;
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -75,9 +75,9 @@ public class TDigestPercentilesAggregator extends AbstractTDigestPercentilesAggr
|
|||
private final double compression;
|
||||
private final boolean keyed;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig,
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput,
|
||||
double[] percents, double compression, boolean keyed) {
|
||||
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceConfig);
|
||||
super(name, InternalTDigestPercentiles.TYPE.name(), valuesSourceInput);
|
||||
this.percents = percents;
|
||||
this.compression = compression;
|
||||
this.keyed = keyed;
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -157,8 +157,8 @@ public class StatsAggregator extends NumericMetricsAggregator.MultiValue {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
|
||||
super(name, InternalStats.TYPE.name(), valuesSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput) {
|
||||
super(name, InternalStats.TYPE.name(), valuesSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.stats;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,7 +33,7 @@ public class StatsParser extends NumericValuesSourceMetricsAggregatorParser<Inte
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config) {
|
||||
return new StatsAggregator.Factory(aggregationName, config);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input) {
|
||||
return new StatsAggregator.Factory(aggregationName, input);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -192,8 +192,8 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
|
|||
|
||||
private final double sigma;
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig, double sigma) {
|
||||
super(name, InternalExtendedStats.TYPE.name(), valuesSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput, double sigma) {
|
||||
super(name, InternalExtendedStats.TYPE.name(), valuesSourceInput);
|
||||
|
||||
this.sigma = sigma;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.elasticsearch.search.SearchParseException;
|
|||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -42,8 +41,8 @@ public class ExtendedStatsParser implements Aggregator.Parser {
|
|||
return InternalExtendedStats.TYPE.name();
|
||||
}
|
||||
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config, double sigma) {
|
||||
return new ExtendedStatsAggregator.Factory(aggregationName, config, sigma);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input, double sigma) {
|
||||
return new ExtendedStatsAggregator.Factory(aggregationName, input, sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -78,6 +77,6 @@ public class ExtendedStatsParser implements Aggregator.Parser {
|
|||
throw new SearchParseException(context, "[sigma] must not be negative. Value provided was" + sigma, parser.getTokenLocation());
|
||||
}
|
||||
|
||||
return createFactory(aggregationName, vsParser.config(), sigma);
|
||||
return createFactory(aggregationName, vsParser.input(), sigma);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -107,8 +107,8 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
|
||||
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
|
||||
super(name, InternalSum.TYPE.name(), valuesSourceConfig);
|
||||
public Factory(String name, ValuesSourceParser.Input<ValuesSource.Numeric> valuesSourceInput) {
|
||||
super(name, InternalSum.TYPE.name(), valuesSourceInput);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics.sum;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericValuesSourceMetricsAggregatorParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -33,7 +33,7 @@ public class SumParser extends NumericValuesSourceMetricsAggregatorParser<Intern
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceConfig<ValuesSource.Numeric> config) {
|
||||
return new SumAggregator.Factory(aggregationName, config);
|
||||
protected AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> input) {
|
||||
return new SumAggregator.Factory(aggregationName, input);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -110,8 +110,8 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
|
||||
public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> {
|
||||
|
||||
public Factory(String name, ValuesSourceConfig<VS> config) {
|
||||
super(name, InternalValueCount.TYPE.name(), config);
|
||||
public Factory(String name, ValuesSourceParser.Input<VS> input) {
|
||||
super(name, InternalValueCount.TYPE.name(), input);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -54,6 +54,6 @@ public class ValueCountParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
return new ValueCountAggregator.Factory(aggregationName, vsParser.config());
|
||||
return new ValueCountAggregator.Factory(aggregationName, vsParser.input());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,18 @@
|
|||
*/
|
||||
package org.elasticsearch.search.aggregations.support;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
|
||||
import org.elasticsearch.index.mapper.core.DateFieldMapper;
|
||||
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
|
||||
import org.elasticsearch.index.mapper.ip.IpFieldMapper;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptContext;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.AggregationInitializationException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
|
@ -25,6 +37,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -37,8 +50,8 @@ 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, ValuesSourceConfig<VS> valuesSourceConfig) {
|
||||
super(name, type, valuesSourceConfig);
|
||||
protected LeafOnly(String name, String type, ValuesSourceParser.Input<VS> input) {
|
||||
super(name, type, input);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -48,10 +61,20 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
|
|||
}
|
||||
|
||||
protected ValuesSourceConfig<VS> config;
|
||||
private ValuesSourceParser.Input<VS> input;
|
||||
|
||||
protected ValuesSourceAggregatorFactory(String name, String type, ValuesSourceConfig<VS> config) {
|
||||
protected ValuesSourceAggregatorFactory(String name, String type, ValuesSourceParser.Input<VS> input) {
|
||||
super(name, type);
|
||||
this.config = config;
|
||||
this.input = input;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInit(AggregationContext context) {
|
||||
this.config = config(input, context);
|
||||
if (config == null || !config.valid()) {
|
||||
resolveValuesSourceConfigFromAncestors(name, this.parent, config.valueSourceType());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -66,9 +89,100 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> ext
|
|||
|
||||
@Override
|
||||
public void doValidate() {
|
||||
if (config == null || !config.valid()) {
|
||||
resolveValuesSourceConfigFromAncestors(name, parent, config.valueSourceType());
|
||||
}
|
||||
|
||||
public ValuesSourceConfig<VS> config(ValuesSourceParser.Input<VS> input, AggregationContext context) {
|
||||
|
||||
ValueType valueType = input.valueType != null ? input.valueType : input.targetValueType;
|
||||
|
||||
if (input.field == null) {
|
||||
if (input.script == null) {
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig(ValuesSource.class);
|
||||
config.format = resolveFormat(null, valueType);
|
||||
return config;
|
||||
}
|
||||
Class valuesSourceType = valueType != null ? (Class<VS>) valueType.getValuesSourceType() : input.valuesSourceType;
|
||||
if (valuesSourceType == null || valuesSourceType == ValuesSource.class) {
|
||||
// the specific value source type is undefined, but for scripts,
|
||||
// we need to have a specific value source
|
||||
// type to know how to handle the script values, so we fallback
|
||||
// on Bytes
|
||||
valuesSourceType = ValuesSource.Bytes.class;
|
||||
}
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig<VS>(valuesSourceType);
|
||||
config.missing = input.missing;
|
||||
config.format = resolveFormat(input.format, valueType);
|
||||
config.script = createScript(input.script, context.searchContext());
|
||||
config.scriptValueType = valueType;
|
||||
return config;
|
||||
}
|
||||
|
||||
MappedFieldType fieldType = context.searchContext().smartNameFieldTypeFromAnyType(input.field);
|
||||
if (fieldType == null) {
|
||||
Class<VS> valuesSourceType = valueType != null ? (Class<VS>) valueType.getValuesSourceType() : input.valuesSourceType;
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig<>(valuesSourceType);
|
||||
config.missing = input.missing;
|
||||
config.format = resolveFormat(input.format, valueType);
|
||||
config.unmapped = true;
|
||||
if (valueType != null) {
|
||||
// todo do we really need this for unmapped?
|
||||
config.scriptValueType = valueType;
|
||||
}
|
||||
return config;
|
||||
}
|
||||
|
||||
IndexFieldData<?> indexFieldData = context.searchContext().fieldData().getForField(fieldType);
|
||||
|
||||
ValuesSourceConfig config;
|
||||
if (input.valuesSourceType == ValuesSource.class) {
|
||||
if (indexFieldData instanceof IndexNumericFieldData) {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.Numeric.class);
|
||||
} else if (indexFieldData instanceof IndexGeoPointFieldData) {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.GeoPoint.class);
|
||||
} else {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.Bytes.class);
|
||||
}
|
||||
} else {
|
||||
config = new ValuesSourceConfig(input.valuesSourceType);
|
||||
}
|
||||
|
||||
config.fieldContext = new FieldContext(input.field, indexFieldData, fieldType);
|
||||
config.missing = input.missing;
|
||||
config.script = createScript(input.script, context.searchContext());
|
||||
config.format = resolveFormat(input.format, fieldType);
|
||||
return config;
|
||||
}
|
||||
|
||||
private SearchScript createScript(Script script, SearchContext context) {
|
||||
return script == null ? null : context.scriptService().search(context.lookup(), script, ScriptContext.Standard.AGGS);
|
||||
}
|
||||
|
||||
private static ValueFormat resolveFormat(@Nullable String format, @Nullable ValueType valueType) {
|
||||
if (valueType == null) {
|
||||
return ValueFormat.RAW; // we can't figure it out
|
||||
}
|
||||
ValueFormat valueFormat = valueType.defaultFormat;
|
||||
if (valueFormat != null && valueFormat instanceof ValueFormat.Patternable && format != null) {
|
||||
return ((ValueFormat.Patternable) valueFormat).create(format);
|
||||
}
|
||||
return valueFormat;
|
||||
}
|
||||
|
||||
private static ValueFormat resolveFormat(@Nullable String format, MappedFieldType fieldType) {
|
||||
if (fieldType instanceof DateFieldMapper.DateFieldType) {
|
||||
return format != null ? ValueFormat.DateTime.format(format) : ValueFormat.DateTime
|
||||
.mapper((DateFieldMapper.DateFieldType) fieldType);
|
||||
}
|
||||
if (fieldType instanceof IpFieldMapper.IpFieldType) {
|
||||
return ValueFormat.IPv4;
|
||||
}
|
||||
if (fieldType instanceof BooleanFieldMapper.BooleanFieldType) {
|
||||
return ValueFormat.BOOLEAN;
|
||||
}
|
||||
if (fieldType instanceof NumberFieldMapper.NumberFieldType) {
|
||||
return format != null ? ValueFormat.Number.format(format) : ValueFormat.RAW;
|
||||
}
|
||||
return ValueFormat.RAW;
|
||||
}
|
||||
|
||||
protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
|
||||
|
|
|
@ -19,26 +19,14 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.support;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
|
||||
import org.elasticsearch.index.mapper.core.DateFieldMapper;
|
||||
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
|
||||
import org.elasticsearch.index.mapper.ip.IpFieldMapper;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.Script.ScriptField;
|
||||
import org.elasticsearch.script.ScriptContext;
|
||||
import org.elasticsearch.script.ScriptParameterParser;
|
||||
import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
|
@ -70,39 +58,43 @@ public class ValuesSourceParser<VS extends ValuesSource> {
|
|||
return new Builder<>(aggName, aggType, context, ValuesSource.GeoPoint.class).targetValueType(ValueType.GEOPOINT).scriptable(false);
|
||||
}
|
||||
|
||||
public static class Input {
|
||||
private String field = null;
|
||||
private Script script = null;
|
||||
public static class Input<VS> {
|
||||
String field = null;
|
||||
Script script = null;
|
||||
@Deprecated
|
||||
private Map<String, Object> params = null; // TODO Remove in 3.0
|
||||
private ValueType valueType = null;
|
||||
private String format = null;
|
||||
private Object missing = null;
|
||||
private DateTimeZone timezone = DateTimeZone.UTC;
|
||||
Map<String, Object> params = null; // TODO Remove in 3.0
|
||||
ValueType valueType = null;
|
||||
String format = null;
|
||||
Object missing = null;
|
||||
Class<VS> valuesSourceType = null;
|
||||
ValueType targetValueType = null;
|
||||
|
||||
public boolean valid() {
|
||||
return field != null || script != null;
|
||||
}
|
||||
|
||||
public DateTimeZone timezone() {
|
||||
return this.timezone;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final String aggName;
|
||||
private final InternalAggregation.Type aggType;
|
||||
private final SearchContext context;
|
||||
private final Class<VS> valuesSourceType;
|
||||
|
||||
private boolean scriptable = true;
|
||||
private boolean formattable = false;
|
||||
private boolean timezoneAware = false;
|
||||
private ValueType targetValueType = null;
|
||||
private ScriptParameterParser scriptParameterParser = new ScriptParameterParser();
|
||||
|
||||
private Input input = new Input();
|
||||
private Input<VS> input = new Input<VS>();
|
||||
|
||||
private ValuesSourceParser(String aggName, InternalAggregation.Type aggType, SearchContext context, Class<VS> valuesSourceType) {
|
||||
this.aggName = aggName;
|
||||
this.aggType = aggType;
|
||||
this.context = context;
|
||||
this.valuesSourceType = valuesSourceType;
|
||||
input.valuesSourceType = valuesSourceType;
|
||||
}
|
||||
|
||||
public boolean token(String currentFieldName, XContentParser.Token token, XContentParser parser) throws IOException {
|
||||
|
@ -120,11 +112,10 @@ public class ValuesSourceParser<VS extends ValuesSource> {
|
|||
} else if (scriptable) {
|
||||
if ("value_type".equals(currentFieldName) || "valueType".equals(currentFieldName)) {
|
||||
input.valueType = ValueType.resolveForScript(parser.text());
|
||||
if (targetValueType != null && input.valueType.isNotA(targetValueType)) {
|
||||
throw new SearchParseException(context, aggType.name() + " aggregation [" + aggName +
|
||||
"] was configured with an incompatible value type [" + input.valueType + "]. [" + aggType +
|
||||
"] aggregation can only work on value of type [" + targetValueType + "]",
|
||||
parser.getTokenLocation());
|
||||
if (input.targetValueType != null && input.valueType.isNotA(input.targetValueType)) {
|
||||
throw new SearchParseException(context, aggType.name() + " aggregation [" + aggName
|
||||
+ "] was configured with an incompatible value type [" + input.valueType + "]. [" + aggType
|
||||
+ "] aggregation can only work on value of type [" + input.targetValueType + "]", parser.getTokenLocation());
|
||||
}
|
||||
} else if (!scriptParameterParser.token(currentFieldName, token, parser, context.parseFieldMatcher())) {
|
||||
return false;
|
||||
|
@ -157,9 +148,9 @@ public class ValuesSourceParser<VS extends ValuesSource> {
|
|||
return false;
|
||||
}
|
||||
|
||||
public ValuesSourceConfig<VS> config() {
|
||||
|
||||
if (input.script == null) { // Didn't find anything using the new API so try using the old one instead
|
||||
public Input<VS> input() {
|
||||
if (input.script == null) { // Didn't find anything using the new API so
|
||||
// try using the old one instead
|
||||
ScriptParameterValue scriptValue = scriptParameterParser.getDefaultScriptParameterValue();
|
||||
if (scriptValue != null) {
|
||||
if (input.params == null) {
|
||||
|
@ -169,98 +160,8 @@ public class ValuesSourceParser<VS extends ValuesSource> {
|
|||
}
|
||||
}
|
||||
|
||||
ValueType valueType = input.valueType != null ? input.valueType : targetValueType;
|
||||
|
||||
if (input.field == null) {
|
||||
if (input.script == null) {
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig(ValuesSource.class);
|
||||
config.format = resolveFormat(null, valueType);
|
||||
return config;
|
||||
return input;
|
||||
}
|
||||
Class valuesSourceType = valueType != null ? (Class<VS>) valueType.getValuesSourceType() : this.valuesSourceType;
|
||||
if (valuesSourceType == null || valuesSourceType == ValuesSource.class) {
|
||||
// the specific value source type is undefined, but for scripts, we need to have a specific value source
|
||||
// type to know how to handle the script values, so we fallback on Bytes
|
||||
valuesSourceType = ValuesSource.Bytes.class;
|
||||
}
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig<VS>(valuesSourceType);
|
||||
config.missing = input.missing;
|
||||
config.format = resolveFormat(input.format, valueType);
|
||||
config.script = createScript();
|
||||
config.scriptValueType = valueType;
|
||||
return config;
|
||||
}
|
||||
|
||||
MappedFieldType fieldType = context.smartNameFieldTypeFromAnyType(input.field);
|
||||
if (fieldType == null) {
|
||||
Class<VS> valuesSourceType = valueType != null ? (Class<VS>) valueType.getValuesSourceType() : this.valuesSourceType;
|
||||
ValuesSourceConfig<VS> config = new ValuesSourceConfig<>(valuesSourceType);
|
||||
config.missing = input.missing;
|
||||
config.format = resolveFormat(input.format, valueType);
|
||||
config.unmapped = true;
|
||||
if (valueType != null) {
|
||||
// todo do we really need this for unmapped?
|
||||
config.scriptValueType = valueType;
|
||||
}
|
||||
return config;
|
||||
}
|
||||
|
||||
IndexFieldData<?> indexFieldData = context.fieldData().getForField(fieldType);
|
||||
|
||||
ValuesSourceConfig config;
|
||||
if (valuesSourceType == ValuesSource.class) {
|
||||
if (indexFieldData instanceof IndexNumericFieldData) {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.Numeric.class);
|
||||
} else if (indexFieldData instanceof IndexGeoPointFieldData) {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.GeoPoint.class);
|
||||
} else {
|
||||
config = new ValuesSourceConfig<>(ValuesSource.Bytes.class);
|
||||
}
|
||||
} else {
|
||||
config = new ValuesSourceConfig(valuesSourceType);
|
||||
}
|
||||
|
||||
config.fieldContext = new FieldContext(input.field, indexFieldData, fieldType);
|
||||
config.missing = input.missing;
|
||||
config.script = createScript();
|
||||
config.format = resolveFormat(input.format, input.timezone, fieldType);
|
||||
return config;
|
||||
}
|
||||
|
||||
private SearchScript createScript() {
|
||||
return input.script == null ? null : context.scriptService().search(context.lookup(), input.script, ScriptContext.Standard.AGGS, Collections.emptyMap());
|
||||
}
|
||||
|
||||
private static ValueFormat resolveFormat(@Nullable String format, @Nullable ValueType valueType) {
|
||||
if (valueType == null) {
|
||||
return ValueFormat.RAW; // we can't figure it out
|
||||
}
|
||||
ValueFormat valueFormat = valueType.defaultFormat;
|
||||
if (valueFormat != null && valueFormat instanceof ValueFormat.Patternable && format != null) {
|
||||
return ((ValueFormat.Patternable) valueFormat).create(format);
|
||||
}
|
||||
return valueFormat;
|
||||
}
|
||||
|
||||
private static ValueFormat resolveFormat(@Nullable String format, @Nullable DateTimeZone timezone, MappedFieldType fieldType) {
|
||||
if (fieldType instanceof DateFieldMapper.DateFieldType) {
|
||||
return format != null ? ValueFormat.DateTime.format(format, timezone) : ValueFormat.DateTime.mapper((DateFieldMapper.DateFieldType) fieldType, timezone);
|
||||
}
|
||||
if (fieldType instanceof IpFieldMapper.IpFieldType) {
|
||||
return ValueFormat.IPv4;
|
||||
}
|
||||
if (fieldType instanceof BooleanFieldMapper.BooleanFieldType) {
|
||||
return ValueFormat.BOOLEAN;
|
||||
}
|
||||
if (fieldType instanceof NumberFieldMapper.NumberFieldType) {
|
||||
return format != null ? ValueFormat.Number.format(format) : ValueFormat.RAW;
|
||||
}
|
||||
return ValueFormat.RAW;
|
||||
}
|
||||
|
||||
public Input input() {
|
||||
return this.input;
|
||||
}
|
||||
|
||||
public static class Builder<VS extends ValuesSource> {
|
||||
|
||||
|
@ -286,7 +187,7 @@ public class ValuesSourceParser<VS extends ValuesSource> {
|
|||
}
|
||||
|
||||
public Builder<VS> targetValueType(ValueType valueType) {
|
||||
parser.targetValueType = valueType;
|
||||
parser.input.targetValueType = valueType;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase {
|
|||
IndexService index = createIndex("idx");
|
||||
client().prepareIndex("idx", "type", "1").setSource("f", 5).execute().get();
|
||||
client().admin().indices().prepareRefresh("idx").get();
|
||||
|
||||
|
||||
// simple field aggregation, no scores needed
|
||||
String fieldAgg = "{ \"my_terms\": {\"terms\": {\"field\": \"f\"}}}";
|
||||
assertFalse(needsScores(index, fieldAgg));
|
||||
|
@ -63,7 +63,8 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase {
|
|||
SearchContext searchContext = createSearchContext(index);
|
||||
final AggregatorFactories factories = parser.parseAggregators(aggParser, searchContext);
|
||||
AggregationContext aggregationContext = new AggregationContext(searchContext);
|
||||
final Aggregator[] aggregators = factories.createTopLevelAggregators(aggregationContext);
|
||||
factories.init(aggregationContext);
|
||||
final Aggregator[] aggregators = factories.createTopLevelAggregators();
|
||||
assertEquals(1, aggregators.length);
|
||||
return aggregators[0].needsScores();
|
||||
}
|
||||
|
|
|
@ -122,7 +122,8 @@ public class NestedAggregatorTests extends ESSingleNodeTestCase {
|
|||
builder.addAggregator(new NestedAggregator.Factory("test", "nested_field"));
|
||||
AggregatorFactories factories = builder.build();
|
||||
searchContext.aggregations(new SearchContextAggregations(factories));
|
||||
Aggregator[] aggs = factories.createTopLevelAggregators(context);
|
||||
factories.init(context);
|
||||
Aggregator[] aggs = factories.createTopLevelAggregators();
|
||||
BucketCollector collector = BucketCollector.wrap(Arrays.asList(aggs));
|
||||
collector.preCollection();
|
||||
// A regular search always exclude nested docs, so we use NonNestedDocsFilter.INSTANCE here (otherwise MatchAllDocsQuery would be sufficient)
|
||||
|
|
Loading…
Reference in New Issue