Make ValuesSourceConfig behave like a config object (#57762) (#58012)

This commit is contained in:
Mark Tozzi 2020-06-11 17:23:55 -04:00 committed by GitHub
parent 5138c0c045
commit 36f551bdb4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
84 changed files with 691 additions and 553 deletions

View File

@ -50,9 +50,9 @@ public abstract class ArrayValuesSourceAggregatorFactory
HashMap<String, ValuesSource> valuesSources = new HashMap<>(); HashMap<String, ValuesSource> valuesSources = new HashMap<>();
for (Map.Entry<String, ValuesSourceConfig> config : configs.entrySet()) { for (Map.Entry<String, ValuesSourceConfig> config : configs.entrySet()) {
ValuesSource vs = config.getValue().toValuesSource(); ValuesSourceConfig vsc = config.getValue();
if (vs != null) { if (vsc.hasValues()) {
valuesSources.put(config.getKey(), vs); valuesSources.put(config.getKey(), vsc.getValuesSource());
} }
} }
if (valuesSources.isEmpty()) { if (valuesSources.isEmpty()) {

View File

@ -61,6 +61,27 @@ public class MatrixStatsAggregatorTests extends AggregatorTestCase {
} }
} }
public void testUnmapped() throws Exception {
MappedFieldType ft =
new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
ft.setName("field");
try (Directory directory = newDirectory();
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) {
if (randomBoolean()) {
indexWriter.addDocument(Collections.singleton(new StringField("another_field", "value", Field.Store.NO)));
}
try (IndexReader reader = indexWriter.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
MatrixStatsAggregationBuilder aggBuilder = new MatrixStatsAggregationBuilder("my_agg")
.fields(Collections.singletonList("bogus"));
InternalMatrixStats stats = search(searcher, new MatchAllDocsQuery(), aggBuilder, ft);
assertNull(stats.getStats());
assertFalse(MatrixAggregationInspectionHelper.hasValue(stats));
}
}
}
public void testTwoFields() throws Exception { public void testTwoFields() throws Exception {
String fieldA = "a"; String fieldA = "a";
MappedFieldType ftA = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE); MappedFieldType ftA = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);

View File

@ -68,11 +68,11 @@ public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource rawValuesSource, protected Aggregator doCreateInternal(SearchContext searchContext, Aggregator parent,
SearchContext searchContext, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
ValuesSource rawValuesSource = config.getValuesSource();
if (rawValuesSource instanceof WithOrdinals == false) { if (rawValuesSource instanceof WithOrdinals == false) {
throw new AggregationExecutionException("ValuesSource type " + rawValuesSource.toString() + throw new AggregationExecutionException("ValuesSource type " + rawValuesSource.toString() +
"is not supported for aggregation " + this.name()); "is not supported for aggregation " + this.name());

View File

@ -69,11 +69,11 @@ public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource rawValuesSource, protected Aggregator doCreateInternal(SearchContext searchContext, Aggregator children,
SearchContext searchContext, Aggregator children,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
ValuesSource rawValuesSource = config.getValuesSource();
if (rawValuesSource instanceof WithOrdinals == false) { if (rawValuesSource instanceof WithOrdinals == false) {
throw new AggregationExecutionException("ValuesSource type " + rawValuesSource.toString() + throw new AggregationExecutionException("ValuesSource type " + rawValuesSource.toString() +
"is not supported for aggregation " + this.name()); "is not supported for aggregation " + this.name());

View File

@ -249,7 +249,7 @@ public class DateHistogramValuesSourceBuilder
@Override @Override
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException { protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException {
Rounding rounding = dateHistogramInterval.createRounding(timeZone(), offset); Rounding rounding = dateHistogramInterval.createRounding(timeZone(), offset);
ValuesSource orig = config.toValuesSource(); ValuesSource orig = config.hasValues() ? config.getValuesSource() : null;
if (orig == null) { if (orig == null) {
orig = ValuesSource.Numeric.EMPTY; orig = ValuesSource.Numeric.EMPTY;
} }

View File

@ -129,7 +129,7 @@ public class GeoTileGridValuesSourceBuilder extends CompositeValuesSourceBuilder
@Override @Override
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException { protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException {
ValuesSource orig = config.toValuesSource(); ValuesSource orig = config.hasValues() ? config.getValuesSource() : null;
if (orig == null) { if (orig == null) {
orig = ValuesSource.GeoPoint.EMPTY; orig = ValuesSource.GeoPoint.EMPTY;
} }

View File

@ -111,7 +111,7 @@ public class HistogramValuesSourceBuilder extends CompositeValuesSourceBuilder<H
@Override @Override
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException { protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException {
ValuesSource orig = config.toValuesSource(); ValuesSource orig = config.hasValues() ? config.getValuesSource() : null;
if (orig == null) { if (orig == null) {
orig = ValuesSource.Numeric.EMPTY; orig = ValuesSource.Numeric.EMPTY;
} }

View File

@ -71,7 +71,7 @@ public class TermsValuesSourceBuilder extends CompositeValuesSourceBuilder<Terms
@Override @Override
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException { protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig config) throws IOException {
ValuesSource vs = config.toValuesSource(); ValuesSource vs = config.hasValues() ? config.getValuesSource() : null;
if (vs == null) { if (vs == null) {
// The field is unmapped so we use a value source that can parse any type of values. // The field is unmapped so we use a value source that can parse any type of values.
// This is needed because the after values are parsed even when there are no values to process. // This is needed because the after values are parsed even when there are no values to process.

View File

@ -74,18 +74,17 @@ public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory
} }
@Override @Override
protected Aggregator doCreateInternal(final ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry() AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry()
.getAggregator(config, GeoHashGridAggregationBuilder.NAME); .getAggregator(config, GeoHashGridAggregationBuilder.NAME);
if (aggregatorSupplier instanceof GeoGridAggregatorSupplier == false) { if (aggregatorSupplier instanceof GeoGridAggregatorSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected " throw new AggregationExecutionException("Registry miss-match - expected "
+ GeoGridAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]"); + GeoGridAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]");
} }
return ((GeoGridAggregatorSupplier) aggregatorSupplier).build(name, factories, valuesSource, precision, geoBoundingBox, return ((GeoGridAggregatorSupplier) aggregatorSupplier).build(name, factories, config.getValuesSource(), precision, geoBoundingBox,
requiredSize, shardSize, searchContext, parent, collectsFromSingleBucket, metadata); requiredSize, shardSize, searchContext, parent, collectsFromSingleBucket, metadata);
} }

View File

@ -72,18 +72,17 @@ public class GeoTileGridAggregatorFactory extends ValuesSourceAggregatorFactory
} }
@Override @Override
protected Aggregator doCreateInternal(final ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry() AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry()
.getAggregator(config, GeoTileGridAggregationBuilder.NAME); .getAggregator(config, GeoTileGridAggregationBuilder.NAME);
if (aggregatorSupplier instanceof GeoGridAggregatorSupplier == false) { if (aggregatorSupplier instanceof GeoGridAggregatorSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected " throw new AggregationExecutionException("Registry miss-match - expected "
+ GeoGridAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]"); + GeoGridAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]");
} }
return ((GeoGridAggregatorSupplier) aggregatorSupplier).build(name, factories, valuesSource, precision, geoBoundingBox, return ((GeoGridAggregatorSupplier) aggregatorSupplier).build(name, factories, config.getValuesSource(), precision, geoBoundingBox,
requiredSize, shardSize, searchContext, parent, collectsFromSingleBucket, metadata); requiredSize, shardSize, searchContext, parent, collectsFromSingleBucket, metadata);
} }

View File

@ -22,7 +22,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Rounding; import org.elasticsearch.common.Rounding;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.LongHash;
@ -38,6 +37,7 @@ import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -48,7 +48,7 @@ import java.util.function.Function;
/** /**
* An aggregator for date values that attempts to return a specific number of * An aggregator for date values that attempts to return a specific number of
* buckets, reconfiguring how it rounds dates to buckets on the fly as new * buckets, reconfiguring how it rounds dates to buckets on the fly as new
* data arrives. * data arrives.
*/ */
class AutoDateHistogramAggregator extends DeferableBucketAggregator { class AutoDateHistogramAggregator extends DeferableBucketAggregator {
@ -63,14 +63,23 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator {
private int targetBuckets; private int targetBuckets;
private MergingBucketsDeferringCollector deferringCollector; private MergingBucketsDeferringCollector deferringCollector;
AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, RoundingInfo[] roundingInfos, AutoDateHistogramAggregator(
Function<Rounding, Rounding.Prepared> roundingPreparer, @Nullable ValuesSource valuesSource, DocValueFormat formatter, String name,
SearchContext aggregationContext, Aggregator parent, Map<String, Object> metadata) throws IOException { AggregatorFactories factories,
int numBuckets,
RoundingInfo[] roundingInfos,
Function<Rounding, Rounding.Prepared> roundingPreparer,
ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, factories, aggregationContext, parent, metadata); super(name, factories, aggregationContext, parent, metadata);
this.targetBuckets = numBuckets; this.targetBuckets = numBuckets;
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: Remove null usage here, by using a different aggregator for create
this.formatter = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
this.formatter = valuesSourceConfig.format();
this.roundingInfos = roundingInfos; this.roundingInfos = roundingInfos;
this.roundingPreparer = roundingPreparer; this.roundingPreparer = roundingPreparer;
preparedRounding = roundingPreparer.apply(roundingInfos[roundingIdx].rounding); preparedRounding = roundingPreparer.apply(roundingInfos[roundingIdx].rounding);

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -64,11 +63,10 @@ public final class AutoDateHistogramAggregatorFactory extends ValuesSourceAggreg
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
if (collectsFromSingleBucket == false) { if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, searchContext, parent); return asMultiBucketAggregator(this, searchContext, parent);
} }
@ -79,16 +77,16 @@ public final class AutoDateHistogramAggregatorFactory extends ValuesSourceAggreg
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
Function<Rounding, Rounding.Prepared> roundingPreparer = Function<Rounding, Rounding.Prepared> roundingPreparer =
valuesSource.roundingPreparer(searchContext.getQueryShardContext().getIndexReader()); config.getValuesSource().roundingPreparer(searchContext.getQueryShardContext().getIndexReader());
return ((AutoDateHistogramAggregatorSupplier) aggregatorSupplier).build(name, factories, numBuckets, roundingInfos, return ((AutoDateHistogramAggregatorSupplier) aggregatorSupplier).build(name, factories, numBuckets, roundingInfos,
roundingPreparer, valuesSource, config.format(), searchContext, parent, metadata); roundingPreparer, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, Rounding::prepareForUnknown, null, return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, Rounding::prepareForUnknown,
config.format(), searchContext, parent, metadata); config, searchContext, parent, metadata);
} }
} }

View File

@ -21,11 +21,10 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Rounding; import org.elasticsearch.common.Rounding;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -41,8 +40,7 @@ public interface AutoDateHistogramAggregatorSupplier extends AggregatorSupplier
AutoDateHistogramAggregationBuilder.RoundingInfo[] roundingInfos, AutoDateHistogramAggregationBuilder.RoundingInfo[] roundingInfos,
@Nullable @Nullable
Function<Rounding, Rounding.Prepared> roundingPreparer, Function<Rounding, Rounding.Prepared> roundingPreparer,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata Map<String, Object> metadata

View File

@ -21,12 +21,11 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Rounding; import org.elasticsearch.common.Rounding;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -42,8 +41,7 @@ public interface DateHistogramAggregationSupplier extends AggregatorSupplier {
boolean keyed, boolean keyed,
long minDocCount, long minDocCount,
@Nullable ExtendedBounds extendedBounds, @Nullable ExtendedBounds extendedBounds,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,

View File

@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -65,11 +66,21 @@ class DateHistogramAggregator extends BucketsAggregator {
private final LongKeyedBucketOrds bucketOrds; private final LongKeyedBucketOrds bucketOrds;
DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, Rounding.Prepared preparedRounding, DateHistogramAggregator(
BucketOrder order, boolean keyed, String name,
long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource valuesSource, AggregatorFactories factories,
DocValueFormat formatter, SearchContext aggregationContext, Rounding rounding,
Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException { Rounding.Prepared preparedRounding,
BucketOrder order,
boolean keyed,
long minDocCount,
@Nullable ExtendedBounds extendedBounds,
ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext,
Aggregator parent,
boolean collectsFromSingleBucket,
Map<String, Object> metadata
) throws IOException {
super(name, factories, aggregationContext, parent, metadata); super(name, factories, aggregationContext, parent, metadata);
this.rounding = rounding; this.rounding = rounding;
@ -79,8 +90,9 @@ class DateHistogramAggregator extends BucketsAggregator {
this.keyed = keyed; this.keyed = keyed;
this.minDocCount = minDocCount; this.minDocCount = minDocCount;
this.extendedBounds = extendedBounds; this.extendedBounds = extendedBounds;
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: Stop using null here
this.formatter = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
this.formatter = valuesSourceConfig.format();
bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket);
} }
@ -129,7 +141,7 @@ class DateHistogramAggregator extends BucketsAggregator {
@Override @Override
public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException {
return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds,
(bucketValue, docCount, subAggregationResults) -> { (bucketValue, docCount, subAggregationResults) -> {
return new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults); return new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults);
}, buckets -> { }, buckets -> {

View File

@ -29,7 +29,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -76,18 +75,21 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(
SearchContext searchContext, SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
DateHistogramAggregationBuilder.NAME); DateHistogramAggregationBuilder.NAME);
if (aggregatorSupplier instanceof DateHistogramAggregationSupplier == false) { if (aggregatorSupplier instanceof DateHistogramAggregationSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected DateHistogramAggregationSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected DateHistogramAggregationSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
Rounding.Prepared preparedRounding = valuesSource.roundingPreparer(queryShardContext.getIndexReader()).apply(shardRounding); // TODO: Is there a reason not to get the prepared rounding in the supplier itself?
Rounding.Prepared preparedRounding = config.getValuesSource()
.roundingPreparer(queryShardContext.getIndexReader())
.apply(shardRounding);
return ((DateHistogramAggregationSupplier) aggregatorSupplier).build( return ((DateHistogramAggregationSupplier) aggregatorSupplier).build(
name, name,
factories, factories,
@ -97,8 +99,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
keyed, keyed,
minDocCount, minDocCount,
extendedBounds, extendedBounds,
valuesSource, config,
config.format(),
searchContext, searchContext,
parent, parent,
collectsFromSingleBucket, collectsFromSingleBucket,
@ -111,6 +112,6 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new DateHistogramAggregator(name, factories, rounding, null, order, keyed, minDocCount, extendedBounds, return new DateHistogramAggregator(name, factories, rounding, null, order, keyed, minDocCount, extendedBounds,
null, config.format(), searchContext, parent, false, metadata); config, searchContext, parent, false, metadata);
} }
} }

View File

@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -69,11 +70,21 @@ class DateRangeHistogramAggregator extends BucketsAggregator {
private final LongKeyedBucketOrds bucketOrds; private final LongKeyedBucketOrds bucketOrds;
DateRangeHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, Rounding.Prepared preparedRounding, DateRangeHistogramAggregator(
BucketOrder order, boolean keyed, String name,
long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource valuesSource, AggregatorFactories factories,
DocValueFormat formatter, SearchContext aggregationContext, Rounding rounding,
Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException { Rounding.Prepared preparedRounding,
BucketOrder order,
boolean keyed,
long minDocCount,
@Nullable ExtendedBounds extendedBounds,
ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext,
Aggregator parent,
boolean collectsFromSingleBucket,
Map<String, Object> metadata
) throws IOException {
super(name, factories, aggregationContext, parent, metadata); super(name, factories, aggregationContext, parent, metadata);
this.rounding = rounding; this.rounding = rounding;
@ -83,8 +94,9 @@ class DateRangeHistogramAggregator extends BucketsAggregator {
this.keyed = keyed; this.keyed = keyed;
this.minDocCount = minDocCount; this.minDocCount = minDocCount;
this.extendedBounds = extendedBounds; this.extendedBounds = extendedBounds;
this.valuesSource = (ValuesSource.Range) valuesSource; // TODO: Stop using null here
this.formatter = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Range) valuesSourceConfig.getValuesSource() : null;
this.formatter = valuesSourceConfig.format();
if (this.valuesSource.rangeType() != RangeType.DATE) { if (this.valuesSource.rangeType() != RangeType.DATE) {
throw new IllegalArgumentException("Expected date range type but found range type [" + this.valuesSource.rangeType().name throw new IllegalArgumentException("Expected date range type but found range type [" + this.valuesSource.rangeType().name
+ "]"); + "]");

View File

@ -29,7 +29,6 @@ import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.HistogramAggregatorSupplier; import org.elasticsearch.search.aggregations.support.HistogramAggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -87,11 +86,10 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
HistogramAggregationBuilder.NAME); HistogramAggregationBuilder.NAME);
if (aggregatorSupplier instanceof HistogramAggregatorSupplier == false) { if (aggregatorSupplier instanceof HistogramAggregatorSupplier == false) {
@ -100,7 +98,7 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
} }
HistogramAggregatorSupplier histogramAggregatorSupplier = (HistogramAggregatorSupplier) aggregatorSupplier; HistogramAggregatorSupplier histogramAggregatorSupplier = (HistogramAggregatorSupplier) aggregatorSupplier;
return histogramAggregatorSupplier.build(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound, return histogramAggregatorSupplier.build(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
valuesSource, config.format(), searchContext, parent, collectsFromSingleBucket, metadata); config, searchContext, parent, collectsFromSingleBucket, metadata);
} }
@Override @Override
@ -108,6 +106,6 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new NumericHistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound, return new NumericHistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
null, config.format(), searchContext, parent, false, metadata); config, searchContext, parent, false, metadata);
} }
} }

View File

@ -21,15 +21,14 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScoreMode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -54,8 +53,7 @@ public class NumericHistogramAggregator extends AbstractHistogramAggregator {
long minDocCount, long minDocCount,
double minBound, double minBound,
double maxBound, double maxBound,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
@ -71,13 +69,14 @@ public class NumericHistogramAggregator extends AbstractHistogramAggregator {
minDocCount, minDocCount,
minBound, minBound,
maxBound, maxBound,
formatter, valuesSourceConfig.format(),
context, context,
parent, parent,
collectsFromSingleBucket, collectsFromSingleBucket,
metadata metadata
); );
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: Stop using null here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
} }
@Override @Override

View File

@ -21,17 +21,16 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.mapper.RangeFieldMapper; import org.elasticsearch.index.mapper.RangeFieldMapper;
import org.elasticsearch.index.mapper.RangeType; import org.elasticsearch.index.mapper.RangeType;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -51,8 +50,7 @@ public class RangeHistogramAggregator extends AbstractHistogramAggregator {
long minDocCount, long minDocCount,
double minBound, double minBound,
double maxBound, double maxBound,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
@ -68,13 +66,14 @@ public class RangeHistogramAggregator extends AbstractHistogramAggregator {
minDocCount, minDocCount,
minBound, minBound,
maxBound, maxBound,
formatter, valuesSourceConfig.format(),
context, context,
parent, parent,
collectsFromSingleBucket, collectsFromSingleBucket,
metadata metadata
); );
this.valuesSource = (ValuesSource.Range) valuesSource; // TODO: Stop using nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Range) valuesSourceConfig.getValuesSource() : null;
if (this.valuesSource.rangeType().isNumeric() == false) { if (this.valuesSource.rangeType().isNumeric() == false) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Expected numeric range type but found non-numeric range [" + this.valuesSource.rangeType().name + "]" "Expected numeric range type but found non-numeric range [" + this.valuesSource.rangeType().name + "]"

View File

@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -40,12 +41,13 @@ public class MissingAggregator extends BucketsAggregator implements SingleBucket
public MissingAggregator( public MissingAggregator(
String name, String name,
AggregatorFactories factories, AggregatorFactories factories,
ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, factories, aggregationContext, parent, metadata); super(name, factories, aggregationContext, parent, metadata);
this.valuesSource = valuesSource; // TODO: Stop using nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? valuesSourceConfig.getValuesSource() : null;
} }
@Override @Override

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -52,15 +51,14 @@ public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory {
protected MissingAggregator createUnmapped(SearchContext searchContext, protected MissingAggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new MissingAggregator(name, factories, null, searchContext, parent, metadata); return new MissingAggregator(name, factories, config, searchContext, parent, metadata);
} }
@Override @Override
protected MissingAggregator doCreateInternal(ValuesSource valuesSource, protected MissingAggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
final AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry() final AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry()
.getAggregator(config, MissingAggregationBuilder.NAME); .getAggregator(config, MissingAggregationBuilder.NAME);
if (aggregatorSupplier instanceof MissingAggregatorSupplier == false) { if (aggregatorSupplier instanceof MissingAggregatorSupplier == false) {
@ -68,8 +66,8 @@ public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory {
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return (MissingAggregator) ((MissingAggregatorSupplier) aggregatorSupplier) return (MissingAggregator) ((MissingAggregatorSupplier) aggregatorSupplier).build(name, factories, config, searchContext, parent,
.build(name, factories, valuesSource, searchContext, parent, metadata); metadata);
} }
} }

View File

@ -19,11 +19,10 @@
package org.elasticsearch.search.aggregations.bucket.missing; package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -34,7 +33,7 @@ public interface MissingAggregatorSupplier extends AggregatorSupplier {
Aggregator build(String name, Aggregator build(String name,
AggregatorFactories factories, AggregatorFactories factories,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException; Map<String, Object> metadata) throws IOException;

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@ -77,11 +76,10 @@ public class AbstractRangeAggregatorFactory<R extends Range> extends ValuesSourc
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
aggregationTypeName); aggregationTypeName);
@ -89,7 +87,17 @@ public class AbstractRangeAggregatorFactory<R extends Range> extends ValuesSourc
throw new AggregationExecutionException("Registry miss-match - expected RangeAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected RangeAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((RangeAggregatorSupplier)aggregatorSupplier).build(name, factories, (Numeric) valuesSource, config.format(), rangeFactory, return ((RangeAggregatorSupplier) aggregatorSupplier).build(
ranges, keyed, searchContext, parent, metadata); name,
factories,
(Numeric) config.getValuesSource(),
config.format(),
rangeFactory,
ranges,
keyed,
searchContext,
parent,
metadata
);
} }
} }

View File

@ -25,7 +25,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -62,8 +61,7 @@ public class BinaryRangeAggregatorFactory extends ValuesSourceAggregatorFactory
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext, Aggregator parent,
SearchContext searchContext, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
@ -73,7 +71,7 @@ public class BinaryRangeAggregatorFactory extends ValuesSourceAggregatorFactory
throw new AggregationExecutionException("Registry miss-match - expected IpRangeAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected IpRangeAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((IpRangeAggregatorSupplier) aggregatorSupplier).build(name, factories, valuesSource, config.format(), return ((IpRangeAggregatorSupplier) aggregatorSupplier).build(name, factories, config.getValuesSource(), config.format(),
ranges, keyed, searchContext, parent, metadata); ranges, keyed, searchContext, parent, metadata);
} }

View File

@ -85,11 +85,10 @@ public class GeoDistanceRangeAggregatorFactory extends ValuesSourceAggregatorFac
} }
@Override @Override
protected Aggregator doCreateInternal(final ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry() AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry()
.getAggregator(config, GeoDistanceAggregationBuilder.NAME); .getAggregator(config, GeoDistanceAggregationBuilder.NAME);
if (aggregatorSupplier instanceof GeoDistanceAggregatorSupplier == false) { if (aggregatorSupplier instanceof GeoDistanceAggregatorSupplier == false) {
@ -97,7 +96,7 @@ public class GeoDistanceRangeAggregatorFactory extends ValuesSourceAggregatorFac
+ GeoDistanceAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]"); + GeoDistanceAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]");
} }
return ((GeoDistanceAggregatorSupplier) aggregatorSupplier).build(name, factories, distanceType, origin, return ((GeoDistanceAggregatorSupplier) aggregatorSupplier).build(name, factories, distanceType, origin,
unit, config.toValuesSource(), config.format(), rangeFactory, ranges, keyed, searchContext, parent, metadata); unit, config.getValuesSource(), config.format(), rangeFactory, ranges, keyed, searchContext, parent, metadata);
} }
private static class DistanceSource extends ValuesSource.Numeric { private static class DistanceSource extends ValuesSource.Numeric {

View File

@ -29,7 +29,6 @@ import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode; import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -47,15 +46,15 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory
CoreValuesSourceType.DATE, CoreValuesSourceType.DATE,
CoreValuesSourceType.BOOLEAN), CoreValuesSourceType.BOOLEAN),
(DiversifiedAggregatorSupplier) (String name, int shardSize, AggregatorFactories factories, SearchContext context, (DiversifiedAggregatorSupplier) (String name, int shardSize, AggregatorFactories factories, SearchContext context,
Aggregator parent, Map<String, Object> metadata, ValuesSource valuesSource, Aggregator parent, Map<String, Object> metadata, ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue, String executionHint) -> int maxDocsPerValue, String executionHint) ->
new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, metadata, valuesSource, new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, metadata, valuesSourceConfig,
maxDocsPerValue) maxDocsPerValue)
); );
builder.register(DiversifiedAggregationBuilder.NAME, CoreValuesSourceType.BYTES, builder.register(DiversifiedAggregationBuilder.NAME, CoreValuesSourceType.BYTES,
(DiversifiedAggregatorSupplier) (String name, int shardSize, AggregatorFactories factories, SearchContext context, (DiversifiedAggregatorSupplier) (String name, int shardSize, AggregatorFactories factories, SearchContext context,
Aggregator parent, Map<String, Object> metadata, ValuesSource valuesSource, Aggregator parent, Map<String, Object> metadata, ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue, String executionHint) -> { int maxDocsPerValue, String executionHint) -> {
ExecutionMode execution = null; ExecutionMode execution = null;
if (executionHint != null) { if (executionHint != null) {
@ -66,10 +65,10 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory
if (execution == null) { if (execution == null) {
execution = ExecutionMode.GLOBAL_ORDINALS; execution = ExecutionMode.GLOBAL_ORDINALS;
} }
if ((execution.needsGlobalOrdinals()) && (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals))) { if ((execution.needsGlobalOrdinals()) && (valuesSourceConfig.hasGlobalOrdinals() == false)) {
execution = ExecutionMode.MAP; execution = ExecutionMode.MAP;
} }
return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, metadata); return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSourceConfig, context, parent, metadata);
}); });
} }
@ -88,20 +87,18 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier supplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier supplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, DiversifiedAggregationBuilder.NAME);
DiversifiedAggregationBuilder.NAME);
if (supplier instanceof DiversifiedAggregatorSupplier == false) { if (supplier instanceof DiversifiedAggregatorSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected " + DiversifiedAggregatorSupplier.class.toString() + throw new AggregationExecutionException("Registry miss-match - expected " + DiversifiedAggregatorSupplier.class.toString() +
", found [" + supplier.getClass().toString() + "]"); ", found [" + supplier.getClass().toString() + "]");
} }
return ((DiversifiedAggregatorSupplier) supplier).build(name, shardSize, factories, searchContext, parent, metadata, valuesSource, return ((DiversifiedAggregatorSupplier) supplier).build(name, shardSize, factories, searchContext, parent, metadata,
maxDocsPerValue, executionHint); config, maxDocsPerValue, executionHint);
} }
@Override @Override

View File

@ -22,7 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -36,7 +36,7 @@ public interface DiversifiedAggregatorSupplier extends AggregatorSupplier {
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata, Map<String, Object> metadata,
ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue, int maxDocsPerValue,
String executionHint) throws IOException; String executionHint) throws IOException;
} }

View File

@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -47,12 +48,19 @@ public class DiversifiedBytesHashSamplerAggregator extends SamplerAggregator {
private ValuesSource valuesSource; private ValuesSource valuesSource;
private int maxDocsPerValue; private int maxDocsPerValue;
DiversifiedBytesHashSamplerAggregator(String name, int shardSize, AggregatorFactories factories, DiversifiedBytesHashSamplerAggregator(
SearchContext context, Aggregator parent, Map<String, Object> metadata, String name,
ValuesSource valuesSource, int shardSize,
int maxDocsPerValue) throws IOException { AggregatorFactories factories,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata,
ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue
) throws IOException {
super(name, shardSize, factories, context, parent, metadata); super(name, shardSize, factories, context, parent, metadata);
this.valuesSource = valuesSource; assert valuesSourceConfig.hasValues();
this.valuesSource = valuesSourceConfig.getValuesSource();
this.maxDocsPerValue = maxDocsPerValue; this.maxDocsPerValue = maxDocsPerValue;
} }

View File

@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -46,11 +47,19 @@ public class DiversifiedMapSamplerAggregator extends SamplerAggregator {
private int maxDocsPerValue; private int maxDocsPerValue;
private BytesRefHash bucketOrds; private BytesRefHash bucketOrds;
DiversifiedMapSamplerAggregator(String name, int shardSize, AggregatorFactories factories, DiversifiedMapSamplerAggregator(
SearchContext context, Aggregator parent, Map<String, Object> metadata, String name,
ValuesSource valuesSource, int maxDocsPerValue) throws IOException { int shardSize,
AggregatorFactories factories,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata,
ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue
) throws IOException {
super(name, shardSize, factories, context, parent, metadata); super(name, shardSize, factories, context, parent, metadata);
this.valuesSource = valuesSource; assert valuesSourceConfig.hasValues();
this.valuesSource = valuesSourceConfig.getValuesSource();
this.maxDocsPerValue = maxDocsPerValue; this.maxDocsPerValue = maxDocsPerValue;
// Need to use super class shardSize since it is limited to maxDoc // Need to use super class shardSize since it is limited to maxDoc
bucketOrds = new BytesRefHash(this.shardSize, context.bigArrays()); bucketOrds = new BytesRefHash(this.shardSize, context.bigArrays());

View File

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -42,11 +43,19 @@ public class DiversifiedNumericSamplerAggregator extends SamplerAggregator {
private ValuesSource.Numeric valuesSource; private ValuesSource.Numeric valuesSource;
private int maxDocsPerValue; private int maxDocsPerValue;
DiversifiedNumericSamplerAggregator(String name, int shardSize, AggregatorFactories factories, DiversifiedNumericSamplerAggregator(
SearchContext context, Aggregator parent, Map<String, Object> metadata, String name,
ValuesSource valuesSource, int maxDocsPerValue) throws IOException { int shardSize,
AggregatorFactories factories,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata,
ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue
) throws IOException {
super(name, shardSize, factories, context, parent, metadata); super(name, shardSize, factories, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; assert valuesSourceConfig.hasValues();
this.valuesSource = (ValuesSource.Numeric) valuesSourceConfig.getValuesSource();
this.maxDocsPerValue = maxDocsPerValue; this.maxDocsPerValue = maxDocsPerValue;
} }

View File

@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -43,11 +44,19 @@ public class DiversifiedOrdinalsSamplerAggregator extends SamplerAggregator {
private ValuesSource.Bytes.WithOrdinals.FieldData valuesSource; private ValuesSource.Bytes.WithOrdinals.FieldData valuesSource;
private int maxDocsPerValue; private int maxDocsPerValue;
DiversifiedOrdinalsSamplerAggregator(String name, int shardSize, AggregatorFactories factories, DiversifiedOrdinalsSamplerAggregator(
SearchContext context, Aggregator parent, Map<String, Object> metadata, String name,
ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, int maxDocsPerValue) throws IOException { int shardSize,
AggregatorFactories factories,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata,
ValuesSourceConfig valuesSourceConfig,
int maxDocsPerValue
) throws IOException {
super(name, shardSize, factories, context, parent, metadata); super(name, shardSize, factories, context, parent, metadata);
this.valuesSource = valuesSource; assert valuesSourceConfig.hasValues();
this.valuesSource = (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSourceConfig.getValuesSource();
this.maxDocsPerValue = maxDocsPerValue; this.maxDocsPerValue = maxDocsPerValue;
} }

View File

@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -63,12 +63,19 @@ public class SamplerAggregator extends DeferableBucketAggregator implements Sing
MAP(new ParseField("map")) { MAP(new ParseField("map")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, Aggregator create(
SearchContext context, Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
AggregatorFactories factories,
int shardSize,
int maxDocsPerValue,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
return new DiversifiedMapSamplerAggregator(name, shardSize, factories, context, parent, metadata, return new DiversifiedMapSamplerAggregator(name, shardSize, factories, context, parent, metadata,
valuesSource, valuesSourceConfig, maxDocsPerValue);
maxDocsPerValue);
} }
@Override @Override
@ -80,12 +87,19 @@ public class SamplerAggregator extends DeferableBucketAggregator implements Sing
BYTES_HASH(new ParseField("bytes_hash")) { BYTES_HASH(new ParseField("bytes_hash")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, Aggregator create(
SearchContext context, Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
AggregatorFactories factories,
int shardSize,
int maxDocsPerValue,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
return new DiversifiedBytesHashSamplerAggregator(name, shardSize, factories, context, parent, metadata, return new DiversifiedBytesHashSamplerAggregator(name, shardSize, factories, context, parent, metadata,
valuesSource, valuesSourceConfig, maxDocsPerValue);
maxDocsPerValue);
} }
@Override @Override
@ -97,10 +111,18 @@ public class SamplerAggregator extends DeferableBucketAggregator implements Sing
GLOBAL_ORDINALS(new ParseField("global_ordinals")) { GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, Aggregator create(
SearchContext context, Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
AggregatorFactories factories,
int shardSize,
int maxDocsPerValue,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
return new DiversifiedOrdinalsSamplerAggregator(name, shardSize, factories, context, parent, metadata, return new DiversifiedOrdinalsSamplerAggregator(name, shardSize, factories, context, parent, metadata,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxDocsPerValue); valuesSourceConfig, maxDocsPerValue);
} }
@Override @Override
@ -125,8 +147,16 @@ public class SamplerAggregator extends DeferableBucketAggregator implements Sing
this.parseField = parseField; this.parseField = parseField;
} }
abstract Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, abstract Aggregator create(
ValuesSource valuesSource, SearchContext context, Aggregator parent, Map<String, Object> metadata) throws IOException; String name,
AggregatorFactories factories,
int shardSize,
int maxDocsPerValue,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException;
abstract boolean needsGlobalOrdinals(); abstract boolean needsGlobalOrdinals();

View File

@ -151,11 +151,10 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
if (collectsFromSingleBucket == false) { if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, searchContext, parent); return asMultiBucketAggregator(this, searchContext, parent);
} }
@ -167,7 +166,7 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory {
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((RareTermsAggregatorSupplier) aggregatorSupplier).build(name, factories, valuesSource, config.format(), return ((RareTermsAggregatorSupplier) aggregatorSupplier).build(name, factories, config.getValuesSource(), config.format(),
maxDocCount, precision, includeExclude, searchContext, parent, metadata); maxDocCount, precision, includeExclude, searchContext, parent, metadata);
} }

View File

@ -187,14 +187,14 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metadata); super(name, config, queryShardContext, parent, subFactoriesBuilder, metadata);
if (config.unmapped() == false) { if (config.hasValues()) {
if (config.fieldContext().fieldType().isSearchable() == false) { if (config.fieldContext().fieldType().isSearchable() == false) {
throw new IllegalArgumentException("SignificantText aggregation requires fields to be searchable, but [" throw new IllegalArgumentException("SignificantText aggregation requires fields to be searchable, but ["
+ config.fieldContext().fieldType().name() + "] is not"); + config.fieldContext().fieldType().name() + "] is not");
} }
} }
if (config.unmapped() == false) { if (config.hasValues()) {
if (config.fieldContext().fieldType().isSearchable() == false) { if (config.fieldContext().fieldType().isSearchable() == false) {
throw new IllegalArgumentException("SignificantText aggregation requires fields to be searchable, but [" throw new IllegalArgumentException("SignificantText aggregation requires fields to be searchable, but ["
+ config.fieldContext().fieldType().name() + "] is not"); + config.fieldContext().fieldType().name() + "] is not");
@ -293,11 +293,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
SignificantTermsAggregationBuilder.NAME); SignificantTermsAggregationBuilder.NAME);
if (aggregatorSupplier instanceof SignificantTermsAggregatorSupplier == false) { if (aggregatorSupplier instanceof SignificantTermsAggregatorSupplier == false) {
@ -325,7 +324,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
// TODO we should refactor so that we don't need to use this Factory as a singleton (e.g. stop passing `this` to the aggregators) // TODO we should refactor so that we don't need to use this Factory as a singleton (e.g. stop passing `this` to the aggregators)
return sigTermsAggregatorSupplier.build(name, factories, valuesSource, config.format(), return sigTermsAggregatorSupplier.build(name, factories, config.getValuesSource(), config.format(),
bucketCountThresholds, includeExclude, executionHint, searchContext, parent, bucketCountThresholds, includeExclude, executionHint, searchContext, parent,
significanceHeuristic, this, collectsFromSingleBucket, metadata); significanceHeuristic, this, collectsFromSingleBucket, metadata);
} }
@ -398,7 +397,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
**/ **/
remapGlobalOrd = false; remapGlobalOrd = false;
} }
return new GlobalOrdinalsStringTermsAggregator( return new GlobalOrdinalsStringTermsAggregator(
name, name,
factories, factories,

View File

@ -146,7 +146,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
subAggCollectMode = pickSubAggColectMode(factories, bucketCountThresholds.getShardSize(), -1); subAggCollectMode = pickSubAggColectMode(factories, bucketCountThresholds.getShardSize(), -1);
} }
ValuesSource.Numeric numericValuesSource = (ValuesSource.Numeric) valuesSource; ValuesSource.Numeric numericValuesSource = (ValuesSource.Numeric) valuesSource;
IncludeExclude.LongFilter longFilter = null; IncludeExclude.LongFilter longFilter = null;
Function<NumericTermsAggregator, ResultStrategy<?, ?>> resultStrategy; Function<NumericTermsAggregator, ResultStrategy<?, ?>> resultStrategy;
if (numericValuesSource.isFloatingPoint()) { if (numericValuesSource.isFloatingPoint()) {
@ -223,8 +223,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -246,7 +245,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
bucketCountThresholds.ensureValidity(); bucketCountThresholds.ensureValidity();
return termsAggregatorSupplier.build(name, factories, valuesSource, order, config.format(), return termsAggregatorSupplier.build(name, factories, config.getValuesSource(), order, config.format(),
bucketCountThresholds, includeExclude, executionHint, searchContext, parent, collectMode, bucketCountThresholds, includeExclude, executionHint, searchContext, parent, collectMode,
showTermDocCountError, collectsFromSingleBucket, metadata); showTermDocCountError, collectsFromSingleBucket, metadata);
} }

View File

@ -46,10 +46,11 @@ class AvgAggregator extends NumericMetricsAggregator.SingleValue {
DoubleArray compensations; DoubleArray compensations;
DocValueFormat format; DocValueFormat format;
AvgAggregator(String name, ValuesSourceConfig valuesSourceConfig, ValuesSource valuesSource, SearchContext context, AvgAggregator(String name, ValuesSourceConfig valuesSourceConfig, SearchContext context,
Aggregator parent, Map<String, Object> metadata) throws IOException { Aggregator parent, Map<String, Object> metadata) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO Stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
this.format = valuesSourceConfig.format(); this.format = valuesSourceConfig.format();
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -54,15 +53,14 @@ class AvgAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new AvgAggregator(name, config, null, searchContext, parent, metadata); return new AvgAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
AvgAggregationBuilder.NAME); AvgAggregationBuilder.NAME);
@ -70,6 +68,6 @@ class AvgAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, valuesSource, searchContext, parent, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -41,6 +41,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -62,13 +63,14 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
public CardinalityAggregator( public CardinalityAggregator(
String name, String name,
ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
int precision, int precision,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = valuesSource; // TODO: Stop using nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? valuesSourceConfig.getValuesSource() : null;
this.precision = precision; this.precision = precision;
this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, context.bigArrays(), 1); this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, context.bigArrays(), 1);
} }

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -58,15 +57,14 @@ class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new CardinalityAggregator(name, null, precision(), searchContext, parent, metadata); return new CardinalityAggregator(name, config, precision(), searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
CardinalityAggregationBuilder.NAME); CardinalityAggregationBuilder.NAME);
if (aggregatorSupplier instanceof CardinalityAggregatorSupplier == false) { if (aggregatorSupplier instanceof CardinalityAggregatorSupplier == false) {
@ -74,7 +72,7 @@ class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory {
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
CardinalityAggregatorSupplier cardinalityAggregatorSupplier = (CardinalityAggregatorSupplier) aggregatorSupplier; CardinalityAggregatorSupplier cardinalityAggregatorSupplier = (CardinalityAggregatorSupplier) aggregatorSupplier;
return cardinalityAggregatorSupplier.build(name, valuesSource, precision(), searchContext, parent, metadata); return cardinalityAggregatorSupplier.build(name, config, precision(), searchContext, parent, metadata);
} }
private int precision() { private int precision() {

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -29,7 +29,7 @@ import java.util.Map;
public interface CardinalityAggregatorSupplier extends AggregatorSupplier { public interface CardinalityAggregatorSupplier extends AggregatorSupplier {
Aggregator build(String name, Aggregator build(String name,
ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
int precision, int precision,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,

View File

@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -53,11 +54,18 @@ class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue {
DoubleArray sumOfSqrs; DoubleArray sumOfSqrs;
DoubleArray compensationOfSqrs; DoubleArray compensationOfSqrs;
ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource, DocValueFormat formatter, ExtendedStatsAggregator(
SearchContext context, Aggregator parent, double sigma, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
double sigma,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = valuesSource; // TODO: stop depending on nulls here
this.format = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
this.format = valuesSourceConfig.format();
this.sigma = sigma; this.sigma = sigma;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();

View File

@ -26,8 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
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.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -62,12 +60,11 @@ class ExtendedStatsAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new ExtendedStatsAggregator(name, null, config.format(), searchContext, parent, sigma, metadata); return new ExtendedStatsAggregator(name, config, searchContext, parent, sigma, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -78,7 +75,6 @@ class ExtendedStatsAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected ExtendedStatsAggregatorProvider, found [" + throw new AggregationExecutionException("Registry miss-match - expected ExtendedStatsAggregatorProvider, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return new ExtendedStatsAggregator(name, (Numeric) valuesSource, config.format(), searchContext, return new ExtendedStatsAggregator(name, config, searchContext, parent, sigma, metadata);
parent, sigma, metadata);
} }
} }

View File

@ -18,10 +18,9 @@
*/ */
package org.elasticsearch.search.aggregations.metrics; package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -30,8 +29,7 @@ import java.util.Map;
public interface ExtendedStatsAggregatorProvider extends AggregatorSupplier { public interface ExtendedStatsAggregatorProvider extends AggregatorSupplier {
Aggregator build(String name, Aggregator build(String name,
ValuesSource.Numeric valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
double sigma, double sigma,

View File

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -49,10 +50,17 @@ final class GeoBoundsAggregator extends MetricsAggregator {
DoubleArray negLefts; DoubleArray negLefts;
DoubleArray negRights; DoubleArray negRights;
GeoBoundsAggregator(String name, SearchContext aggregationContext, Aggregator parent, GeoBoundsAggregator(
ValuesSource valuesSource, boolean wrapLongitude, Map<String, Object> metadata) throws IOException { String name,
SearchContext aggregationContext,
Aggregator parent,
ValuesSourceConfig valuesSourceConfig,
boolean wrapLongitude,
Map<String, Object> metadata
) throws IOException {
super(name, aggregationContext, parent, metadata); super(name, aggregationContext, parent, metadata);
this.valuesSource = (ValuesSource.GeoPoint) valuesSource; // TODO: stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.GeoPoint) valuesSourceConfig.getValuesSource() : null;
this.wrapLongitude = wrapLongitude; this.wrapLongitude = wrapLongitude;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -54,15 +53,14 @@ class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new GeoBoundsAggregator(name, searchContext, parent, null, wrapLongitude, metadata); return new GeoBoundsAggregator(name, searchContext, parent, config, wrapLongitude, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry() AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry()
.getAggregator(config, GeoBoundsAggregationBuilder.NAME); .getAggregator(config, GeoBoundsAggregationBuilder.NAME);
@ -71,7 +69,7 @@ class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory {
+ GeoBoundsAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]"); + GeoBoundsAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]");
} }
return ((GeoBoundsAggregatorSupplier) aggregatorSupplier).build(name, searchContext, parent, valuesSource, wrapLongitude, metadata); return ((GeoBoundsAggregatorSupplier) aggregatorSupplier).build(name, searchContext, parent, config, wrapLongitude, metadata);
} }
static void registerAggregators(ValuesSourceRegistry.Builder builder) { static void registerAggregators(ValuesSourceRegistry.Builder builder) {

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -30,7 +30,11 @@ import java.util.Map;
@FunctionalInterface @FunctionalInterface
public interface GeoBoundsAggregatorSupplier extends AggregatorSupplier { public interface GeoBoundsAggregatorSupplier extends AggregatorSupplier {
MetricsAggregator build(String name, SearchContext aggregationContext, Aggregator parent, MetricsAggregator build(
ValuesSource valuesSource, boolean wrapLongitude, String name,
Map<String, Object> metadata) throws IOException; SearchContext aggregationContext,
Aggregator parent,
ValuesSourceConfig valuesSourceConfig,
boolean wrapLongitude,
Map<String, Object> metadata) throws IOException;
} }

View File

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -44,10 +45,16 @@ final class GeoCentroidAggregator extends MetricsAggregator {
private DoubleArray lonSum, lonCompensations, latSum, latCompensations; private DoubleArray lonSum, lonCompensations, latSum, latCompensations;
private LongArray counts; private LongArray counts;
GeoCentroidAggregator(String name, SearchContext context, Aggregator parent, GeoCentroidAggregator(
ValuesSource valuesSource, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.GeoPoint) valuesSource; // TODO: Stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.GeoPoint) valuesSourceConfig.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();
lonSum = bigArrays.newDoubleArray(1, true); lonSum = bigArrays.newDoubleArray(1, true);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -50,27 +49,26 @@ class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new GeoCentroidAggregator(name, searchContext, parent, null, metadata); return new GeoCentroidAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
GeoCentroidAggregationBuilder.NAME); GeoCentroidAggregationBuilder.NAME);
if (aggregatorSupplier instanceof GeoCentroidAggregatorSupplier == false) { if (aggregatorSupplier instanceof MetricAggregatorSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected " throw new AggregationExecutionException("Registry miss-match - expected "
+ GeoCentroidAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]"); + MetricAggregatorSupplier.class.getName() + ", found [" + aggregatorSupplier.getClass().toString() + "]");
} }
return ((GeoCentroidAggregatorSupplier) aggregatorSupplier).build(name, searchContext, parent, valuesSource, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
static void registerAggregators(ValuesSourceRegistry.Builder builder) { static void registerAggregators(ValuesSourceRegistry.Builder builder) {
builder.register(GeoCentroidAggregationBuilder.NAME, CoreValuesSourceType.GEOPOINT, builder.register(GeoCentroidAggregationBuilder.NAME, CoreValuesSourceType.GEOPOINT,
(GeoCentroidAggregatorSupplier) GeoCentroidAggregator::new); (MetricAggregatorSupplier) GeoCentroidAggregator::new);
} }
} }

View File

@ -1,36 +0,0 @@
/*
* 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.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@FunctionalInterface
public interface GeoCentroidAggregatorSupplier extends AggregatorSupplier {
MetricsAggregator build(String name, SearchContext context, Aggregator parent,
ValuesSource valuesSource,
Map<String, Object> metadata) throws IOException;
}

View File

@ -58,11 +58,11 @@ class MaxAggregator extends NumericMetricsAggregator.SingleValue {
MaxAggregator(String name, MaxAggregator(String name,
ValuesSourceConfig config, ValuesSourceConfig config,
ValuesSource valuesSource,
SearchContext context, SearchContext context,
Aggregator parent, Map<String, Object> metadata) throws IOException { Aggregator parent, Map<String, Object> metadata) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO stop expecting nulls here
this.valuesSource = config.hasValues() ? (ValuesSource.Numeric) config.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
maxes = context.bigArrays().newDoubleArray(1, false); maxes = context.bigArrays().newDoubleArray(1, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -53,15 +52,14 @@ class MaxAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new MaxAggregator(name, config, null, searchContext, parent, metadata); return new MaxAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
MaxAggregationBuilder.NAME); MaxAggregationBuilder.NAME);
@ -69,6 +67,6 @@ class MaxAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, valuesSource, searchContext, parent, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -73,11 +72,10 @@ public class MedianAbsoluteDeviationAggregatorFactory extends ValuesSourceAggreg
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
MedianAbsoluteDeviationAggregationBuilder.NAME); MedianAbsoluteDeviationAggregationBuilder.NAME);
@ -85,7 +83,7 @@ public class MedianAbsoluteDeviationAggregatorFactory extends ValuesSourceAggreg
throw new AggregationExecutionException("Registry miss-match - expected MedianAbsoluteDeviationAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MedianAbsoluteDeviationAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MedianAbsoluteDeviationAggregatorSupplier) aggregatorSupplier).build(name, valuesSource, config.format(), return ((MedianAbsoluteDeviationAggregatorSupplier) aggregatorSupplier).build(name, config.getValuesSource(), config.format(),
searchContext, parent, metadata, compression); searchContext, parent, metadata, compression);
} }
} }

View File

@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
@ -30,7 +29,6 @@ import java.util.Map;
public interface MetricAggregatorSupplier extends AggregatorSupplier { public interface MetricAggregatorSupplier extends AggregatorSupplier {
Aggregator build(String name, Aggregator build(String name,
ValuesSourceConfig valuesSourceConfig, ValuesSourceConfig valuesSourceConfig,
ValuesSource valuesSource,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException; Map<String, Object> metadata) throws IOException;

View File

@ -61,12 +61,12 @@ class MinAggregator extends NumericMetricsAggregator.SingleValue {
MinAggregator(String name, MinAggregator(String name,
ValuesSourceConfig config, ValuesSourceConfig config,
ValuesSource valuesSource,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: Stop using nulls here
this.valuesSource = config.hasValues() ? (ValuesSource.Numeric) config.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
mins = context.bigArrays().newDoubleArray(1, false); mins = context.bigArrays().newDoubleArray(1, false);
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -53,15 +52,14 @@ class MinAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new MinAggregator(name, config, null, searchContext, parent, metadata); return new MinAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
MinAggregationBuilder.NAME); MinAggregationBuilder.NAME);
@ -69,6 +67,6 @@ class MinAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, valuesSource, searchContext, parent, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -84,8 +84,7 @@ class PercentileRanksAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -97,7 +96,16 @@ class PercentileRanksAggregatorFactory extends ValuesSourceAggregatorFactory {
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
PercentilesAggregatorSupplier percentilesAggregatorSupplier = (PercentilesAggregatorSupplier) aggregatorSupplier; PercentilesAggregatorSupplier percentilesAggregatorSupplier = (PercentilesAggregatorSupplier) aggregatorSupplier;
return percentilesAggregatorSupplier.build(name, valuesSource, searchContext, parent, percents, percentilesConfig, keyed, return percentilesAggregatorSupplier.build(
config.format(), metadata); name,
config.getValuesSource(),
searchContext,
parent,
percents,
percentilesConfig,
keyed,
config.format(),
metadata
);
} }
} }

View File

@ -83,8 +83,7 @@ class PercentilesAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -97,7 +96,16 @@ class PercentilesAggregatorFactory extends ValuesSourceAggregatorFactory {
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
PercentilesAggregatorSupplier percentilesAggregatorSupplier = (PercentilesAggregatorSupplier) aggregatorSupplier; PercentilesAggregatorSupplier percentilesAggregatorSupplier = (PercentilesAggregatorSupplier) aggregatorSupplier;
return percentilesAggregatorSupplier.build(name, valuesSource, searchContext, parent, percents, percentilesConfig, keyed, return percentilesAggregatorSupplier.build(
config.format(), metadata); name,
config.getValuesSource(),
searchContext,
parent,
percents,
percentilesConfig,
keyed,
config.format(),
metadata
);
} }
} }

View File

@ -48,10 +48,16 @@ class StatsAggregator extends NumericMetricsAggregator.MultiValue {
DoubleArray mins; DoubleArray mins;
DoubleArray maxes; DoubleArray maxes;
StatsAggregator(String name, ValuesSourceConfig valuesSourceConfig, ValuesSource valuesSource, StatsAggregator(
SearchContext context, Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: stop using nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();
counts = bigArrays.newLongArray(1, true); counts = bigArrays.newLongArray(1, true);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -56,15 +55,14 @@ class StatsAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new StatsAggregator(name, config, null, searchContext, parent, metadata); return new StatsAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
StatsAggregationBuilder.NAME); StatsAggregationBuilder.NAME);
@ -72,6 +70,6 @@ class StatsAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, valuesSource, searchContext, parent, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -44,10 +44,16 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray sums; private DoubleArray sums;
private DoubleArray compensations; private DoubleArray compensations;
SumAggregator(String name, ValuesSourceConfig valuesSourceConfig, ValuesSource valuesSource, SearchContext context, SumAggregator(
Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = (ValuesSource.Numeric) valuesSource; // TODO: stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null;
this.format = valuesSourceConfig.format(); this.format = valuesSourceConfig.format();
if (valuesSource != null) { if (valuesSource != null) {
sums = context.bigArrays().newDoubleArray(1, true); sums = context.bigArrays().newDoubleArray(1, true);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -57,15 +56,14 @@ class SumAggregatorFactory extends ValuesSourceAggregatorFactory {
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) Map<String, Object> metadata)
throws IOException { throws IOException {
return new SumAggregator(name, config, null, searchContext, parent, metadata); return new SumAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
SumAggregationBuilder.NAME); SumAggregationBuilder.NAME);
@ -73,6 +71,6 @@ class SumAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, valuesSource, searchContext, parent, metadata); return ((MetricAggregatorSupplier) aggregatorSupplier).build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -50,12 +51,13 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
public ValueCountAggregator( public ValueCountAggregator(
String name, String name,
ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, aggregationContext, parent, metadata); super(name, aggregationContext, parent, metadata);
this.valuesSource = valuesSource; // TODO: stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? valuesSourceConfig.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
counts = context.bigArrays().newLongArray(1, true); counts = context.bigArrays().newLongArray(1, true);
} }

View File

@ -26,7 +26,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -39,7 +38,7 @@ class ValueCountAggregatorFactory extends ValuesSourceAggregatorFactory {
public static void registerAggregators(ValuesSourceRegistry.Builder builder) { public static void registerAggregators(ValuesSourceRegistry.Builder builder) {
builder.register(ValueCountAggregationBuilder.NAME, CoreValuesSourceType.ALL_CORE, builder.register(ValueCountAggregationBuilder.NAME, CoreValuesSourceType.ALL_CORE,
(ValueCountAggregatorSupplier) ValueCountAggregator::new); (MetricAggregatorSupplier) ValueCountAggregator::new);
} }
ValueCountAggregatorFactory(String name, ValuesSourceConfig config, QueryShardContext queryShardContext, ValueCountAggregatorFactory(String name, ValuesSourceConfig config, QueryShardContext queryShardContext,
@ -52,22 +51,21 @@ class ValueCountAggregatorFactory extends ValuesSourceAggregatorFactory {
protected Aggregator createUnmapped(SearchContext searchContext, protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
return new ValueCountAggregator(name, null, searchContext, parent, metadata); return new ValueCountAggregator(name, config, searchContext, parent, metadata);
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext, Aggregator parent,
Aggregator parent, boolean collectsFromSingleBucket,
boolean collectsFromSingleBucket, Map<String, Object> metadata) throws IOException {
Map<String, Object> metadata) throws IOException {
AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config, AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config,
ValueCountAggregationBuilder.NAME); ValueCountAggregationBuilder.NAME);
if (aggregatorSupplier instanceof ValueCountAggregatorSupplier == false) { if (aggregatorSupplier instanceof MetricAggregatorSupplier == false) {
throw new AggregationExecutionException("Registry miss-match - expected ValueCountAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected MetricAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((ValueCountAggregatorSupplier) aggregatorSupplier) return ((MetricAggregatorSupplier) aggregatorSupplier)
.build(name, valuesSource, searchContext, parent, metadata); .build(name, config, searchContext, parent, metadata);
} }
} }

View File

@ -1,35 +0,0 @@
/*
* 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.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
public interface ValueCountAggregatorSupplier extends AggregatorSupplier {
Aggregator build(String name,
ValuesSource valuesSource,
SearchContext aggregationContext,
Aggregator parent,
Map<String, Object> metadata) throws IOException;
}

View File

@ -18,5 +18,21 @@
*/ */
package org.elasticsearch.search.aggregations.support; package org.elasticsearch.search.aggregations.support;
/**
* {@link AggregatorSupplier} serves as a marker for what the {@link ValuesSourceRegistry} holds to construct aggregator instances.
* The aggregators for each aggregation should all share a signature, and that signature should be used to create an AggregatorSupplier for
* that aggregation. Alternatively, if an existing supplier has a matching signature, please re-use that.
*
* In many cases, this can be a simple wrapper over the aggregator constructor. If that is sufficient, please just use a reference to the
* constructor. Implementing a supplier (typically as a lambda) should only be necessary when factors besides the {@link ValuesSourceType}
* are necessary for selecting the correct aggregator implementation. This happens in terms for example where we make decisions based on
* the availability of global ordinals.
*
* The suppliers are responsible for any casting of {@link ValuesSource} that needs to happen. They must accept a base {@link ValuesSource}
* instance. The suppliers may perform additional logic to configure the aggregator as needed, such as in
* {@link org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory} deciding the execution mode.
*
* There is ongoing work to normalize aggregator constructor signatures, and thus reduce the number of AggregatorSupplier interfaces.
*/
public interface AggregatorSupplier { public interface AggregatorSupplier {
} }

View File

@ -18,11 +18,11 @@
*/ */
package org.elasticsearch.search.aggregations.support; package org.elasticsearch.search.aggregations.support;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException; import java.io.IOException;
@ -39,8 +39,7 @@ public interface HistogramAggregatorSupplier extends AggregatorSupplier {
long minDocCount, long minDocCount,
double minBound, double minBound,
double maxBound, double maxBound,
@Nullable ValuesSource valuesSource, ValuesSourceConfig valuesSourceConfig,
DocValueFormat formatter,
SearchContext context, SearchContext context,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,

View File

@ -39,7 +39,7 @@ public abstract class MultiValuesSource <VS extends ValuesSource> {
QueryShardContext context) { QueryShardContext context) {
values = new HashMap<>(valuesSourceConfigs.size()); values = new HashMap<>(valuesSourceConfigs.size());
for (Map.Entry<String, ValuesSourceConfig> entry : valuesSourceConfigs.entrySet()) { for (Map.Entry<String, ValuesSourceConfig> entry : valuesSourceConfigs.entrySet()) {
final ValuesSource valuesSource = entry.getValue().toValuesSource(); final ValuesSource valuesSource = entry.getValue().getValuesSource();
if (valuesSource instanceof ValuesSource.Numeric == false) { if (valuesSource instanceof ValuesSource.Numeric == false) {
throw new AggregationExecutionException("ValuesSource type " + valuesSource.toString() + throw new AggregationExecutionException("ValuesSource type " + valuesSource.toString() +
"is not supported for multi-valued aggregation"); "is not supported for multi-valued aggregation");

View File

@ -78,6 +78,13 @@ public abstract class ValuesSource {
*/ */
public abstract Function<Rounding, Rounding.Prepared> roundingPreparer(IndexReader reader) throws IOException; public abstract Function<Rounding, Rounding.Prepared> roundingPreparer(IndexReader reader) throws IOException;
/**
* Check if this values source supports using global ordinals
*/
public boolean hasGlobalOrdinals() {
return false;
}
public static class Range extends ValuesSource { public static class Range extends ValuesSource {
private final RangeType rangeType; private final RangeType rangeType;
protected final IndexFieldData<?> indexFieldData; protected final IndexFieldData<?> indexFieldData;
@ -166,6 +173,11 @@ public abstract class ValuesSource {
return true; return true;
} }
@Override
public boolean hasGlobalOrdinals() {
return true;
}
/** Returns a mapping from segment ordinals to global ordinals. */ /** Returns a mapping from segment ordinals to global ordinals. */
public abstract LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context) public abstract LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context)
throws IOException; throws IOException;

View File

@ -42,19 +42,17 @@ public abstract class ValuesSourceAggregatorFactory extends AggregatorFactory {
@Override @Override
public Aggregator createInternal(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket, public Aggregator createInternal(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
ValuesSource vs = config.toValuesSource(); if (config.hasValues() == false) {
if (vs == null) {
return createUnmapped(searchContext, parent, metadata); return createUnmapped(searchContext, parent, metadata);
} }
return doCreateInternal(vs, searchContext, parent, collectsFromSingleBucket, metadata); return doCreateInternal(searchContext, parent, collectsFromSingleBucket, metadata);
} }
protected abstract Aggregator createUnmapped(SearchContext searchContext, protected abstract Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException; Map<String, Object> metadata) throws IOException;
protected abstract Aggregator doCreateInternal(ValuesSource valuesSource, protected abstract Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException; Map<String, Object> metadata) throws IOException;

View File

@ -154,10 +154,18 @@ public class ValuesSourceConfig {
if (valuesSourceType == null) { if (valuesSourceType == null) {
valuesSourceType = defaultValueSourceType; valuesSourceType = defaultValueSourceType;
} }
config = new ValuesSourceConfig(valuesSourceType, fieldContext, unmapped, aggregationScript, scriptValueType, context::nowInMillis); DocValueFormat docValueFormat = resolveFormat(format, valuesSourceType, timeZone, fieldType);
config.format(resolveFormat(format, valuesSourceType, timeZone, fieldType)); config = new ValuesSourceConfig(
config.missing(missing); valuesSourceType,
config.timezone(timeZone); fieldContext,
unmapped,
aggregationScript,
scriptValueType,
missing,
timeZone,
docValueFormat,
context::nowInMillis
);
return config; return config;
} }
@ -231,6 +239,9 @@ public class ValuesSourceConfig {
false, false,
null, null,
null, null,
null,
null,
null,
queryShardContext::nowInMillis queryShardContext::nowInMillis
); );
} }
@ -239,19 +250,23 @@ public class ValuesSourceConfig {
* Convenience method for creating unmapped configs * Convenience method for creating unmapped configs
*/ */
public static ValuesSourceConfig resolveUnmapped(ValuesSourceType valuesSourceType, QueryShardContext queryShardContext) { public static ValuesSourceConfig resolveUnmapped(ValuesSourceType valuesSourceType, QueryShardContext queryShardContext) {
return new ValuesSourceConfig(valuesSourceType, null, true, null, null, queryShardContext::nowInMillis); return new ValuesSourceConfig(valuesSourceType, null, true, null, null, null, null, null, queryShardContext::nowInMillis);
} }
private final ValuesSourceType valuesSourceType; private final ValuesSourceType valuesSourceType;
private FieldContext fieldContext; private final FieldContext fieldContext;
private AggregationScript.LeafFactory script; private final AggregationScript.LeafFactory script;
private ValueType scriptValueType; private final ValueType scriptValueType;
private boolean unmapped; private final boolean unmapped;
private DocValueFormat format = DocValueFormat.RAW; private final DocValueFormat format;
private Object missing; private final Object missing;
private ZoneId timeZone; private final ZoneId timeZone;
private LongSupplier nowSupplier; private final LongSupplier nowSupplier;
private final ValuesSource valuesSource;
private ValuesSourceConfig() {
throw new UnsupportedOperationException();
}
public ValuesSourceConfig( public ValuesSourceConfig(
ValuesSourceType valuesSourceType, ValuesSourceType valuesSourceType,
@ -259,6 +274,9 @@ public class ValuesSourceConfig {
boolean unmapped, boolean unmapped,
AggregationScript.LeafFactory script, AggregationScript.LeafFactory script,
ValueType scriptValueType, ValueType scriptValueType,
Object missing,
ZoneId timeZone,
DocValueFormat format,
LongSupplier nowSupplier LongSupplier nowSupplier
) { ) {
if (unmapped && fieldContext != null) { if (unmapped && fieldContext != null) {
@ -269,8 +287,38 @@ public class ValuesSourceConfig {
this.unmapped = unmapped; this.unmapped = unmapped;
this.script = script; this.script = script;
this.scriptValueType = scriptValueType; this.scriptValueType = scriptValueType;
this.missing = missing;
this.timeZone = timeZone;
this.format = format == null ? DocValueFormat.RAW : format;
this.nowSupplier = nowSupplier; this.nowSupplier = nowSupplier;
if (!valid()) {
// TODO: resolve no longer generates invalid configs. Once VSConfig is immutable, we can drop this check
throw new IllegalStateException(
"value source config is invalid; must have either a field context or a script or marked as unwrapped");
}
valuesSource = ConstructValuesSource(missing, format, nowSupplier);
}
private ValuesSource ConstructValuesSource(Object missing, DocValueFormat format, LongSupplier nowSupplier) {
final ValuesSource vs;
if (this.unmapped) {
vs = valueSourceType().getEmpty();
} else {
if (fieldContext() == null) {
// Script case
vs = valueSourceType().getScript(script(), scriptValueType());
} else {
// Field or Value Script case
vs = valueSourceType().getField(fieldContext(), script());
}
}
if (missing() != null) {
return valueSourceType().replaceMissing(vs, missing, format, nowSupplier);
} else {
return vs;
}
} }
public ValuesSourceType valueSourceType() { public ValuesSourceType valueSourceType() {
@ -285,8 +333,12 @@ public class ValuesSourceConfig {
return script; return script;
} }
public boolean unmapped() { /**
return unmapped; * Returns true if the values source configured by this object can yield values. We might not be able to yield values if, for example,
* the specified field does not exist on this index.
*/
public boolean hasValues() {
return fieldContext != null || script != null || missing != null;
} }
public boolean valid() { public boolean valid() {
@ -297,25 +349,10 @@ public class ValuesSourceConfig {
return this.scriptValueType; return this.scriptValueType;
} }
private ValuesSourceConfig format(final DocValueFormat format) {
this.format = format;
return this;
}
private ValuesSourceConfig missing(final Object missing) {
this.missing = missing;
return this;
}
public Object missing() { public Object missing() {
return this.missing; return this.missing;
} }
private ValuesSourceConfig timezone(final ZoneId timeZone) {
this.timeZone = timeZone;
return this;
}
public ZoneId timezone() { public ZoneId timezone() {
return this.timeZone; return this.timeZone;
} }
@ -324,41 +361,11 @@ public class ValuesSourceConfig {
return format; return format;
} }
/** public ValuesSource getValuesSource() {
* Transform the {@link ValuesSourceType} we selected in resolve into the specific {@link ValuesSource} instance to use for this shard return valuesSource;
* @return - A {@link ValuesSource} ready to be read from by an aggregator }
*/
@Nullable
public ValuesSource toValuesSource() {
if (!valid()) {
// TODO: resolve no longer generates invalid configs. Once VSConfig is immutable, we can drop this check
throw new IllegalStateException(
"value source config is invalid; must have either a field context or a script or marked as unwrapped");
}
final ValuesSource vs; public boolean hasGlobalOrdinals() {
if (unmapped()) { return valuesSource.hasGlobalOrdinals();
if (missing() == null) {
/* Null values source signals to the AggregationBuilder to use the createUnmapped method, which aggregator factories can
* override to provide an aggregator optimized to return empty values
*/
vs = null;
} else {
vs = valueSourceType().getEmpty();
}
} else {
if (fieldContext() == null) {
// Script case
vs = valueSourceType().getScript(script(), scriptValueType());
} else {
// Field or Value Script case
vs = valueSourceType().getField(fieldContext(), script());
}
}
if (missing() == null) {
return vs;
}
return valueSourceType().replaceMissing(vs, missing, format, nowSupplier);
} }
} }

View File

@ -83,6 +83,36 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}); });
} }
public void testUnmappedWeight() throws IOException {
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("value_field").build();
MultiValuesSourceFieldConfig weightConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("weight_field").build();
WeightedAvgAggregationBuilder aggregationBuilder = new WeightedAvgAggregationBuilder("_name")
.value(valueConfig)
.weight(weightConfig);
testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> {
iw.addDocument(singleton(new SortedNumericDocValuesField("value_field", 7)));
iw.addDocument(singleton(new SortedNumericDocValuesField("value_field", 3)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
public void testUnmappedValue() throws IOException {
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("value_field").build();
MultiValuesSourceFieldConfig weightConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("weight_field").build();
WeightedAvgAggregationBuilder aggregationBuilder = new WeightedAvgAggregationBuilder("_name")
.value(valueConfig)
.weight(weightConfig);
testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> {
iw.addDocument(singleton(new SortedNumericDocValuesField("weight_field", 7)));
iw.addDocument(singleton(new SortedNumericDocValuesField("weight_field", 3)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
public void testSomeMatchesSortedNumericDocValuesNoWeight() throws IOException { public void testSomeMatchesSortedNumericDocValuesNoWeight() throws IOException {
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("value_field").build(); MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("value_field").build();
MultiValuesSourceFieldConfig weightConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("weight_field").build(); MultiValuesSourceFieldConfig weightConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("weight_field").build();

View File

@ -47,7 +47,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.toValuesSource(); ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedBinaryDocValues values = valuesSource.bytesValues(ctx); SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -69,14 +69,14 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.toValuesSource(); ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedBinaryDocValues values = valuesSource.bytesValues(ctx); SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
assertFalse(values.advanceExact(0)); assertFalse(values.advanceExact(0));
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, null, "bytes", null, "abc", null, null, CoreValuesSourceType.BYTES, null); context, null, "bytes", null, "abc", null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Bytes) config.toValuesSource(); valuesSource = (ValuesSource.Bytes) config.getValuesSource();
values = valuesSource.bytesValues(ctx); values = valuesSource.bytesValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
assertEquals(1, values.docValueCount()); assertEquals(1, values.docValueCount());
@ -95,12 +95,13 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null); QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, ValueType.STRING, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.STRING, "bytes", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.toValuesSource(); ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.getValuesSource();
assertNull(valuesSource); assertNotNull(valuesSource);
assertFalse(config.hasValues());
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, ValueType.STRING, "bytes", null, "abc", null, null, CoreValuesSourceType.BYTES, null); context, ValueType.STRING, "bytes", null, "abc", null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Bytes) config.toValuesSource(); valuesSource = (ValuesSource.Bytes) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedBinaryDocValues values = valuesSource.bytesValues(ctx); SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -122,7 +123,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "long", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "long", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -144,14 +145,14 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "long", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "long", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertFalse(values.advanceExact(0)); assertFalse(values.advanceExact(0));
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, null, "long", null, 42, null, null, CoreValuesSourceType.BYTES, null); context, null, "long", null, 42, null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Numeric) config.toValuesSource(); valuesSource = (ValuesSource.Numeric) config.getValuesSource();
values = valuesSource.longValues(ctx); values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
assertEquals(1, values.docValueCount()); assertEquals(1, values.docValueCount());
@ -171,12 +172,13 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, ValueType.NUMBER, "long", null, null, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.NUMBER, "long", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
assertNull(valuesSource); assertNotNull(valuesSource);
assertFalse(config.hasValues());
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, ValueType.NUMBER, "long", null, 42, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.NUMBER, "long", null, 42, null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Numeric) config.toValuesSource(); valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -198,7 +200,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -220,14 +222,14 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, null, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null); context, null, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertFalse(values.advanceExact(0)); assertFalse(values.advanceExact(0));
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, null, "bool", null, true, null, null, CoreValuesSourceType.BYTES, null); context, null, "bool", null, true, null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Numeric) config.toValuesSource(); valuesSource = (ValuesSource.Numeric) config.getValuesSource();
values = valuesSource.longValues(ctx); values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
assertEquals(1, values.docValueCount()); assertEquals(1, values.docValueCount());
@ -247,12 +249,13 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, ValueType.BOOLEAN, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.BOOLEAN, "bool", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.toValuesSource(); ValuesSource.Numeric valuesSource = (ValuesSource.Numeric) config.getValuesSource();
assertNull(valuesSource); assertNotNull(valuesSource);
assertFalse(config.hasValues());
config = ValuesSourceConfig.resolve( config = ValuesSourceConfig.resolve(
context, ValueType.BOOLEAN, "bool", null, true, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.BOOLEAN, "bool", null, true, null, null, CoreValuesSourceType.BYTES, null);
valuesSource = (ValuesSource.Numeric) config.toValuesSource(); valuesSource = (ValuesSource.Numeric) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedNumericDocValues values = valuesSource.longValues(ctx); SortedNumericDocValues values = valuesSource.longValues(ctx);
assertTrue(values.advanceExact(0)); assertTrue(values.advanceExact(0));
@ -284,7 +287,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null); QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig config = ValuesSourceConfig.resolve( ValuesSourceConfig config = ValuesSourceConfig.resolve(
context, ValueType.STRING, "alias", null, null, null, null, CoreValuesSourceType.BYTES, null); context, ValueType.STRING, "alias", null, null, null, null, CoreValuesSourceType.BYTES, null);
ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.toValuesSource(); ValuesSource.Bytes valuesSource = (ValuesSource.Bytes) config.getValuesSource();
LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0); LeafReaderContext ctx = searcher.getIndexReader().leaves().get(0);
SortedBinaryDocValues values = valuesSource.bytesValues(ctx); SortedBinaryDocValues values = valuesSource.bytesValues(ctx);

View File

@ -14,10 +14,8 @@ import org.elasticsearch.search.aggregations.metrics.PercentilesConfig;
import org.elasticsearch.search.aggregations.metrics.PercentilesMethod; import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
import org.elasticsearch.xpack.analytics.aggregations.support.AnalyticsValuesSourceType; import org.elasticsearch.xpack.analytics.aggregations.support.AnalyticsValuesSourceType;
import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
public class AnalyticsAggregatorFactory { public class AnalyticsAggregatorFactory {
@ -68,36 +66,21 @@ public class AnalyticsAggregatorFactory {
public static void registerHistoBackedSumAggregator(ValuesSourceRegistry.Builder builder) { public static void registerHistoBackedSumAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(SumAggregationBuilder.NAME, builder.register(SumAggregationBuilder.NAME,
AnalyticsValuesSourceType.HISTOGRAM, AnalyticsValuesSourceType.HISTOGRAM,
(MetricAggregatorSupplier) (name, valuesSourceConfig, valuesSource, context, parent, metadata) -> new HistoBackedSumAggregator( (MetricAggregatorSupplier) HistoBackedSumAggregator::new
name,
(HistogramValuesSource.Histogram) valuesSource,
valuesSourceConfig.format(),
context,
parent,
metadata
)
); );
} }
public static void registerHistoBackedValueCountAggregator(ValuesSourceRegistry.Builder builder) { public static void registerHistoBackedValueCountAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(ValueCountAggregationBuilder.NAME, builder.register(ValueCountAggregationBuilder.NAME,
AnalyticsValuesSourceType.HISTOGRAM, AnalyticsValuesSourceType.HISTOGRAM,
(ValueCountAggregatorSupplier) (name, valuesSource, context, parent, metadata) -> (MetricAggregatorSupplier) HistoBackedValueCountAggregator::new
new HistoBackedValueCountAggregator(name, (HistogramValuesSource.Histogram) valuesSource, context, parent, metadata)
); );
} }
public static void registerHistoBackedAverageAggregator(ValuesSourceRegistry.Builder builder) { public static void registerHistoBackedAverageAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(AvgAggregationBuilder.NAME, builder.register(AvgAggregationBuilder.NAME,
AnalyticsValuesSourceType.HISTOGRAM, AnalyticsValuesSourceType.HISTOGRAM,
(MetricAggregatorSupplier) (name, valuesSourceConfig, valuesSource, context, parent, metadata) -> new HistoBackedAvgAggregator( (MetricAggregatorSupplier) HistoBackedAvgAggregator::new
name,
(HistogramValuesSource.Histogram) valuesSource,
valuesSourceConfig.format(),
context,
parent,
metadata
)
); );
} }
} }

View File

@ -21,6 +21,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.CompensatedSum; import org.elasticsearch.search.aggregations.metrics.CompensatedSum;
import org.elasticsearch.search.aggregations.metrics.InternalAvg; import org.elasticsearch.search.aggregations.metrics.InternalAvg;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource; import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
@ -40,11 +41,17 @@ class HistoBackedAvgAggregator extends NumericMetricsAggregator.SingleValue {
DoubleArray compensations; DoubleArray compensations;
DocValueFormat format; DocValueFormat format;
HistoBackedAvgAggregator(String name, HistogramValuesSource.Histogram valuesSource, DocValueFormat formatter, SearchContext context, HistoBackedAvgAggregator(
Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = valuesSource; // TODO: Stop depending on nulls here
this.format = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (HistogramValuesSource.Histogram) valuesSourceConfig.getValuesSource() : null;
this.format = valuesSourceConfig.format();
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();
counts = bigArrays.newLongArray(1, true); counts = bigArrays.newLongArray(1, true);

View File

@ -20,6 +20,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.CompensatedSum; import org.elasticsearch.search.aggregations.metrics.CompensatedSum;
import org.elasticsearch.search.aggregations.metrics.InternalSum; import org.elasticsearch.search.aggregations.metrics.InternalSum;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource; import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
@ -40,11 +41,17 @@ class HistoBackedSumAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray sums; private DoubleArray sums;
private DoubleArray compensations; private DoubleArray compensations;
HistoBackedSumAggregator(String name, HistogramValuesSource.Histogram valuesSource, DocValueFormat formatter, SearchContext context, HistoBackedSumAggregator(
Aggregator parent, Map<String, Object> metadata) throws IOException { String name,
ValuesSourceConfig valuesSourceConfig,
SearchContext context,
Aggregator parent,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = valuesSource; // TODO: stop expecting a null here
this.format = formatter; this.valuesSource = valuesSourceConfig.hasValues() ? (HistogramValuesSource.Histogram) valuesSourceConfig.getValuesSource() : null;
this.format = valuesSourceConfig.format();
if (valuesSource != null) { if (valuesSource != null) {
sums = context.bigArrays().newDoubleArray(1, true); sums = context.bigArrays().newDoubleArray(1, true);
compensations = context.bigArrays().newDoubleArray(1, true); compensations = context.bigArrays().newDoubleArray(1, true);

View File

@ -17,6 +17,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.InternalValueCount; import org.elasticsearch.search.aggregations.metrics.InternalValueCount;
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource; import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
@ -37,12 +38,13 @@ public class HistoBackedValueCountAggregator extends NumericMetricsAggregator.Si
public HistoBackedValueCountAggregator( public HistoBackedValueCountAggregator(
String name, String name,
HistogramValuesSource.Histogram valuesSource, ValuesSourceConfig valuesSourceConfig,
SearchContext aggregationContext, SearchContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
super(name, aggregationContext, parent, metadata); super(name, aggregationContext, parent, metadata);
this.valuesSource = valuesSource; // TODO: stop using nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (HistogramValuesSource.Histogram) valuesSourceConfig.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
counts = context.bigArrays().newLongArray(1, true); counts = context.bigArrays().newLongArray(1, true);
} }

View File

@ -13,7 +13,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -54,8 +53,7 @@ public class BoxplotAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -66,7 +64,7 @@ public class BoxplotAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected BoxplotAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected BoxplotAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((BoxplotAggregatorSupplier) aggregatorSupplier).build(name, valuesSource, config.format(), compression, return ((BoxplotAggregatorSupplier) aggregatorSupplier).build(name, config.getValuesSource(), config.format(), compression,
searchContext, parent, metadata); searchContext, parent, metadata);
} }
} }

View File

@ -13,7 +13,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.AggregatorSupplier; import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
@ -48,8 +47,7 @@ class StringStatsAggregatorFactory extends ValuesSourceAggregatorFactory {
} }
@Override @Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator doCreateInternal(SearchContext searchContext,
SearchContext searchContext,
Aggregator parent, Aggregator parent,
boolean collectsFromSingleBucket, boolean collectsFromSingleBucket,
Map<String, Object> metadata) throws IOException { Map<String, Object> metadata) throws IOException {
@ -60,7 +58,7 @@ class StringStatsAggregatorFactory extends ValuesSourceAggregatorFactory {
throw new AggregationExecutionException("Registry miss-match - expected StringStatsAggregatorSupplier, found [" + throw new AggregationExecutionException("Registry miss-match - expected StringStatsAggregatorSupplier, found [" +
aggregatorSupplier.getClass().toString() + "]"); aggregatorSupplier.getClass().toString() + "]");
} }
return ((StringStatsAggregatorSupplier) aggregatorSupplier).build(name, valuesSource, showDistribution, config.format(), return ((StringStatsAggregatorSupplier) aggregatorSupplier).build(name, config.getValuesSource(), showDistribution, config.format(),
searchContext, parent, metadata); searchContext, parent, metadata);
} }

View File

@ -63,7 +63,7 @@ public class TopMetricsAggregatorFactory extends AggregatorFactory {
config.getFieldName(), config.getScript(), config.getMissing(), config.getTimeZone(), null, config.getFieldName(), config.getScript(), config.getMissing(), config.getTimeZone(), null,
CoreValuesSourceType.NUMERIC, TopMetricsAggregationBuilder.NAME); CoreValuesSourceType.NUMERIC, TopMetricsAggregationBuilder.NAME);
return new TopMetricsAggregator.MetricSource(config.getFieldName(), resolved.format(), return new TopMetricsAggregator.MetricSource(config.getFieldName(), resolved.format(),
(ValuesSource.Numeric) resolved.toValuesSource()); (ValuesSource.Numeric) resolved.getValuesSource());
}).collect(toList()); }).collect(toList());
return new TopMetricsAggregator(name, searchContext, parent, metadata, size, sortBuilders.get(0), metricSources); return new TopMetricsAggregator(name, searchContext, parent, metadata, size, sortBuilders.get(0), metricSources);
} }

View File

@ -22,11 +22,10 @@ import org.elasticsearch.search.aggregations.metrics.CardinalityAggregatorSuppli
import org.elasticsearch.search.aggregations.metrics.GeoBoundsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.GeoBoundsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.GeoBoundsAggregatorSupplier; import org.elasticsearch.search.aggregations.metrics.GeoBoundsAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.GeoGridAggregatorSupplier; import org.elasticsearch.search.aggregations.metrics.GeoGridAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.MetricAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregator; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregator;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.spatial.aggregations.metrics.GeoShapeCentroidAggregator; import org.elasticsearch.xpack.spatial.aggregations.metrics.GeoShapeCentroidAggregator;
@ -101,17 +100,15 @@ public class SpatialPlugin extends GeoPlugin implements MapperPlugin, SearchPlug
private static void registerGeoShapeBoundsAggregator(ValuesSourceRegistry.Builder builder) { private static void registerGeoShapeBoundsAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(GeoBoundsAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(), builder.register(GeoBoundsAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(),
(GeoBoundsAggregatorSupplier) (name, aggregationContext, parent, valuesSource, wrapLongitude, metadata) (GeoBoundsAggregatorSupplier) GeoShapeBoundsAggregator::new);
-> new GeoShapeBoundsAggregator(name, aggregationContext, parent, (GeoShapeValuesSource) valuesSource,
wrapLongitude, metadata));
} }
private void registerGeoShapeCentroidAggregator(ValuesSourceRegistry.Builder builder) { private void registerGeoShapeCentroidAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(GeoCentroidAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(), builder.register(GeoCentroidAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(),
(GeoCentroidAggregatorSupplier) (name, aggregationContext, parent, valuesSource, metadata) (MetricAggregatorSupplier) (name, valuesSourceConfig, aggregationContext, parent, metadata)
-> { -> {
if (getLicenseState().isAllowed(XPackLicenseState.Feature.SPATIAL_GEO_CENTROID)) { if (getLicenseState().isAllowed(XPackLicenseState.Feature.SPATIAL_GEO_CENTROID)) {
return new GeoShapeCentroidAggregator(name, aggregationContext, parent, (GeoShapeValuesSource) valuesSource, metadata); return new GeoShapeCentroidAggregator(name, aggregationContext, parent, valuesSourceConfig, metadata);
} }
throw LicenseUtils.newComplianceException("geo_centroid aggregation on geo_shape fields"); throw LicenseUtils.newComplianceException("geo_centroid aggregation on geo_shape fields");
}); });
@ -161,7 +158,7 @@ public class SpatialPlugin extends GeoPlugin implements MapperPlugin, SearchPlug
private static void registerValueCountAggregator(ValuesSourceRegistry.Builder builder) { private static void registerValueCountAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(ValueCountAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(), builder.register(ValueCountAggregationBuilder.NAME, GeoShapeValuesSourceType.instance(),
(ValueCountAggregatorSupplier) ValueCountAggregator::new (MetricAggregatorSupplier) ValueCountAggregator::new
); );
} }

View File

@ -21,6 +21,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.CompensatedSum; import org.elasticsearch.search.aggregations.metrics.CompensatedSum;
import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroid; import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroid;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.xpack.spatial.index.fielddata.DimensionalShapeType; import org.elasticsearch.xpack.spatial.index.fielddata.DimensionalShapeType;
import org.elasticsearch.xpack.spatial.index.fielddata.MultiGeoShapeValues; import org.elasticsearch.xpack.spatial.index.fielddata.MultiGeoShapeValues;
@ -38,10 +39,16 @@ public final class GeoShapeCentroidAggregator extends MetricsAggregator {
private LongArray counts; private LongArray counts;
private ByteArray dimensionalShapeTypes; private ByteArray dimensionalShapeTypes;
public GeoShapeCentroidAggregator(String name, SearchContext context, Aggregator parent, public GeoShapeCentroidAggregator(
GeoShapeValuesSource valuesSource, Map<String, Object> metadata) throws IOException { String name,
SearchContext context,
Aggregator parent,
ValuesSourceConfig valuesSourceConfig,
Map<String, Object> metadata
) throws IOException {
super(name, context, parent, metadata); super(name, context, parent, metadata);
this.valuesSource = valuesSource; // TODO: stop expecting nulls here
this.valuesSource = valuesSourceConfig.hasValues() ? (GeoShapeValuesSource) valuesSourceConfig.getValuesSource() : null;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();
lonSum = bigArrays.newDoubleArray(1, true); lonSum = bigArrays.newDoubleArray(1, true);

View File

@ -16,6 +16,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.InternalGeoBounds; import org.elasticsearch.search.aggregations.metrics.InternalGeoBounds;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.xpack.spatial.index.fielddata.MultiGeoShapeValues; import org.elasticsearch.xpack.spatial.index.fielddata.MultiGeoShapeValues;
import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoShapeValuesSource; import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoShapeValuesSource;
@ -33,10 +34,16 @@ public final class GeoShapeBoundsAggregator extends MetricsAggregator {
private DoubleArray negLefts; private DoubleArray negLefts;
private DoubleArray negRights; private DoubleArray negRights;
public GeoShapeBoundsAggregator(String name, SearchContext aggregationContext, Aggregator parent, public GeoShapeBoundsAggregator(
GeoShapeValuesSource valuesSource, boolean wrapLongitude, Map<String, Object> metadata) throws IOException { String name,
SearchContext aggregationContext,
Aggregator parent,
ValuesSourceConfig valuesSourceConfig,
boolean wrapLongitude,
Map<String, Object> metadata
) throws IOException {
super(name, aggregationContext, parent, metadata); super(name, aggregationContext, parent, metadata);
this.valuesSource = valuesSource; this.valuesSource = valuesSourceConfig.hasValues() ? (GeoShapeValuesSource) valuesSourceConfig.getValuesSource() : null;
this.wrapLongitude = wrapLongitude; this.wrapLongitude = wrapLongitude;
if (valuesSource != null) { if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays(); final BigArrays bigArrays = context.bigArrays();

View File

@ -12,8 +12,8 @@ import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileGridAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.GeoGridAggregatorSupplier; import org.elasticsearch.search.aggregations.metrics.GeoGridAggregatorSupplier;
import org.elasticsearch.search.aggregations.metrics.MetricAggregatorSupplier;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -35,8 +35,8 @@ public class SpatialPluginTests extends ESTestCase {
List<Consumer<ValuesSourceRegistry.Builder>> registrar = plugin.getAggregationExtentions(); List<Consumer<ValuesSourceRegistry.Builder>> registrar = plugin.getAggregationExtentions();
registrar.forEach(c -> c.accept(registryBuilder)); registrar.forEach(c -> c.accept(registryBuilder));
ValuesSourceRegistry registry = registryBuilder.build(); ValuesSourceRegistry registry = registryBuilder.build();
GeoCentroidAggregatorSupplier centroidSupplier = (GeoCentroidAggregatorSupplier) registry.getAggregator( MetricAggregatorSupplier centroidSupplier = (MetricAggregatorSupplier) registry.getAggregator(
new ValuesSourceConfig(GeoShapeValuesSourceType.instance(), null, false, null, null, null), new ValuesSourceConfig(GeoShapeValuesSourceType.instance(), null, true, null, null, null, null, null, null),
GeoCentroidAggregationBuilder.NAME); GeoCentroidAggregationBuilder.NAME);
if (License.OperationMode.TRIAL != operationMode && if (License.OperationMode.TRIAL != operationMode &&
License.OperationMode.compare(operationMode, License.OperationMode.GOLD) < 0) { License.OperationMode.compare(operationMode, License.OperationMode.GOLD) < 0) {
@ -57,7 +57,7 @@ public class SpatialPluginTests extends ESTestCase {
registrar.forEach(c -> c.accept(registryBuilder)); registrar.forEach(c -> c.accept(registryBuilder));
ValuesSourceRegistry registry = registryBuilder.build(); ValuesSourceRegistry registry = registryBuilder.build();
GeoGridAggregatorSupplier supplier = (GeoGridAggregatorSupplier) registry.getAggregator( GeoGridAggregatorSupplier supplier = (GeoGridAggregatorSupplier) registry.getAggregator(
new ValuesSourceConfig(GeoShapeValuesSourceType.instance(), null, false, null, null, null), new ValuesSourceConfig(GeoShapeValuesSourceType.instance(), null, true, null, null, null, null, null, null),
builderName); builderName);
if (License.OperationMode.TRIAL != operationMode && if (License.OperationMode.TRIAL != operationMode &&
License.OperationMode.compare(operationMode, License.OperationMode.GOLD) < 0) { License.OperationMode.compare(operationMode, License.OperationMode.GOLD) < 0) {