Moved AggregatorBuilder implementations into their own class files
Also renamed histogram.AbstractBuilcer to AbstractHistogramBuilder, range.AbstractBuilder to AbstractRangeBuilder and org.elasticsearch.search.aggregations.pipeline.having to org.elasticsearch.search.aggregations.pipeline.bucketselector
This commit is contained in:
parent
e546db0753
commit
ebc745211d
|
@ -192,13 +192,13 @@ import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum.SumBucke
|
|||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum.SumBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketselector.BucketSelectorParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketselector.BucketSelectorPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativeParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.derivative.InternalDerivative;
|
||||
import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgParser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel;
|
||||
|
|
|
@ -22,65 +22,65 @@ import org.elasticsearch.common.geo.GeoDistance;
|
|||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.children.Children;
|
||||
import org.elasticsearch.search.aggregations.bucket.children.ParentToChildrenAggregator.ChildrenAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregator.FilterAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filters.Filters;
|
||||
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.FiltersAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
|
||||
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid;
|
||||
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridParser.GeoGridAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.global.Global;
|
||||
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator.GlobalAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.HistogramAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.missing.Missing;
|
||||
import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregator.MissingAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.Nested;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregator.NestedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNestedAggregator.ReverseNestedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNestedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.Range;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.RangeAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.date.DateRangeAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser.GeoDistanceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.ipv4.IPv4RangeAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.DiversifiedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.Sampler;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.DiversifiedAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.SamplerAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms;
|
||||
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTermsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator.AvgAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
|
||||
import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregator.GeoBoundsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroid;
|
||||
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator.GeoCentroidAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.Max;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator.MaxAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.MaxAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.min.Min;
|
||||
import org.elasticsearch.search.aggregations.metrics.min.MinAggregator.MinAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.min.MinAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
|
||||
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles;
|
||||
import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetric;
|
||||
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.scripted.ScriptedMetricAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.StatsAggregator.StatsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.StatsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregator.SumAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.tophits.TopHits;
|
||||
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregator.TopHitsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountAggregator.ValueCountAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountAggregatorBuilder;
|
||||
|
||||
/**
|
||||
* Utility class to create aggregations.
|
||||
|
|
|
@ -0,0 +1,132 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.children;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ChildrenAggregatorBuilder extends ValuesSourceAggregatorBuilder<ParentChild, ChildrenAggregatorBuilder> {
|
||||
|
||||
static final ChildrenAggregatorBuilder PROTOTYPE = new ChildrenAggregatorBuilder("", "");
|
||||
|
||||
private String parentType;
|
||||
private final String childType;
|
||||
private Query parentFilter;
|
||||
private Query childFilter;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param childType
|
||||
* the type of children documents
|
||||
*/
|
||||
public ChildrenAggregatorBuilder(String name, String childType) {
|
||||
super(name, InternalChildren.TYPE, ValuesSourceType.BYTES, ValueType.STRING);
|
||||
if (childType == null) {
|
||||
throw new IllegalArgumentException("[childType] must not be null: [" + name + "]");
|
||||
}
|
||||
this.childType = childType;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ParentChild> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new ChildrenAggregatorFactory(name, type, config, parentType, childFilter, parentFilter, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceConfig<ParentChild> resolveConfig(AggregationContext aggregationContext) {
|
||||
ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
|
||||
DocumentMapper childDocMapper = aggregationContext.searchContext().mapperService().documentMapper(childType);
|
||||
|
||||
if (childDocMapper != null) {
|
||||
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
|
||||
if (!parentFieldMapper.active()) {
|
||||
throw new IllegalArgumentException("[children] no [_parent] field not configured that points to a parent type");
|
||||
}
|
||||
parentType = parentFieldMapper.type();
|
||||
DocumentMapper parentDocMapper = aggregationContext.searchContext().mapperService().documentMapper(parentType);
|
||||
if (parentDocMapper != null) {
|
||||
parentFilter = parentDocMapper.typeFilter();
|
||||
childFilter = childDocMapper.typeFilter();
|
||||
ParentChildIndexFieldData parentChildIndexFieldData = aggregationContext.searchContext().fieldData()
|
||||
.getForField(parentFieldMapper.fieldType());
|
||||
config.fieldContext(new FieldContext(parentFieldMapper.fieldType().name(), parentChildIndexFieldData,
|
||||
parentFieldMapper.fieldType()));
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ParentToChildrenAggregator.TYPE_FIELD.getPreferredName(), childType);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ChildrenAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
String childType = in.readString();
|
||||
ChildrenAggregatorBuilder factory = new ChildrenAggregatorBuilder(name, childType);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeString(childType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(childType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
ChildrenAggregatorBuilder other = (ChildrenAggregatorBuilder) obj;
|
||||
return Objects.equals(childType, other.childType);
|
||||
}
|
||||
|
||||
}
|
|
@ -35,7 +35,7 @@ public class ChildrenParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ParentToChildrenAggregator.ChildrenAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
public ChildrenAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
QueryParseContext context) throws IOException {
|
||||
String childType = null;
|
||||
|
||||
|
@ -62,11 +62,11 @@ public class ChildrenParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
|
||||
return new ParentToChildrenAggregator.ChildrenAggregatorBuilder(aggregationName, childType);
|
||||
return new ChildrenAggregatorBuilder(aggregationName, childType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParentToChildrenAggregator.ChildrenAggregatorBuilder getFactoryPrototypes() {
|
||||
return ParentToChildrenAggregator.ChildrenAggregatorBuilder.PROTOTYPE;
|
||||
public ChildrenAggregatorBuilder getFactoryPrototypes() {
|
||||
return ChildrenAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,39 +28,23 @@ import org.apache.lucene.search.Scorer;
|
|||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.util.LongObjectPagedHashMap;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
// The RecordingPerReaderBucketCollector assumes per segment recording which isn't the case for this
|
||||
// aggregation, for this reason that collector can't be used
|
||||
|
@ -190,94 +174,4 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
|
|||
protected void doClose() {
|
||||
Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets);
|
||||
}
|
||||
|
||||
public static class ChildrenAggregatorBuilder extends ValuesSourceAggregatorBuilder<ParentChild, ChildrenAggregatorBuilder> {
|
||||
|
||||
static final ChildrenAggregatorBuilder PROTOTYPE = new ChildrenAggregatorBuilder("", "");
|
||||
|
||||
private String parentType;
|
||||
private final String childType;
|
||||
private Query parentFilter;
|
||||
private Query childFilter;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param childType
|
||||
* the type of children documents
|
||||
*/
|
||||
public ChildrenAggregatorBuilder(String name, String childType) {
|
||||
super(name, InternalChildren.TYPE, ValuesSourceType.BYTES, ValueType.STRING);
|
||||
if (childType == null) {
|
||||
throw new IllegalArgumentException("[childType] must not be null: [" + name + "]");
|
||||
}
|
||||
this.childType = childType;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ParentChild> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new ChildrenAggregatorFactory(name, type, config, parentType, childFilter, parentFilter, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceConfig<ParentChild> resolveConfig(AggregationContext aggregationContext) {
|
||||
ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
|
||||
DocumentMapper childDocMapper = aggregationContext.searchContext().mapperService().documentMapper(childType);
|
||||
|
||||
if (childDocMapper != null) {
|
||||
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
|
||||
if (!parentFieldMapper.active()) {
|
||||
throw new IllegalArgumentException("[children] no [_parent] field not configured that points to a parent type");
|
||||
}
|
||||
parentType = parentFieldMapper.type();
|
||||
DocumentMapper parentDocMapper = aggregationContext.searchContext().mapperService().documentMapper(parentType);
|
||||
if (parentDocMapper != null) {
|
||||
parentFilter = parentDocMapper.typeFilter();
|
||||
childFilter = childDocMapper.typeFilter();
|
||||
ParentChildIndexFieldData parentChildIndexFieldData = aggregationContext.searchContext().fieldData()
|
||||
.getForField(parentFieldMapper.fieldType());
|
||||
config.fieldContext(new FieldContext(parentFieldMapper.fieldType().name(), parentChildIndexFieldData,
|
||||
parentFieldMapper.fieldType()));
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
} else {
|
||||
config.unmapped(true);
|
||||
}
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(TYPE_FIELD.getPreferredName(), childType);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ChildrenAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
String childType = in.readString();
|
||||
ChildrenAggregatorBuilder factory = new ChildrenAggregatorBuilder(name, childType);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeString(childType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(childType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
ChildrenAggregatorBuilder other = (ChildrenAggregatorBuilder) obj;
|
||||
return Objects.equals(childType, other.childType);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,16 +21,9 @@ package org.elasticsearch.search.aggregations.bucket.filter;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.query.EmptyQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
|
@ -41,7 +34,6 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Aggregate all docs that match a filter.
|
||||
|
@ -85,66 +77,6 @@ public class FilterAggregator extends SingleBucketAggregator {
|
|||
public InternalAggregation buildEmptyAggregation() {
|
||||
return new InternalFilter(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class FilterAggregatorBuilder extends AggregatorBuilder<FilterAggregatorBuilder> {
|
||||
|
||||
static final FilterAggregatorBuilder PROTOTYPE = new FilterAggregatorBuilder("", EmptyQueryBuilder.PROTOTYPE);
|
||||
|
||||
private final QueryBuilder<?> filter;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filter
|
||||
* Set the filter to use, only documents that match this
|
||||
* filter will fall into the bucket defined by this
|
||||
* {@link Filter} aggregation.
|
||||
*/
|
||||
public FilterAggregatorBuilder(String name, QueryBuilder<?> filter) {
|
||||
super(name, InternalFilter.TYPE);
|
||||
if (filter == null) {
|
||||
throw new IllegalArgumentException("[filter] must not be null: [" + name + "]");
|
||||
}
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
|
||||
return new FilterAggregatorFactory(name, type, filter, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
if (filter != null) {
|
||||
filter.toXContent(builder, params);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FilterAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
FilterAggregatorBuilder factory = new FilterAggregatorBuilder(name, in.readQuery());
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeQuery(filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
FilterAggregatorBuilder other = (FilterAggregatorBuilder) obj;
|
||||
return Objects.equals(filter, other.filter);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.filter;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.query.EmptyQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class FilterAggregatorBuilder extends AggregatorBuilder<FilterAggregatorBuilder> {
|
||||
|
||||
static final FilterAggregatorBuilder PROTOTYPE = new FilterAggregatorBuilder("", EmptyQueryBuilder.PROTOTYPE);
|
||||
|
||||
private final QueryBuilder<?> filter;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filter
|
||||
* Set the filter to use, only documents that match this
|
||||
* filter will fall into the bucket defined by this
|
||||
* {@link Filter} aggregation.
|
||||
*/
|
||||
public FilterAggregatorBuilder(String name, QueryBuilder<?> filter) {
|
||||
super(name, InternalFilter.TYPE);
|
||||
if (filter == null) {
|
||||
throw new IllegalArgumentException("[filter] must not be null: [" + name + "]");
|
||||
}
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
|
||||
return new FilterAggregatorFactory(name, type, filter, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
if (filter != null) {
|
||||
filter.toXContent(builder, params);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FilterAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
FilterAggregatorBuilder factory = new FilterAggregatorBuilder(name, in.readQuery());
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeQuery(filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
FilterAggregatorBuilder other = (FilterAggregatorBuilder) obj;
|
||||
return Objects.equals(filter, other.filter);
|
||||
}
|
||||
|
||||
}
|
|
@ -37,7 +37,7 @@ public class FilterParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FilterAggregator.FilterAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public FilterAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
QueryBuilder<?> filter = context.parseInnerQueryBuilder();
|
||||
|
||||
|
@ -45,14 +45,14 @@ public class FilterParser implements Aggregator.Parser {
|
|||
throw new ParsingException(null, "filter cannot be null in filter aggregation [{}]", aggregationName);
|
||||
}
|
||||
|
||||
FilterAggregator.FilterAggregatorBuilder factory = new FilterAggregator.FilterAggregatorBuilder(aggregationName,
|
||||
FilterAggregatorBuilder factory = new FilterAggregatorBuilder(aggregationName,
|
||||
filter == null ? new MatchAllQueryBuilder() : filter);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FilterAggregator.FilterAggregatorBuilder getFactoryPrototypes() {
|
||||
return FilterAggregator.FilterAggregatorBuilder.PROTOTYPE;
|
||||
public FilterAggregatorBuilder getFactoryPrototypes() {
|
||||
return FilterAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -33,20 +33,16 @@ import org.elasticsearch.index.query.EmptyQueryBuilder;
|
|||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -205,165 +201,4 @@ public class FiltersAggregator extends BucketsAggregator {
|
|||
return owningBucketOrdinal * totalNumKeys + filterOrd;
|
||||
}
|
||||
|
||||
public static class FiltersAggregatorBuilder extends AggregatorBuilder<FiltersAggregatorBuilder> {
|
||||
|
||||
static final FiltersAggregatorBuilder PROTOTYPE = new FiltersAggregatorBuilder("", EmptyQueryBuilder.PROTOTYPE);
|
||||
|
||||
private final List<KeyedFilter> filters;
|
||||
private final boolean keyed;
|
||||
private boolean otherBucket = false;
|
||||
private String otherBucketKey = "_other_";
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filters
|
||||
* the KeyedFilters to use with this aggregation.
|
||||
*/
|
||||
public FiltersAggregatorBuilder(String name, KeyedFilter... filters) {
|
||||
this(name, Arrays.asList(filters));
|
||||
}
|
||||
|
||||
private FiltersAggregatorBuilder(String name, List<KeyedFilter> filters) {
|
||||
super(name, InternalFilters.TYPE);
|
||||
this.filters = filters;
|
||||
this.keyed = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filters
|
||||
* the filters to use with this aggregation
|
||||
*/
|
||||
public FiltersAggregatorBuilder(String name, QueryBuilder<?>... filters) {
|
||||
super(name, InternalFilters.TYPE);
|
||||
List<KeyedFilter> keyedFilters = new ArrayList<>(filters.length);
|
||||
for (int i = 0; i < filters.length; i++) {
|
||||
keyedFilters.add(new KeyedFilter(String.valueOf(i), filters[i]));
|
||||
}
|
||||
this.filters = keyedFilters;
|
||||
this.keyed = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set whether to include a bucket for documents not matching any filter
|
||||
*/
|
||||
public FiltersAggregatorBuilder otherBucket(boolean otherBucket) {
|
||||
this.otherBucket = otherBucket;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get whether to include a bucket for documents not matching any filter
|
||||
*/
|
||||
public boolean otherBucket() {
|
||||
return otherBucket;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the key to use for the bucket for documents not matching any
|
||||
* filter.
|
||||
*/
|
||||
public FiltersAggregatorBuilder otherBucketKey(String otherBucketKey) {
|
||||
if (otherBucketKey == null) {
|
||||
throw new IllegalArgumentException("[otherBucketKey] must not be null: [" + name + "]");
|
||||
}
|
||||
this.otherBucketKey = otherBucketKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the key to use for the bucket for documents not matching any
|
||||
* filter.
|
||||
*/
|
||||
public String otherBucketKey() {
|
||||
return otherBucketKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new FiltersAggregatorFactory(name, type, filters, keyed, otherBucket, otherBucketKey, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (keyed) {
|
||||
builder.startObject(FILTERS_FIELD.getPreferredName());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
builder.field(keyedFilter.key(), keyedFilter.filter());
|
||||
}
|
||||
builder.endObject();
|
||||
} else {
|
||||
builder.startArray(FILTERS_FIELD.getPreferredName());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
builder.value(keyedFilter.filter());
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
builder.field(OTHER_BUCKET_FIELD.getPreferredName(), otherBucket);
|
||||
builder.field(OTHER_BUCKET_KEY_FIELD.getPreferredName(), otherBucketKey);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FiltersAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
FiltersAggregatorBuilder factory;
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<KeyedFilter> filters = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
filters.add(KeyedFilter.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory = new FiltersAggregatorBuilder(name, filters);
|
||||
} else {
|
||||
int size = in.readVInt();
|
||||
QueryBuilder<?>[] filters = new QueryBuilder<?>[size];
|
||||
for (int i = 0; i < size; i++) {
|
||||
filters[i] = in.readQuery();
|
||||
}
|
||||
factory = new FiltersAggregatorBuilder(name, filters);
|
||||
}
|
||||
factory.otherBucket = in.readBoolean();
|
||||
factory.otherBucketKey = in.readString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(keyed);
|
||||
if (keyed) {
|
||||
out.writeVInt(filters.size());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
keyedFilter.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeVInt(filters.size());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
out.writeQuery(keyedFilter.filter());
|
||||
}
|
||||
}
|
||||
out.writeBoolean(otherBucket);
|
||||
out.writeString(otherBucketKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(filters, keyed, otherBucket, otherBucketKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
FiltersAggregatorBuilder other = (FiltersAggregatorBuilder) obj;
|
||||
return Objects.equals(filters, other.filters)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(otherBucket, other.otherBucket)
|
||||
&& Objects.equals(otherBucketKey, other.otherBucketKey);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,198 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.filters;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.query.EmptyQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class FiltersAggregatorBuilder extends AggregatorBuilder<FiltersAggregatorBuilder> {
|
||||
|
||||
static final FiltersAggregatorBuilder PROTOTYPE = new FiltersAggregatorBuilder("", EmptyQueryBuilder.PROTOTYPE);
|
||||
|
||||
private final List<KeyedFilter> filters;
|
||||
private final boolean keyed;
|
||||
private boolean otherBucket = false;
|
||||
private String otherBucketKey = "_other_";
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filters
|
||||
* the KeyedFilters to use with this aggregation.
|
||||
*/
|
||||
public FiltersAggregatorBuilder(String name, KeyedFilter... filters) {
|
||||
this(name, Arrays.asList(filters));
|
||||
}
|
||||
|
||||
private FiltersAggregatorBuilder(String name, List<KeyedFilter> filters) {
|
||||
super(name, InternalFilters.TYPE);
|
||||
this.filters = filters;
|
||||
this.keyed = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param filters
|
||||
* the filters to use with this aggregation
|
||||
*/
|
||||
public FiltersAggregatorBuilder(String name, QueryBuilder<?>... filters) {
|
||||
super(name, InternalFilters.TYPE);
|
||||
List<KeyedFilter> keyedFilters = new ArrayList<>(filters.length);
|
||||
for (int i = 0; i < filters.length; i++) {
|
||||
keyedFilters.add(new KeyedFilter(String.valueOf(i), filters[i]));
|
||||
}
|
||||
this.filters = keyedFilters;
|
||||
this.keyed = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set whether to include a bucket for documents not matching any filter
|
||||
*/
|
||||
public FiltersAggregatorBuilder otherBucket(boolean otherBucket) {
|
||||
this.otherBucket = otherBucket;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get whether to include a bucket for documents not matching any filter
|
||||
*/
|
||||
public boolean otherBucket() {
|
||||
return otherBucket;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the key to use for the bucket for documents not matching any
|
||||
* filter.
|
||||
*/
|
||||
public FiltersAggregatorBuilder otherBucketKey(String otherBucketKey) {
|
||||
if (otherBucketKey == null) {
|
||||
throw new IllegalArgumentException("[otherBucketKey] must not be null: [" + name + "]");
|
||||
}
|
||||
this.otherBucketKey = otherBucketKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the key to use for the bucket for documents not matching any
|
||||
* filter.
|
||||
*/
|
||||
public String otherBucketKey() {
|
||||
return otherBucketKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new FiltersAggregatorFactory(name, type, filters, keyed, otherBucket, otherBucketKey, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (keyed) {
|
||||
builder.startObject(FiltersAggregator.FILTERS_FIELD.getPreferredName());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
builder.field(keyedFilter.key(), keyedFilter.filter());
|
||||
}
|
||||
builder.endObject();
|
||||
} else {
|
||||
builder.startArray(FiltersAggregator.FILTERS_FIELD.getPreferredName());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
builder.value(keyedFilter.filter());
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
builder.field(FiltersAggregator.OTHER_BUCKET_FIELD.getPreferredName(), otherBucket);
|
||||
builder.field(FiltersAggregator.OTHER_BUCKET_KEY_FIELD.getPreferredName(), otherBucketKey);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FiltersAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
FiltersAggregatorBuilder factory;
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<KeyedFilter> filters = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
filters.add(KeyedFilter.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory = new FiltersAggregatorBuilder(name, filters);
|
||||
} else {
|
||||
int size = in.readVInt();
|
||||
QueryBuilder<?>[] filters = new QueryBuilder<?>[size];
|
||||
for (int i = 0; i < size; i++) {
|
||||
filters[i] = in.readQuery();
|
||||
}
|
||||
factory = new FiltersAggregatorBuilder(name, filters);
|
||||
}
|
||||
factory.otherBucket = in.readBoolean();
|
||||
factory.otherBucketKey = in.readString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(keyed);
|
||||
if (keyed) {
|
||||
out.writeVInt(filters.size());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
keyedFilter.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeVInt(filters.size());
|
||||
for (KeyedFilter keyedFilter : filters) {
|
||||
out.writeQuery(keyedFilter.filter());
|
||||
}
|
||||
}
|
||||
out.writeBoolean(otherBucket);
|
||||
out.writeString(otherBucketKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(filters, keyed, otherBucket, otherBucketKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
FiltersAggregatorBuilder other = (FiltersAggregatorBuilder) obj;
|
||||
return Objects.equals(filters, other.filters)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(otherBucket, other.otherBucket)
|
||||
&& Objects.equals(otherBucketKey, other.otherBucketKey);
|
||||
}
|
||||
}
|
|
@ -53,7 +53,7 @@ public class FiltersParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FiltersAggregator.FiltersAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public FiltersAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
|
||||
List<FiltersAggregator.KeyedFilter> keyedFilters = null;
|
||||
|
@ -124,12 +124,12 @@ public class FiltersParser implements Aggregator.Parser {
|
|||
otherBucketKey = "_other_";
|
||||
}
|
||||
|
||||
FiltersAggregator.FiltersAggregatorBuilder factory;
|
||||
FiltersAggregatorBuilder factory;
|
||||
if (keyedFilters != null) {
|
||||
factory = new FiltersAggregator.FiltersAggregatorBuilder(aggregationName,
|
||||
factory = new FiltersAggregatorBuilder(aggregationName,
|
||||
keyedFilters.toArray(new FiltersAggregator.KeyedFilter[keyedFilters.size()]));
|
||||
} else {
|
||||
factory = new FiltersAggregator.FiltersAggregatorBuilder(aggregationName,
|
||||
factory = new FiltersAggregatorBuilder(aggregationName,
|
||||
nonKeyedFilters.toArray(new QueryBuilder<?>[nonKeyedFilters.size()]));
|
||||
}
|
||||
if (otherBucket != null) {
|
||||
|
@ -142,8 +142,8 @@ public class FiltersParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FiltersAggregator.FiltersAggregatorBuilder getFactoryPrototypes() {
|
||||
return FiltersAggregator.FiltersAggregatorBuilder.PROTOTYPE;
|
||||
public FiltersAggregatorBuilder getFactoryPrototypes() {
|
||||
return FiltersAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,221 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.geogrid;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.spatial.util.GeoHashUtils;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
|
||||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.index.fielddata.SortingNumericDocValues;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class GeoGridAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoGridAggregatorBuilder> {
|
||||
|
||||
static final GeoGridAggregatorBuilder PROTOTYPE = new GeoGridAggregatorBuilder("");
|
||||
|
||||
private int precision = GeoHashGridParser.DEFAULT_PRECISION;
|
||||
private int requiredSize = GeoHashGridParser.DEFAULT_MAX_NUM_CELLS;
|
||||
private int shardSize = -1;
|
||||
|
||||
public GeoGridAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoHashGrid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder precision(int precision) {
|
||||
this.precision = GeoHashGridParams.checkPrecision(precision);
|
||||
return this;
|
||||
}
|
||||
|
||||
public int precision() {
|
||||
return precision;
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder size(int size) {
|
||||
if (size < -1) {
|
||||
throw new IllegalArgumentException(
|
||||
"[size] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.requiredSize = size;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return requiredSize;
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder shardSize(int shardSize) {
|
||||
if (shardSize < -1) {
|
||||
throw new IllegalArgumentException(
|
||||
"[shardSize] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
int shardSize = this.shardSize;
|
||||
if (shardSize == 0) {
|
||||
shardSize = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
int requiredSize = this.requiredSize;
|
||||
if (requiredSize == 0) {
|
||||
requiredSize = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
if (shardSize < 0) {
|
||||
// Use default heuristic to avoid any wrong-ranking caused by distributed counting
|
||||
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.searchContext().numberOfShards());
|
||||
}
|
||||
|
||||
if (shardSize < requiredSize) {
|
||||
shardSize = requiredSize;
|
||||
}
|
||||
return new GeoHashGridAggregatorFactory(name, type, config, precision, requiredSize, shardSize, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoGridAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType,
|
||||
StreamInput in) throws IOException {
|
||||
GeoGridAggregatorBuilder factory = new GeoGridAggregatorBuilder(name);
|
||||
factory.precision = in.readVInt();
|
||||
factory.requiredSize = in.readVInt();
|
||||
factory.shardSize = in.readVInt();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(precision);
|
||||
out.writeVInt(requiredSize);
|
||||
out.writeVInt(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(GeoHashGridParams.FIELD_PRECISION.getPreferredName(), precision);
|
||||
builder.field(GeoHashGridParams.FIELD_SIZE.getPreferredName(), requiredSize);
|
||||
builder.field(GeoHashGridParams.FIELD_SHARD_SIZE.getPreferredName(), shardSize);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoGridAggregatorBuilder other = (GeoGridAggregatorBuilder) obj;
|
||||
if (precision != other.precision) {
|
||||
return false;
|
||||
}
|
||||
if (requiredSize != other.requiredSize) {
|
||||
return false;
|
||||
}
|
||||
if (shardSize != other.shardSize) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(precision, requiredSize, shardSize);
|
||||
}
|
||||
|
||||
private static class CellValues extends SortingNumericDocValues {
|
||||
private MultiGeoPointValues geoValues;
|
||||
private int precision;
|
||||
|
||||
protected CellValues(MultiGeoPointValues geoValues, int precision) {
|
||||
this.geoValues = geoValues;
|
||||
this.precision = precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docId) {
|
||||
geoValues.setDocument(docId);
|
||||
resize(geoValues.count());
|
||||
for (int i = 0; i < count(); ++i) {
|
||||
GeoPoint target = geoValues.valueAt(i);
|
||||
values[i] = GeoHashUtils.longEncode(target.getLon(), target.getLat(), precision);
|
||||
}
|
||||
sort();
|
||||
}
|
||||
}
|
||||
|
||||
static class CellIdSource extends ValuesSource.Numeric {
|
||||
private final ValuesSource.GeoPoint valuesSource;
|
||||
private final int precision;
|
||||
|
||||
public CellIdSource(ValuesSource.GeoPoint valuesSource, int precision) {
|
||||
this.valuesSource = valuesSource;
|
||||
//different GeoPoints could map to the same or different geohash cells.
|
||||
this.precision = precision;
|
||||
}
|
||||
|
||||
public int precision() {
|
||||
return precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFloatingPoint() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDocValues longValues(LeafReaderContext ctx) {
|
||||
return new CellValues(valuesSource.geoPointValues(ctx), precision);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -46,10 +46,10 @@ public class GeoHashGridAggregator extends BucketsAggregator {
|
|||
|
||||
private final int requiredSize;
|
||||
private final int shardSize;
|
||||
private final GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource valuesSource;
|
||||
private final GeoGridAggregatorBuilder.CellIdSource valuesSource;
|
||||
private final LongHash bucketOrds;
|
||||
|
||||
public GeoHashGridAggregator(String name, AggregatorFactories factories, GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource valuesSource,
|
||||
public GeoHashGridAggregator(String name, AggregatorFactories factories, GeoGridAggregatorBuilder.CellIdSource valuesSource,
|
||||
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
|
||||
|
|
|
@ -23,22 +23,21 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridParser.GeoGridAggregatorBuilder.CellIdSource;
|
||||
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregatorBuilder.CellIdSource;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.GeoPoint;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
||||
public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoHashGridAggregatorFactory> {
|
||||
|
||||
private final int precision;
|
||||
|
|
|
@ -18,37 +18,17 @@
|
|||
*/
|
||||
package org.elasticsearch.search.aggregations.bucket.geogrid;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.spatial.util.GeoHashUtils;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser.Token;
|
||||
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
|
||||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.index.fielddata.SortingNumericDocValues;
|
||||
import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
|
||||
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Aggregates Geo information into cells determined by geohashes of a given precision.
|
||||
|
@ -110,180 +90,4 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static class GeoGridAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoGridAggregatorBuilder> {
|
||||
|
||||
static final GeoGridAggregatorBuilder PROTOTYPE = new GeoGridAggregatorBuilder("");
|
||||
|
||||
private int precision = DEFAULT_PRECISION;
|
||||
private int requiredSize = DEFAULT_MAX_NUM_CELLS;
|
||||
private int shardSize = -1;
|
||||
|
||||
public GeoGridAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoHashGrid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder precision(int precision) {
|
||||
this.precision = GeoHashGridParams.checkPrecision(precision);
|
||||
return this;
|
||||
}
|
||||
|
||||
public int precision() {
|
||||
return precision;
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder size(int size) {
|
||||
if (size < -1) {
|
||||
throw new IllegalArgumentException(
|
||||
"[size] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.requiredSize = size;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return requiredSize;
|
||||
}
|
||||
|
||||
public GeoGridAggregatorBuilder shardSize(int shardSize) {
|
||||
if (shardSize < -1) {
|
||||
throw new IllegalArgumentException(
|
||||
"[shardSize] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
int shardSize = this.shardSize;
|
||||
if (shardSize == 0) {
|
||||
shardSize = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
int requiredSize = this.requiredSize;
|
||||
if (requiredSize == 0) {
|
||||
requiredSize = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
if (shardSize < 0) {
|
||||
// Use default heuristic to avoid any wrong-ranking caused by distributed counting
|
||||
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.searchContext().numberOfShards());
|
||||
}
|
||||
|
||||
if (shardSize < requiredSize) {
|
||||
shardSize = requiredSize;
|
||||
}
|
||||
return new GeoHashGridAggregatorFactory(name, type, config, precision, requiredSize, shardSize, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoGridAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType,
|
||||
StreamInput in) throws IOException {
|
||||
GeoGridAggregatorBuilder factory = new GeoGridAggregatorBuilder(name);
|
||||
factory.precision = in.readVInt();
|
||||
factory.requiredSize = in.readVInt();
|
||||
factory.shardSize = in.readVInt();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(precision);
|
||||
out.writeVInt(requiredSize);
|
||||
out.writeVInt(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(GeoHashGridParams.FIELD_PRECISION.getPreferredName(), precision);
|
||||
builder.field(GeoHashGridParams.FIELD_SIZE.getPreferredName(), requiredSize);
|
||||
builder.field(GeoHashGridParams.FIELD_SHARD_SIZE.getPreferredName(), shardSize);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoGridAggregatorBuilder other = (GeoGridAggregatorBuilder) obj;
|
||||
if (precision != other.precision) {
|
||||
return false;
|
||||
}
|
||||
if (requiredSize != other.requiredSize) {
|
||||
return false;
|
||||
}
|
||||
if (shardSize != other.shardSize) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(precision, requiredSize, shardSize);
|
||||
}
|
||||
|
||||
private static class CellValues extends SortingNumericDocValues {
|
||||
private MultiGeoPointValues geoValues;
|
||||
private int precision;
|
||||
|
||||
protected CellValues(MultiGeoPointValues geoValues, int precision) {
|
||||
this.geoValues = geoValues;
|
||||
this.precision = precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docId) {
|
||||
geoValues.setDocument(docId);
|
||||
resize(geoValues.count());
|
||||
for (int i = 0; i < count(); ++i) {
|
||||
GeoPoint target = geoValues.valueAt(i);
|
||||
values[i] = GeoHashUtils.longEncode(target.getLon(), target.getLat(), precision);
|
||||
}
|
||||
sort();
|
||||
}
|
||||
}
|
||||
|
||||
static class CellIdSource extends ValuesSource.Numeric {
|
||||
private final ValuesSource.GeoPoint valuesSource;
|
||||
private final int precision;
|
||||
|
||||
public CellIdSource(ValuesSource.GeoPoint valuesSource, int precision) {
|
||||
this.valuesSource = valuesSource;
|
||||
//different GeoPoints could map to the same or different geohash cells.
|
||||
this.precision = precision;
|
||||
}
|
||||
|
||||
public int precision() {
|
||||
return precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFloatingPoint() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDocValues longValues(LeafReaderContext ctx) {
|
||||
return new CellValues(valuesSource.geoPointValues(ctx), precision);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,16 +19,10 @@
|
|||
package org.elasticsearch.search.aggregations.bucket.global;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
@ -70,47 +64,4 @@ public class GlobalAggregator extends SingleBucketAggregator {
|
|||
public InternalAggregation buildEmptyAggregation() {
|
||||
throw new UnsupportedOperationException("global aggregations cannot serve as sub-aggregations, hence should never be called on #buildEmptyAggregations");
|
||||
}
|
||||
|
||||
public static class GlobalAggregatorBuilder extends AggregatorBuilder<GlobalAggregatorBuilder> {
|
||||
|
||||
static final GlobalAggregatorBuilder PROTOTYPE = new GlobalAggregatorBuilder("");
|
||||
|
||||
public GlobalAggregatorBuilder(String name) {
|
||||
super(name, InternalGlobal.TYPE);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new GlobalAggregatorFactory(name, type, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GlobalAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
return new GlobalAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
// Nothing to write
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.global;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class GlobalAggregatorBuilder extends AggregatorBuilder<GlobalAggregatorBuilder> {
|
||||
|
||||
static final GlobalAggregatorBuilder PROTOTYPE = new GlobalAggregatorBuilder("");
|
||||
|
||||
public GlobalAggregatorBuilder(String name) {
|
||||
super(name, InternalGlobal.TYPE);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new GlobalAggregatorFactory(name, type, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GlobalAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
return new GlobalAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
// Nothing to write
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
}
|
|
@ -34,15 +34,15 @@ public class GlobalParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public GlobalAggregator.GlobalAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public GlobalAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
parser.nextToken();
|
||||
return new GlobalAggregator.GlobalAggregatorBuilder(aggregationName);
|
||||
return new GlobalAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GlobalAggregator.GlobalAggregatorBuilder getFactoryPrototypes() {
|
||||
return GlobalAggregator.GlobalAggregatorBuilder.PROTOTYPE;
|
||||
public GlobalAggregatorBuilder getFactoryPrototypes() {
|
||||
return GlobalAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,211 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.rounding.Rounding;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class AbstractHistogramBuilder<AB extends AbstractHistogramBuilder<AB>>
|
||||
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
|
||||
|
||||
protected long interval;
|
||||
protected long offset = 0;
|
||||
protected InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
|
||||
protected boolean keyed = false;
|
||||
protected long minDocCount = 0;
|
||||
protected ExtendedBounds extendedBounds;
|
||||
|
||||
AbstractHistogramBuilder(String name, InternalHistogram.Factory<?> histogramFactory) {
|
||||
super(name, histogramFactory.type(), ValuesSourceType.NUMERIC, histogramFactory.valueType());
|
||||
}
|
||||
|
||||
public long interval() {
|
||||
return interval;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB interval(long interval) {
|
||||
if (interval < 1) {
|
||||
throw new IllegalArgumentException("[interval] must be 1 or greater for histogram aggregation [" + name + "]");
|
||||
}
|
||||
this.interval = interval;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public long offset() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB offset(long offset) {
|
||||
this.offset = offset;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public Histogram.Order order() {
|
||||
return order;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB order(Histogram.Order order) {
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
|
||||
}
|
||||
this.order = (InternalOrder) order;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public long minDocCount() {
|
||||
return minDocCount;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB minDocCount(long minDocCount) {
|
||||
if (minDocCount < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[minDocCount] must be greater than or equal to 0. Found [" + minDocCount + "] in [" + name + "]");
|
||||
}
|
||||
this.minDocCount = minDocCount;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public ExtendedBounds extendedBounds() {
|
||||
return extendedBounds;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB extendedBounds(ExtendedBounds extendedBounds) {
|
||||
if (extendedBounds == null) {
|
||||
throw new IllegalArgumentException("[extendedBounds] must not be null: [" + name + "]");
|
||||
}
|
||||
this.extendedBounds = extendedBounds;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
|
||||
builder.field(Rounding.Interval.INTERVAL_FIELD.getPreferredName());
|
||||
doXContentInterval(builder, params);
|
||||
builder.field(Rounding.OffsetRounding.OFFSET_FIELD.getPreferredName(), offset);
|
||||
|
||||
if (order != null) {
|
||||
builder.field(HistogramAggregator.ORDER_FIELD.getPreferredName());
|
||||
order.toXContent(builder, params);
|
||||
}
|
||||
|
||||
builder.field(HistogramAggregator.KEYED_FIELD.getPreferredName(), keyed);
|
||||
|
||||
builder.field(HistogramAggregator.MIN_DOC_COUNT_FIELD.getPreferredName(), minDocCount);
|
||||
|
||||
if (extendedBounds != null) {
|
||||
extendedBounds.toXContent(builder, params);
|
||||
}
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.value(interval);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalHistogram.TYPE.name();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in)
|
||||
throws IOException {
|
||||
AbstractHistogramBuilder<AB> factory = createFactoryFromStream(name, in);
|
||||
factory.interval = in.readVLong();
|
||||
factory.offset = in.readLong();
|
||||
if (in.readBoolean()) {
|
||||
factory.order = InternalOrder.Streams.readOrder(in);
|
||||
}
|
||||
factory.keyed = in.readBoolean();
|
||||
factory.minDocCount = in.readVLong();
|
||||
if (in.readBoolean()) {
|
||||
factory.extendedBounds = ExtendedBounds.readFrom(in);
|
||||
}
|
||||
return (AB) factory;
|
||||
}
|
||||
|
||||
protected abstract AB createFactoryFromStream(String name, StreamInput in) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
writeFactoryToStream(out);
|
||||
out.writeVLong(interval);
|
||||
out.writeLong(offset);
|
||||
boolean hasOrder = order != null;
|
||||
out.writeBoolean(hasOrder);
|
||||
if (hasOrder) {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
out.writeVLong(minDocCount);
|
||||
boolean hasExtendedBounds = extendedBounds != null;
|
||||
out.writeBoolean(hasExtendedBounds);
|
||||
if (hasExtendedBounds) {
|
||||
extendedBounds.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeFactoryToStream(StreamOutput out) throws IOException {
|
||||
// Default impl does nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(interval, offset, order, keyed, minDocCount, extendedBounds);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
AbstractHistogramBuilder<?> other = (AbstractHistogramBuilder<?>) obj;
|
||||
return Objects.equals(interval, other.interval)
|
||||
&& Objects.equals(offset, other.offset)
|
||||
&& Objects.equals(order, other.order)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(minDocCount, other.minDocCount)
|
||||
&& Objects.equals(extendedBounds, other.extendedBounds);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,129 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DateHistogramAggregatorBuilder extends AbstractHistogramBuilder<DateHistogramAggregatorBuilder> {
|
||||
|
||||
public static final DateHistogramAggregatorBuilder PROTOTYPE = new DateHistogramAggregatorBuilder("");
|
||||
|
||||
private DateHistogramInterval dateHistogramInterval;
|
||||
|
||||
public DateHistogramAggregatorBuilder(String name) {
|
||||
super(name, InternalDateHistogram.HISTOGRAM_FACTORY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the interval.
|
||||
*/
|
||||
public DateHistogramAggregatorBuilder dateHistogramInterval(DateHistogramInterval dateHistogramInterval) {
|
||||
if (dateHistogramInterval == null) {
|
||||
throw new IllegalArgumentException("[dateHistogramInterval] must not be null: [" + name + "]");
|
||||
}
|
||||
this.dateHistogramInterval = dateHistogramInterval;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DateHistogramAggregatorBuilder offset(String offset) {
|
||||
if (offset == null) {
|
||||
throw new IllegalArgumentException("[offset] must not be null: [" + name + "]");
|
||||
}
|
||||
return offset(parseStringOffset(offset));
|
||||
}
|
||||
|
||||
protected static long parseStringOffset(String offset) {
|
||||
if (offset.charAt(0) == '-') {
|
||||
return -TimeValue
|
||||
.parseTimeValue(offset.substring(1), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
|
||||
.millis();
|
||||
}
|
||||
int beginIndex = offset.charAt(0) == '+' ? 1 : 0;
|
||||
return TimeValue
|
||||
.parseTimeValue(offset.substring(beginIndex), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
|
||||
.millis();
|
||||
}
|
||||
|
||||
public DateHistogramInterval dateHistogramInterval() {
|
||||
return dateHistogramInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DateHistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
|
||||
extendedBounds, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalDateHistogram.TYPE.name();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException {
|
||||
if (dateHistogramInterval == null) {
|
||||
super.doXContentInterval(builder, params);
|
||||
} else {
|
||||
builder.value(dateHistogramInterval.toString());
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DateHistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
|
||||
DateHistogramAggregatorBuilder factory = new DateHistogramAggregatorBuilder(name);
|
||||
if (in.readBoolean()) {
|
||||
factory.dateHistogramInterval = DateHistogramInterval.readFromStream(in);
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeFactoryToStream(StreamOutput out) throws IOException {
|
||||
boolean hasDateInterval = dateHistogramInterval != null;
|
||||
out.writeBoolean(hasDateInterval);
|
||||
if (hasDateInterval) {
|
||||
dateHistogramInterval.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(super.innerHashCode(), dateHistogramInterval);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
DateHistogramAggregatorBuilder other = (DateHistogramAggregatorBuilder) obj;
|
||||
return super.innerEquals(obj) && Objects.equals(dateHistogramInterval, other.dateHistogramInterval);
|
||||
}
|
||||
}
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
|
|||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.rounding.Rounding;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator.DateHistogramAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
|
@ -50,7 +49,7 @@ public class DateHistogramParser extends HistogramParser {
|
|||
@Override
|
||||
protected DateHistogramAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
HistogramAggregator.DateHistogramAggregatorBuilder factory = new HistogramAggregator.DateHistogramAggregatorBuilder(aggregationName);
|
||||
DateHistogramAggregatorBuilder factory = new DateHistogramAggregatorBuilder(aggregationName);
|
||||
Object interval = otherOptions.get(Rounding.Interval.INTERVAL_FIELD);
|
||||
if (interval == null) {
|
||||
throw new ParsingException(null, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
|
||||
|
@ -100,6 +99,6 @@ public class DateHistogramParser extends HistogramParser {
|
|||
|
||||
@Override
|
||||
public DateHistogramAggregatorBuilder getFactoryPrototypes() {
|
||||
return HistogramAggregator.DateHistogramAggregatorBuilder.PROTOTYPE;
|
||||
return DateHistogramAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,29 +23,18 @@ import org.apache.lucene.index.SortedNumericDocValues;
|
|||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.inject.internal.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.rounding.Rounding;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.LongHash;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -53,7 +42,6 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class HistogramAggregator extends BucketsAggregator {
|
||||
|
||||
|
@ -158,301 +146,4 @@ public class HistogramAggregator extends BucketsAggregator {
|
|||
public void doClose() {
|
||||
Releasables.close(bucketOrds);
|
||||
}
|
||||
|
||||
public static class HistogramAggregatorBuilder extends AbstractBuilder<HistogramAggregatorBuilder> {
|
||||
public static final HistogramAggregatorBuilder PROTOTYPE = new HistogramAggregatorBuilder("");
|
||||
|
||||
public HistogramAggregatorBuilder(String name) {
|
||||
super(name, InternalHistogram.HISTOGRAM_FACTORY);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
|
||||
return new HistogramAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, context,
|
||||
parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static abstract class AbstractBuilder<AB extends AbstractBuilder<AB>>
|
||||
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
|
||||
|
||||
protected long interval;
|
||||
protected long offset = 0;
|
||||
protected InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
|
||||
protected boolean keyed = false;
|
||||
protected long minDocCount = 0;
|
||||
protected ExtendedBounds extendedBounds;
|
||||
|
||||
private AbstractBuilder(String name, InternalHistogram.Factory<?> histogramFactory) {
|
||||
super(name, histogramFactory.type(), ValuesSourceType.NUMERIC, histogramFactory.valueType());
|
||||
}
|
||||
|
||||
public long interval() {
|
||||
return interval;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB interval(long interval) {
|
||||
if (interval < 1) {
|
||||
throw new IllegalArgumentException("[interval] must be 1 or greater for histogram aggregation [" + name + "]");
|
||||
}
|
||||
this.interval = interval;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public long offset() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB offset(long offset) {
|
||||
this.offset = offset;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public Histogram.Order order() {
|
||||
return order;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB order(Histogram.Order order) {
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
|
||||
}
|
||||
this.order = (InternalOrder) order;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public long minDocCount() {
|
||||
return minDocCount;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB minDocCount(long minDocCount) {
|
||||
if (minDocCount < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[minDocCount] must be greater than or equal to 0. Found [" + minDocCount + "] in [" + name + "]");
|
||||
}
|
||||
this.minDocCount = minDocCount;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public ExtendedBounds extendedBounds() {
|
||||
return extendedBounds;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public AB extendedBounds(ExtendedBounds extendedBounds) {
|
||||
if (extendedBounds == null) {
|
||||
throw new IllegalArgumentException("[extendedBounds] must not be null: [" + name + "]");
|
||||
}
|
||||
this.extendedBounds = extendedBounds;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
|
||||
builder.field(Rounding.Interval.INTERVAL_FIELD.getPreferredName());
|
||||
doXContentInterval(builder, params);
|
||||
builder.field(Rounding.OffsetRounding.OFFSET_FIELD.getPreferredName(), offset);
|
||||
|
||||
if (order != null) {
|
||||
builder.field(ORDER_FIELD.getPreferredName());
|
||||
order.toXContent(builder, params);
|
||||
}
|
||||
|
||||
builder.field(KEYED_FIELD.getPreferredName(), keyed);
|
||||
|
||||
builder.field(MIN_DOC_COUNT_FIELD.getPreferredName(), minDocCount);
|
||||
|
||||
if (extendedBounds != null) {
|
||||
extendedBounds.toXContent(builder, params);
|
||||
}
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.value(interval);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalHistogram.TYPE.name();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in)
|
||||
throws IOException {
|
||||
AbstractBuilder<AB> factory = createFactoryFromStream(name, in);
|
||||
factory.interval = in.readVLong();
|
||||
factory.offset = in.readLong();
|
||||
if (in.readBoolean()) {
|
||||
factory.order = InternalOrder.Streams.readOrder(in);
|
||||
}
|
||||
factory.keyed = in.readBoolean();
|
||||
factory.minDocCount = in.readVLong();
|
||||
if (in.readBoolean()) {
|
||||
factory.extendedBounds = ExtendedBounds.readFrom(in);
|
||||
}
|
||||
return (AB) factory;
|
||||
}
|
||||
|
||||
protected abstract AB createFactoryFromStream(String name, StreamInput in) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
writeFactoryToStream(out);
|
||||
out.writeVLong(interval);
|
||||
out.writeLong(offset);
|
||||
boolean hasOrder = order != null;
|
||||
out.writeBoolean(hasOrder);
|
||||
if (hasOrder) {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
out.writeVLong(minDocCount);
|
||||
boolean hasExtendedBounds = extendedBounds != null;
|
||||
out.writeBoolean(hasExtendedBounds);
|
||||
if (hasExtendedBounds) {
|
||||
extendedBounds.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeFactoryToStream(StreamOutput out) throws IOException {
|
||||
// Default impl does nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(interval, offset, order, keyed, minDocCount, extendedBounds);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
AbstractBuilder<?> other = (AbstractBuilder<?>) obj;
|
||||
return Objects.equals(interval, other.interval)
|
||||
&& Objects.equals(offset, other.offset)
|
||||
&& Objects.equals(order, other.order)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(minDocCount, other.minDocCount)
|
||||
&& Objects.equals(extendedBounds, other.extendedBounds);
|
||||
}
|
||||
}
|
||||
|
||||
public static class DateHistogramAggregatorBuilder extends AbstractBuilder<DateHistogramAggregatorBuilder> {
|
||||
|
||||
public static final DateHistogramAggregatorBuilder PROTOTYPE = new DateHistogramAggregatorBuilder("");
|
||||
|
||||
private DateHistogramInterval dateHistogramInterval;
|
||||
|
||||
public DateHistogramAggregatorBuilder(String name) {
|
||||
super(name, InternalDateHistogram.HISTOGRAM_FACTORY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the interval.
|
||||
*/
|
||||
public DateHistogramAggregatorBuilder dateHistogramInterval(DateHistogramInterval dateHistogramInterval) {
|
||||
if (dateHistogramInterval == null) {
|
||||
throw new IllegalArgumentException("[dateHistogramInterval] must not be null: [" + name + "]");
|
||||
}
|
||||
this.dateHistogramInterval = dateHistogramInterval;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DateHistogramAggregatorBuilder offset(String offset) {
|
||||
if (offset == null) {
|
||||
throw new IllegalArgumentException("[offset] must not be null: [" + name + "]");
|
||||
}
|
||||
return offset(parseStringOffset(offset));
|
||||
}
|
||||
|
||||
protected static long parseStringOffset(String offset) {
|
||||
if (offset.charAt(0) == '-') {
|
||||
return -TimeValue.parseTimeValue(offset.substring(1), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
|
||||
.millis();
|
||||
}
|
||||
int beginIndex = offset.charAt(0) == '+' ? 1 : 0;
|
||||
return TimeValue.parseTimeValue(offset.substring(beginIndex), null, DateHistogramAggregatorBuilder.class.getSimpleName() + ".parseOffset")
|
||||
.millis();
|
||||
}
|
||||
|
||||
public DateHistogramInterval dateHistogramInterval() {
|
||||
return dateHistogramInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DateHistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed,
|
||||
minDocCount, extendedBounds, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalDateHistogram.TYPE.name();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException {
|
||||
if (dateHistogramInterval == null) {
|
||||
super.doXContentInterval(builder, params);
|
||||
} else {
|
||||
builder.value(dateHistogramInterval.toString());
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DateHistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in)
|
||||
throws IOException {
|
||||
DateHistogramAggregatorBuilder factory = new DateHistogramAggregatorBuilder(name);
|
||||
if (in.readBoolean()) {
|
||||
factory.dateHistogramInterval = DateHistogramInterval.readFromStream(in);
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeFactoryToStream(StreamOutput out) throws IOException {
|
||||
boolean hasDateInterval = dateHistogramInterval != null;
|
||||
out.writeBoolean(hasDateInterval);
|
||||
if (hasDateInterval) {
|
||||
dateHistogramInterval.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(super.innerHashCode(), dateHistogramInterval);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
DateHistogramAggregatorBuilder other = (DateHistogramAggregatorBuilder) obj;
|
||||
return super.innerEquals(obj)
|
||||
&& Objects.equals(dateHistogramInterval, other.dateHistogramInterval);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HistogramAggregatorBuilder extends AbstractHistogramBuilder<HistogramAggregatorBuilder> {
|
||||
public static final HistogramAggregatorBuilder PROTOTYPE = new HistogramAggregatorBuilder("");
|
||||
|
||||
public HistogramAggregatorBuilder(String name) {
|
||||
super(name, InternalHistogram.HISTOGRAM_FACTORY);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HistogramAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
|
||||
return new HistogramAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, context,
|
||||
parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
}
|
|
@ -50,9 +50,9 @@ public class HistogramParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected HistogramAggregator.AbstractBuilder<?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected AbstractHistogramBuilder<?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
HistogramAggregator.HistogramAggregatorBuilder factory = new HistogramAggregator.HistogramAggregatorBuilder(aggregationName);
|
||||
HistogramAggregatorBuilder factory = new HistogramAggregatorBuilder(aggregationName);
|
||||
Long interval = (Long) otherOptions.get(Rounding.Interval.INTERVAL_FIELD);
|
||||
if (interval == null) {
|
||||
throw new ParsingException(null, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
|
||||
|
@ -162,7 +162,7 @@ public class HistogramParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public HistogramAggregator.AbstractBuilder<?> getFactoryPrototypes() {
|
||||
return HistogramAggregator.HistogramAggregatorBuilder.PROTOTYPE;
|
||||
public AbstractHistogramBuilder<?> getFactoryPrototypes() {
|
||||
return HistogramAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,25 +20,15 @@ package org.elasticsearch.search.aggregations.bucket.missing;
|
|||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -89,47 +79,6 @@ public class MissingAggregator extends SingleBucketAggregator {
|
|||
return new InternalMissing(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class MissingAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, MissingAggregatorBuilder> {
|
||||
|
||||
static final MissingAggregatorBuilder PROTOTYPE = new MissingAggregatorBuilder("", null);
|
||||
|
||||
public MissingAggregatorBuilder(String name, ValueType targetValueType) {
|
||||
super(name, InternalMissing.TYPE, ValuesSourceType.ANY, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MissingAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MissingAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MissingAggregatorBuilder(name, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.missing;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class MissingAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, MissingAggregatorBuilder> {
|
||||
|
||||
static final MissingAggregatorBuilder PROTOTYPE = new MissingAggregatorBuilder("", null);
|
||||
|
||||
public MissingAggregatorBuilder(String name, ValueType targetValueType) {
|
||||
super(name, InternalMissing.TYPE, ValuesSourceType.ANY, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MissingAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MissingAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MissingAggregatorBuilder(name, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -46,13 +46,13 @@ public class MissingParser extends AnyValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected MissingAggregator.MissingAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected MissingAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new MissingAggregator.MissingAggregatorBuilder(aggregationName, targetValueType);
|
||||
return new MissingAggregatorBuilder(aggregationName, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MissingAggregator.MissingAggregatorBuilder getFactoryPrototypes() {
|
||||
return MissingAggregator.MissingAggregatorBuilder.PROTOTYPE;
|
||||
public MissingAggregatorBuilder getFactoryPrototypes() {
|
||||
return MissingAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,19 +29,13 @@ import org.apache.lucene.search.Weight;
|
|||
import org.apache.lucene.search.join.BitSetProducer;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
@ -49,7 +43,6 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -150,70 +143,4 @@ public class NestedAggregator extends SingleBucketAggregator {
|
|||
return null;
|
||||
}
|
||||
|
||||
public static class NestedAggregatorBuilder extends AggregatorBuilder<NestedAggregatorBuilder> {
|
||||
|
||||
static final NestedAggregatorBuilder PROTOTYPE = new NestedAggregatorBuilder("", "");
|
||||
|
||||
private final String path;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param path
|
||||
* the path to use for this nested aggregation. The path must
|
||||
* match the path to a nested object in the mappings.
|
||||
*/
|
||||
public NestedAggregatorBuilder(String name, String path) {
|
||||
super(name, InternalNested.TYPE);
|
||||
if (path == null) {
|
||||
throw new IllegalArgumentException("[path] must not be null: [" + name + "]");
|
||||
}
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the path to use for this nested aggregation.
|
||||
*/
|
||||
public String path() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new NestedAggregatorFactory(name, type, path, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(PATH_FIELD.getPreferredName(), path);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected NestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
String path = in.readString();
|
||||
NestedAggregatorBuilder factory = new NestedAggregatorBuilder(name, path);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeString(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
NestedAggregatorBuilder other = (NestedAggregatorBuilder) obj;
|
||||
return Objects.equals(path, other.path);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.nested;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class NestedAggregatorBuilder extends AggregatorBuilder<NestedAggregatorBuilder> {
|
||||
|
||||
static final NestedAggregatorBuilder PROTOTYPE = new NestedAggregatorBuilder("", "");
|
||||
|
||||
private final String path;
|
||||
|
||||
/**
|
||||
* @param name
|
||||
* the name of this aggregation
|
||||
* @param path
|
||||
* the path to use for this nested aggregation. The path must
|
||||
* match the path to a nested object in the mappings.
|
||||
*/
|
||||
public NestedAggregatorBuilder(String name, String path) {
|
||||
super(name, InternalNested.TYPE);
|
||||
if (path == null) {
|
||||
throw new IllegalArgumentException("[path] must not be null: [" + name + "]");
|
||||
}
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the path to use for this nested aggregation.
|
||||
*/
|
||||
public String path() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new NestedAggregatorFactory(name, type, path, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(NestedAggregator.PATH_FIELD.getPreferredName(), path);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected NestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
String path = in.readString();
|
||||
NestedAggregatorBuilder factory = new NestedAggregatorBuilder(name, path);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeString(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
NestedAggregatorBuilder other = (NestedAggregatorBuilder) obj;
|
||||
return Objects.equals(path, other.path);
|
||||
}
|
||||
}
|
|
@ -35,7 +35,7 @@ public class NestedParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NestedAggregator.NestedAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public NestedAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
String path = null;
|
||||
|
||||
|
@ -61,11 +61,11 @@ public class NestedParser implements Aggregator.Parser {
|
|||
throw new ParsingException(parser.getTokenLocation(), "Missing [path] field for nested aggregation [" + aggregationName + "]");
|
||||
}
|
||||
|
||||
return new NestedAggregator.NestedAggregatorBuilder(aggregationName, path);
|
||||
return new NestedAggregatorBuilder(aggregationName, path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NestedAggregator.NestedAggregatorBuilder getFactoryPrototypes() {
|
||||
return NestedAggregator.NestedAggregatorBuilder.PROTOTYPE;
|
||||
public NestedAggregatorBuilder getFactoryPrototypes() {
|
||||
return NestedAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,19 +25,13 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.join.BitSetProducer;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
@ -45,7 +39,6 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -114,74 +107,4 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
|
|||
Query getParentFilter() {
|
||||
return parentFilter;
|
||||
}
|
||||
|
||||
public static class ReverseNestedAggregatorBuilder extends AggregatorBuilder<ReverseNestedAggregatorBuilder> {
|
||||
|
||||
static final ReverseNestedAggregatorBuilder PROTOTYPE = new ReverseNestedAggregatorBuilder("");
|
||||
|
||||
private String path;
|
||||
|
||||
public ReverseNestedAggregatorBuilder(String name) {
|
||||
super(name, InternalReverseNested.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the path to use for this nested aggregation. The path must match
|
||||
* the path to a nested object in the mappings. If it is not specified
|
||||
* then this aggregation will go back to the root document.
|
||||
*/
|
||||
public ReverseNestedAggregatorBuilder path(String path) {
|
||||
if (path == null) {
|
||||
throw new IllegalArgumentException("[path] must not be null: [" + name + "]");
|
||||
}
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the path to use for this nested aggregation.
|
||||
*/
|
||||
public String path() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new ReverseNestedAggregatorFactory(name, type, path, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (path != null) {
|
||||
builder.field(PATH_FIELD.getPreferredName(), path);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ReverseNestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregatorBuilder(name);
|
||||
factory.path = in.readOptionalString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalString(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
ReverseNestedAggregatorBuilder other = (ReverseNestedAggregatorBuilder) obj;
|
||||
return Objects.equals(path, other.path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.nested;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ReverseNestedAggregatorBuilder extends AggregatorBuilder<ReverseNestedAggregatorBuilder> {
|
||||
|
||||
static final ReverseNestedAggregatorBuilder PROTOTYPE = new ReverseNestedAggregatorBuilder("");
|
||||
|
||||
private String path;
|
||||
|
||||
public ReverseNestedAggregatorBuilder(String name) {
|
||||
super(name, InternalReverseNested.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the path to use for this nested aggregation. The path must match
|
||||
* the path to a nested object in the mappings. If it is not specified
|
||||
* then this aggregation will go back to the root document.
|
||||
*/
|
||||
public ReverseNestedAggregatorBuilder path(String path) {
|
||||
if (path == null) {
|
||||
throw new IllegalArgumentException("[path] must not be null: [" + name + "]");
|
||||
}
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the path to use for this nested aggregation.
|
||||
*/
|
||||
public String path() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new ReverseNestedAggregatorFactory(name, type, path, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (path != null) {
|
||||
builder.field(ReverseNestedAggregator.PATH_FIELD.getPreferredName(), path);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ReverseNestedAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregatorBuilder(name);
|
||||
factory.path = in.readOptionalString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalString(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
ReverseNestedAggregatorBuilder other = (ReverseNestedAggregatorBuilder) obj;
|
||||
return Objects.equals(path, other.path);
|
||||
}
|
||||
}
|
|
@ -35,7 +35,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ReverseNestedAggregator.ReverseNestedAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
public ReverseNestedAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
QueryParseContext context) throws IOException {
|
||||
String path = null;
|
||||
|
||||
|
@ -56,7 +56,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
ReverseNestedAggregator.ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregator.ReverseNestedAggregatorBuilder(
|
||||
ReverseNestedAggregatorBuilder factory = new ReverseNestedAggregatorBuilder(
|
||||
aggregationName);
|
||||
if (path != null) {
|
||||
factory.path(path);
|
||||
|
@ -65,7 +65,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ReverseNestedAggregator.ReverseNestedAggregatorBuilder getFactoryPrototypes() {
|
||||
return ReverseNestedAggregator.ReverseNestedAggregatorBuilder.PROTOTYPE;
|
||||
public ReverseNestedAggregatorBuilder getFactoryPrototypes() {
|
||||
return ReverseNestedAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,105 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.range;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class AbstractRangeBuilder<AB extends AbstractRangeBuilder<AB, R>, R extends Range>
|
||||
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
|
||||
|
||||
protected final InternalRange.Factory<?, ?> rangeFactory;
|
||||
protected List<R> ranges = new ArrayList<>();
|
||||
protected boolean keyed = false;
|
||||
|
||||
protected AbstractRangeBuilder(String name, InternalRange.Factory<?, ?> rangeFactory) {
|
||||
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
|
||||
this.rangeFactory = rangeFactory;
|
||||
}
|
||||
|
||||
public AB addRange(R range) {
|
||||
if (range == null) {
|
||||
throw new IllegalArgumentException("[range] must not be null: [" + name + "]");
|
||||
}
|
||||
ranges.add(range);
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public List<R> ranges() {
|
||||
return ranges;
|
||||
}
|
||||
|
||||
public AB keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(RangeAggregator.RANGES_FIELD.getPreferredName(), ranges);
|
||||
builder.field(RangeAggregator.KEYED_FIELD.getPreferredName(), keyed);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
AbstractRangeBuilder<AB, R> factory = createFactoryFromStream(name, in);
|
||||
factory.keyed = in.readBoolean();
|
||||
return (AB) factory;
|
||||
}
|
||||
|
||||
protected abstract AbstractRangeBuilder<AB, R> createFactoryFromStream(String name, StreamInput in) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(ranges.size());
|
||||
for (Range range : ranges) {
|
||||
range.writeTo(out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(ranges, keyed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
AbstractRangeBuilder<AB, R> other = (AbstractRangeBuilder<AB, R>) obj;
|
||||
return Objects.equals(ranges, other.ranges)
|
||||
&& Objects.equals(keyed, other.keyed);
|
||||
}
|
||||
}
|
|
@ -31,22 +31,15 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueParser;
|
||||
|
@ -400,168 +393,4 @@ public class RangeAggregator extends BucketsAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
public static abstract class AbstractBuilder<AB extends AbstractBuilder<AB, R>, R extends Range>
|
||||
extends ValuesSourceAggregatorBuilder<ValuesSource.Numeric, AB> {
|
||||
|
||||
protected final InternalRange.Factory<?, ?> rangeFactory;
|
||||
protected List<R> ranges = new ArrayList<>();
|
||||
protected boolean keyed = false;
|
||||
|
||||
protected AbstractBuilder(String name, InternalRange.Factory<?, ?> rangeFactory) {
|
||||
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
|
||||
this.rangeFactory = rangeFactory;
|
||||
}
|
||||
|
||||
public AB addRange(R range) {
|
||||
if (range == null) {
|
||||
throw new IllegalArgumentException("[range] must not be null: [" + name + "]");
|
||||
}
|
||||
ranges.add(range);
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public List<R> ranges() {
|
||||
return ranges;
|
||||
}
|
||||
|
||||
public AB keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return (AB) this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(RANGES_FIELD.getPreferredName(), ranges);
|
||||
builder.field(KEYED_FIELD.getPreferredName(), keyed);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AB innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
AbstractBuilder<AB, R> factory = createFactoryFromStream(name, in);
|
||||
factory.keyed = in.readBoolean();
|
||||
return (AB) factory;
|
||||
}
|
||||
|
||||
protected abstract AbstractBuilder<AB, R> createFactoryFromStream(String name, StreamInput in) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(ranges.size());
|
||||
for (Range range : ranges) {
|
||||
range.writeTo(out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(ranges, keyed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
AbstractBuilder<AB, R> other = (AbstractBuilder<AB, R>) obj;
|
||||
return Objects.equals(ranges, other.ranges)
|
||||
&& Objects.equals(keyed, other.keyed);
|
||||
}
|
||||
}
|
||||
|
||||
public static class RangeAggregatorBuilder extends AbstractBuilder<RangeAggregatorBuilder, Range> {
|
||||
|
||||
static final RangeAggregatorBuilder PROTOTYPE = new RangeAggregatorBuilder("");
|
||||
|
||||
public RangeAggregatorBuilder(String name) {
|
||||
super(name, InternalRange.FACTORY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range to this aggregation.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addRange(String key, double from, double to) {
|
||||
addRange(new Range(key, from, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addRange(String, double, double)} but the key will be
|
||||
* automatically generated based on <code>from</code> and
|
||||
* <code>to</code>.
|
||||
*/
|
||||
public RangeAggregatorBuilder addRange(double from, double to) {
|
||||
return addRange(null, from, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no lower bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedTo(String key, double to) {
|
||||
addRange(new Range(key, null, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedTo(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedTo(double to) {
|
||||
return addUnboundedTo(null, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no upper bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedFrom(String key, double from) {
|
||||
addRange(new Range(key, from, null));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedFrom(double from) {
|
||||
return addUnboundedFrom(null, from);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new RangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
|
||||
metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RangeAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
|
||||
int size = in.readVInt();
|
||||
RangeAggregatorBuilder factory = new RangeAggregatorBuilder(name);
|
||||
for (int i = 0; i < size; i++) {
|
||||
factory.addRange(Range.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.range;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class RangeAggregatorBuilder extends AbstractRangeBuilder<RangeAggregatorBuilder, Range> {
|
||||
|
||||
static final RangeAggregatorBuilder PROTOTYPE = new RangeAggregatorBuilder("");
|
||||
|
||||
public RangeAggregatorBuilder(String name) {
|
||||
super(name, InternalRange.FACTORY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range to this aggregation.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addRange(String key, double from, double to) {
|
||||
addRange(new Range(key, from, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addRange(String, double, double)} but the key will be
|
||||
* automatically generated based on <code>from</code> and
|
||||
* <code>to</code>.
|
||||
*/
|
||||
public RangeAggregatorBuilder addRange(double from, double to) {
|
||||
return addRange(null, from, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no lower bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedTo(String key, double to) {
|
||||
addRange(new Range(key, null, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedTo(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedTo(double to) {
|
||||
return addUnboundedTo(null, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no upper bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedFrom(String key, double from) {
|
||||
addRange(new Range(key, from, null));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public RangeAggregatorBuilder addUnboundedFrom(double from) {
|
||||
return addUnboundedFrom(null, from);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new RangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
|
||||
metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RangeAggregatorBuilder createFactoryFromStream(String name, StreamInput in) throws IOException {
|
||||
int size = in.readVInt();
|
||||
RangeAggregatorBuilder factory = new RangeAggregatorBuilder(name);
|
||||
for (int i = 0; i < size; i++) {
|
||||
factory.addRange(Range.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
}
|
|
@ -51,9 +51,9 @@ public class RangeParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected RangeAggregator.AbstractBuilder<?, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected AbstractRangeBuilder<?, ?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
RangeAggregator.RangeAggregatorBuilder factory = new RangeAggregator.RangeAggregatorBuilder(aggregationName);
|
||||
RangeAggregatorBuilder factory = new RangeAggregatorBuilder(aggregationName);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<? extends Range> ranges = (List<? extends Range>) otherOptions.get(RangeAggregator.RANGES_FIELD);
|
||||
for (Range range : ranges) {
|
||||
|
@ -94,7 +94,7 @@ public class RangeParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public RangeAggregator.AbstractBuilder<?, ?> getFactoryPrototypes() {
|
||||
return RangeAggregator.RangeAggregatorBuilder.PROTOTYPE;
|
||||
public AbstractRangeBuilder<?, ?> getFactoryPrototypes() {
|
||||
return RangeAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,8 +22,8 @@ package org.elasticsearch.search.aggregations.bucket.range.date;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -32,7 +32,7 @@ import org.joda.time.DateTime;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
public class DateRangeAggregatorBuilder extends AbstractBuilder<DateRangeAggregatorBuilder, RangeAggregator.Range> {
|
||||
public class DateRangeAggregatorBuilder extends AbstractRangeBuilder<DateRangeAggregatorBuilder, RangeAggregator.Range> {
|
||||
|
||||
static final DateRangeAggregatorBuilder PROTOTYPE = new DateRangeAggregatorBuilder("");
|
||||
|
||||
|
|
|
@ -0,0 +1,246 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.range.geodistance;
|
||||
|
||||
import org.elasticsearch.common.geo.GeoDistance;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.DistanceUnit;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser.Range;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class GeoDistanceAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoDistanceAggregatorBuilder> {
|
||||
|
||||
static final GeoDistanceAggregatorBuilder PROTOTYPE = new GeoDistanceAggregatorBuilder("", new GeoPoint());
|
||||
|
||||
private final GeoPoint origin;
|
||||
private List<Range> ranges = new ArrayList<>();
|
||||
private DistanceUnit unit = DistanceUnit.DEFAULT;
|
||||
private GeoDistance distanceType = GeoDistance.DEFAULT;
|
||||
private boolean keyed = false;
|
||||
|
||||
public GeoDistanceAggregatorBuilder(String name, GeoPoint origin) {
|
||||
this(name, origin, InternalGeoDistance.FACTORY);
|
||||
}
|
||||
|
||||
private GeoDistanceAggregatorBuilder(String name, GeoPoint origin,
|
||||
InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> rangeFactory) {
|
||||
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
|
||||
if (origin == null) {
|
||||
throw new IllegalArgumentException("[origin] must not be null: [" + name + "]");
|
||||
}
|
||||
this.origin = origin;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder addRange(Range range) {
|
||||
if (range == null) {
|
||||
throw new IllegalArgumentException("[range] must not be null: [" + name + "]");
|
||||
}
|
||||
ranges.add(range);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range to this aggregation.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addRange(String key, double from, double to) {
|
||||
ranges.add(new Range(key, from, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addRange(String, double, double)} but the key will be
|
||||
* automatically generated based on <code>from</code> and
|
||||
* <code>to</code>.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addRange(double from, double to) {
|
||||
return addRange(null, from, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no lower bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedTo(String key, double to) {
|
||||
ranges.add(new Range(key, null, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedTo(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedTo(double to) {
|
||||
return addUnboundedTo(null, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no upper bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedFrom(String key, double from) {
|
||||
addRange(new Range(key, from, null));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedFrom(double from) {
|
||||
return addUnboundedFrom(null, from);
|
||||
}
|
||||
|
||||
public List<Range> range() {
|
||||
return ranges;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalGeoDistance.TYPE.name();
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder unit(DistanceUnit unit) {
|
||||
if (unit == null) {
|
||||
throw new IllegalArgumentException("[unit] must not be null: [" + name + "]");
|
||||
}
|
||||
this.unit = unit;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DistanceUnit unit() {
|
||||
return unit;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder distanceType(GeoDistance distanceType) {
|
||||
if (distanceType == null) {
|
||||
throw new IllegalArgumentException("[distanceType] must not be null: [" + name + "]");
|
||||
}
|
||||
this.distanceType = distanceType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public GeoDistance distanceType() {
|
||||
return distanceType;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new GeoDistanceRangeAggregatorFactory(name, type, config, origin, ranges, unit, distanceType, keyed, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(GeoDistanceParser.ORIGIN_FIELD.getPreferredName(), origin);
|
||||
builder.field(RangeAggregator.RANGES_FIELD.getPreferredName(), ranges);
|
||||
builder.field(RangeAggregator.KEYED_FIELD.getPreferredName(), keyed);
|
||||
builder.field(GeoDistanceParser.UNIT_FIELD.getPreferredName(), unit);
|
||||
builder.field(GeoDistanceParser.DISTANCE_TYPE_FIELD.getPreferredName(), distanceType);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoDistanceAggregatorBuilder innerReadFrom(
|
||||
String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in) throws IOException {
|
||||
GeoPoint origin = new GeoPoint(in.readDouble(), in.readDouble());
|
||||
int size = in.readVInt();
|
||||
GeoDistanceAggregatorBuilder factory = new GeoDistanceAggregatorBuilder(name, origin);
|
||||
for (int i = 0; i < size; i++) {
|
||||
factory.addRange(Range.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory.keyed = in.readBoolean();
|
||||
factory.distanceType = GeoDistance.readGeoDistanceFrom(in);
|
||||
factory.unit = DistanceUnit.readDistanceUnit(in);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(origin.lat());
|
||||
out.writeDouble(origin.lon());
|
||||
out.writeVInt(ranges.size());
|
||||
for (Range range : ranges) {
|
||||
range.writeTo(out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
distanceType.writeTo(out);
|
||||
DistanceUnit.writeDistanceUnit(out, unit);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(origin, ranges, keyed, distanceType, unit);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoDistanceAggregatorBuilder other = (GeoDistanceAggregatorBuilder) obj;
|
||||
return Objects.equals(origin, other.origin)
|
||||
&& Objects.equals(ranges, other.ranges)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(distanceType, other.distanceType)
|
||||
&& Objects.equals(unit, other.unit);
|
||||
}
|
||||
|
||||
}
|
|
@ -25,37 +25,27 @@ import org.elasticsearch.common.geo.GeoPoint;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.DistanceUnit;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser.Token;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.GeoPointParser;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class GeoDistanceParser extends GeoPointValuesSourceParser {
|
||||
|
||||
private static final ParseField ORIGIN_FIELD = new ParseField("origin", "center", "point", "por");
|
||||
private static final ParseField UNIT_FIELD = new ParseField("unit");
|
||||
private static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
|
||||
static final ParseField ORIGIN_FIELD = new ParseField("origin", "center", "point", "por");
|
||||
static final ParseField UNIT_FIELD = new ParseField("unit");
|
||||
static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
|
||||
|
||||
private GeoPointParser geoPointParser = new GeoPointParser(InternalGeoDistance.TYPE, ORIGIN_FIELD);
|
||||
|
||||
|
@ -192,208 +182,6 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
|
|||
return false;
|
||||
}
|
||||
|
||||
public static class GeoDistanceAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoDistanceAggregatorBuilder> {
|
||||
|
||||
static final GeoDistanceAggregatorBuilder PROTOTYPE = new GeoDistanceAggregatorBuilder("", new GeoPoint());
|
||||
|
||||
private final GeoPoint origin;
|
||||
private List<Range> ranges = new ArrayList<>();
|
||||
private DistanceUnit unit = DistanceUnit.DEFAULT;
|
||||
private GeoDistance distanceType = GeoDistance.DEFAULT;
|
||||
private boolean keyed = false;
|
||||
|
||||
public GeoDistanceAggregatorBuilder(String name, GeoPoint origin) {
|
||||
this(name, origin, InternalGeoDistance.FACTORY);
|
||||
}
|
||||
|
||||
private GeoDistanceAggregatorBuilder(String name, GeoPoint origin,
|
||||
InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> rangeFactory) {
|
||||
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
|
||||
if (origin == null) {
|
||||
throw new IllegalArgumentException("[origin] must not be null: [" + name + "]");
|
||||
}
|
||||
this.origin = origin;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder addRange(Range range) {
|
||||
if (range == null) {
|
||||
throw new IllegalArgumentException("[range] must not be null: [" + name + "]");
|
||||
}
|
||||
ranges.add(range);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range to this aggregation.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addRange(String key, double from, double to) {
|
||||
ranges.add(new Range(key, from, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addRange(String, double, double)} but the key will be
|
||||
* automatically generated based on <code>from</code> and
|
||||
* <code>to</code>.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addRange(double from, double to) {
|
||||
return addRange(null, from, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no lower bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param to
|
||||
* the upper bound on the distances, exclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedTo(String key, double to) {
|
||||
ranges.add(new Range(key, null, to));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedTo(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedTo(double to) {
|
||||
return addUnboundedTo(null, to);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new range with no upper bound.
|
||||
*
|
||||
* @param key
|
||||
* the key to use for this range in the response
|
||||
* @param from
|
||||
* the lower bound on the distances, inclusive
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedFrom(String key, double from) {
|
||||
addRange(new Range(key, from, null));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #addUnboundedFrom(String, double)} but the key will be
|
||||
* computed automatically.
|
||||
*/
|
||||
public GeoDistanceAggregatorBuilder addUnboundedFrom(double from) {
|
||||
return addUnboundedFrom(null, from);
|
||||
}
|
||||
|
||||
public List<Range> range() {
|
||||
return ranges;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return InternalGeoDistance.TYPE.name();
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder unit(DistanceUnit unit) {
|
||||
if (unit == null) {
|
||||
throw new IllegalArgumentException("[unit] must not be null: [" + name + "]");
|
||||
}
|
||||
this.unit = unit;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DistanceUnit unit() {
|
||||
return unit;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder distanceType(GeoDistance distanceType) {
|
||||
if (distanceType == null) {
|
||||
throw new IllegalArgumentException("[distanceType] must not be null: [" + name + "]");
|
||||
}
|
||||
this.distanceType = distanceType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public GeoDistance distanceType() {
|
||||
return distanceType;
|
||||
}
|
||||
|
||||
public GeoDistanceAggregatorBuilder keyed(boolean keyed) {
|
||||
this.keyed = keyed;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean keyed() {
|
||||
return keyed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new GeoDistanceRangeAggregatorFactory(name, type, config, origin, ranges, unit, distanceType, keyed, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ORIGIN_FIELD.getPreferredName(), origin);
|
||||
builder.field(RangeAggregator.RANGES_FIELD.getPreferredName(), ranges);
|
||||
builder.field(RangeAggregator.KEYED_FIELD.getPreferredName(), keyed);
|
||||
builder.field(UNIT_FIELD.getPreferredName(), unit);
|
||||
builder.field(DISTANCE_TYPE_FIELD.getPreferredName(), distanceType);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoDistanceAggregatorBuilder innerReadFrom(
|
||||
String name, ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in) throws IOException {
|
||||
GeoPoint origin = new GeoPoint(in.readDouble(), in.readDouble());
|
||||
int size = in.readVInt();
|
||||
GeoDistanceAggregatorBuilder factory = new GeoDistanceAggregatorBuilder(name, origin);
|
||||
for (int i = 0; i < size; i++) {
|
||||
factory.addRange(Range.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory.keyed = in.readBoolean();
|
||||
factory.distanceType = GeoDistance.readGeoDistanceFrom(in);
|
||||
factory.unit = DistanceUnit.readDistanceUnit(in);
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(origin.lat());
|
||||
out.writeDouble(origin.lon());
|
||||
out.writeVInt(ranges.size());
|
||||
for (Range range : ranges) {
|
||||
range.writeTo(out);
|
||||
}
|
||||
out.writeBoolean(keyed);
|
||||
distanceType.writeTo(out);
|
||||
DistanceUnit.writeDistanceUnit(out, unit);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(origin, ranges, keyed, distanceType, unit);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoDistanceAggregatorBuilder other = (GeoDistanceAggregatorBuilder) obj;
|
||||
return Objects.equals(origin, other.origin)
|
||||
&& Objects.equals(ranges, other.ranges)
|
||||
&& Objects.equals(keyed, other.keyed)
|
||||
&& Objects.equals(distanceType, other.distanceType)
|
||||
&& Objects.equals(unit, other.unit);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public GeoDistanceAggregatorBuilder getFactoryPrototypes() {
|
||||
return GeoDistanceAggregatorBuilder.PROTOTYPE;
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.AbstractBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
@ -37,7 +37,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
|||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class IPv4RangeAggregatorBuilder extends AbstractBuilder<IPv4RangeAggregatorBuilder, IPv4RangeAggregatorBuilder.Range> {
|
||||
public class IPv4RangeAggregatorBuilder extends AbstractRangeBuilder<IPv4RangeAggregatorBuilder, IPv4RangeAggregatorBuilder.Range> {
|
||||
|
||||
static final IPv4RangeAggregatorBuilder PROTOTYPE = new IPv4RangeAggregatorBuilder("");
|
||||
|
||||
|
|
|
@ -0,0 +1,154 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DiversifiedAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, DiversifiedAggregatorBuilder> {
|
||||
|
||||
public static final Type TYPE = new Type("diversified_sampler");
|
||||
|
||||
static final DiversifiedAggregatorBuilder PROTOTYPE = new DiversifiedAggregatorBuilder("");
|
||||
|
||||
public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
|
||||
|
||||
private int shardSize = SamplerAggregatorBuilder.DEFAULT_SHARD_SAMPLE_SIZE;
|
||||
private int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT;
|
||||
private String executionHint = null;
|
||||
|
||||
public DiversifiedAggregatorBuilder(String name) {
|
||||
super(name, TYPE, ValuesSourceType.ANY, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned from each shard.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder shardSize(int shardSize) {
|
||||
if (shardSize < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[shardSize] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned from each shard.
|
||||
*/
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned per value.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder maxDocsPerValue(int maxDocsPerValue) {
|
||||
if (maxDocsPerValue < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[maxDocsPerValue] must be greater than or equal to 0. Found [" + maxDocsPerValue + "] in [" + name + "]");
|
||||
}
|
||||
this.maxDocsPerValue = maxDocsPerValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned per value.
|
||||
*/
|
||||
public int maxDocsPerValue() {
|
||||
return maxDocsPerValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the execution hint.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder executionHint(String executionHint) {
|
||||
this.executionHint = executionHint;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the execution hint.
|
||||
*/
|
||||
public String executionHint() {
|
||||
return executionHint;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new DiversifiedAggregatorFactory(name, TYPE, config, shardSize, maxDocsPerValue, executionHint, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(SamplerAggregator.SHARD_SIZE_FIELD.getPreferredName(), shardSize);
|
||||
builder.field(SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD.getPreferredName(), maxDocsPerValue);
|
||||
if (executionHint != null) {
|
||||
builder.field(SamplerAggregator.EXECUTION_HINT_FIELD.getPreferredName(), executionHint);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DiversifiedAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
DiversifiedAggregatorBuilder factory = new DiversifiedAggregatorBuilder(name);
|
||||
factory.shardSize = in.readVInt();
|
||||
factory.maxDocsPerValue = in.readVInt();
|
||||
factory.executionHint = in.readOptionalString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(shardSize);
|
||||
out.writeVInt(maxDocsPerValue);
|
||||
out.writeOptionalString(executionHint);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(shardSize, maxDocsPerValue, executionHint);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
DiversifiedAggregatorBuilder other = (DiversifiedAggregatorBuilder) obj;
|
||||
return Objects.equals(shardSize, other.shardSize)
|
||||
&& Objects.equals(maxDocsPerValue, other.maxDocsPerValue)
|
||||
&& Objects.equals(executionHint, other.executionHint);
|
||||
}
|
||||
}
|
|
@ -41,13 +41,13 @@ public class DiversifiedSamplerParser extends AnyValuesSourceParser {
|
|||
|
||||
@Override
|
||||
public String type() {
|
||||
return SamplerAggregator.DiversifiedAggregatorBuilder.TYPE.name();
|
||||
return DiversifiedAggregatorBuilder.TYPE.name();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SamplerAggregator.DiversifiedAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected DiversifiedAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
SamplerAggregator.DiversifiedAggregatorBuilder factory = new SamplerAggregator.DiversifiedAggregatorBuilder(aggregationName);
|
||||
DiversifiedAggregatorBuilder factory = new DiversifiedAggregatorBuilder(aggregationName);
|
||||
Integer shardSize = (Integer) otherOptions.get(SamplerAggregator.SHARD_SIZE_FIELD);
|
||||
if (shardSize != null) {
|
||||
factory.shardSize(shardSize);
|
||||
|
@ -88,7 +88,7 @@ public class DiversifiedSamplerParser extends AnyValuesSourceParser {
|
|||
|
||||
@Override
|
||||
public AggregatorBuilder<?> getFactoryPrototypes() {
|
||||
return SamplerAggregator.DiversifiedAggregatorBuilder.PROTOTYPE;
|
||||
return DiversifiedAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,34 +21,22 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Aggregate on only the top-scoring docs on a shard.
|
||||
|
@ -191,189 +179,6 @@ public class SamplerAggregator extends SingleBucketAggregator {
|
|||
return new InternalSampler(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class SamplerAggregatorBuilder extends AggregatorBuilder<SamplerAggregatorBuilder> {
|
||||
|
||||
static final SamplerAggregatorBuilder PROTOTYPE = new SamplerAggregatorBuilder("");
|
||||
|
||||
public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
|
||||
|
||||
private int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
|
||||
|
||||
public SamplerAggregatorBuilder(String name) {
|
||||
super(name, InternalSampler.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned from each shard.
|
||||
*/
|
||||
public SamplerAggregatorBuilder shardSize(int shardSize) {
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned from each shard.
|
||||
*/
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SamplerAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new SamplerAggregatorFactory(name, type, shardSize, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(SHARD_SIZE_FIELD.getPreferredName(), shardSize);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SamplerAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
SamplerAggregatorBuilder factory = new SamplerAggregatorBuilder(name);
|
||||
factory.shardSize = in.readVInt();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
SamplerAggregatorBuilder other = (SamplerAggregatorBuilder) obj;
|
||||
return Objects.equals(shardSize, other.shardSize);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class DiversifiedAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource, DiversifiedAggregatorBuilder> {
|
||||
|
||||
public static final Type TYPE = new Type("diversified_sampler");
|
||||
|
||||
static final DiversifiedAggregatorBuilder PROTOTYPE = new DiversifiedAggregatorBuilder("");
|
||||
|
||||
public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
|
||||
|
||||
private int shardSize = SamplerAggregatorBuilder.DEFAULT_SHARD_SAMPLE_SIZE;
|
||||
private int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT;
|
||||
private String executionHint = null;
|
||||
|
||||
public DiversifiedAggregatorBuilder(String name) {
|
||||
super(name, TYPE, ValuesSourceType.ANY, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned from each shard.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder shardSize(int shardSize) {
|
||||
if (shardSize < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[shardSize] must be greater than or equal to 0. Found [" + shardSize + "] in [" + name + "]");
|
||||
}
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned from each shard.
|
||||
*/
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned per value.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder maxDocsPerValue(int maxDocsPerValue) {
|
||||
if (maxDocsPerValue < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"[maxDocsPerValue] must be greater than or equal to 0. Found [" + maxDocsPerValue + "] in [" + name + "]");
|
||||
}
|
||||
this.maxDocsPerValue = maxDocsPerValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned per value.
|
||||
*/
|
||||
public int maxDocsPerValue() {
|
||||
return maxDocsPerValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the execution hint.
|
||||
*/
|
||||
public DiversifiedAggregatorBuilder executionHint(String executionHint) {
|
||||
this.executionHint = executionHint;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the execution hint.
|
||||
*/
|
||||
public String executionHint() {
|
||||
return executionHint;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
|
||||
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new DiversifiedAggregatorFactory(name, TYPE, config, shardSize, maxDocsPerValue, executionHint, context, parent,
|
||||
subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(SHARD_SIZE_FIELD.getPreferredName(), shardSize);
|
||||
builder.field(MAX_DOCS_PER_VALUE_FIELD.getPreferredName(), maxDocsPerValue);
|
||||
if (executionHint != null) {
|
||||
builder.field(EXECUTION_HINT_FIELD.getPreferredName(), executionHint);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DiversifiedAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
DiversifiedAggregatorBuilder factory = new DiversifiedAggregatorBuilder(name);
|
||||
factory.shardSize = in.readVInt();
|
||||
factory.maxDocsPerValue = in.readVInt();
|
||||
factory.executionHint = in.readOptionalString();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(shardSize);
|
||||
out.writeVInt(maxDocsPerValue);
|
||||
out.writeOptionalString(executionHint);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(shardSize, maxDocsPerValue, executionHint);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
DiversifiedAggregatorBuilder other = (DiversifiedAggregatorBuilder) obj;
|
||||
return Objects.equals(shardSize, other.shardSize)
|
||||
&& Objects.equals(maxDocsPerValue, other.maxDocsPerValue)
|
||||
&& Objects.equals(executionHint, other.executionHint);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
|
||||
if (bdd == null) {
|
||||
|
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.bucket.sampler;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class SamplerAggregatorBuilder extends AggregatorBuilder<SamplerAggregatorBuilder> {
|
||||
|
||||
static final SamplerAggregatorBuilder PROTOTYPE = new SamplerAggregatorBuilder("");
|
||||
|
||||
public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
|
||||
|
||||
private int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
|
||||
|
||||
public SamplerAggregatorBuilder(String name) {
|
||||
super(name, InternalSampler.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max num docs to be returned from each shard.
|
||||
*/
|
||||
public SamplerAggregatorBuilder shardSize(int shardSize) {
|
||||
this.shardSize = shardSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the max num docs to be returned from each shard.
|
||||
*/
|
||||
public int shardSize() {
|
||||
return shardSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SamplerAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
|
||||
throws IOException {
|
||||
return new SamplerAggregatorFactory(name, type, shardSize, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(SamplerAggregator.SHARD_SIZE_FIELD.getPreferredName(), shardSize);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SamplerAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
SamplerAggregatorBuilder factory = new SamplerAggregatorBuilder(name);
|
||||
factory.shardSize = in.readVInt();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(shardSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
SamplerAggregatorBuilder other = (SamplerAggregatorBuilder) obj;
|
||||
return Objects.equals(shardSize, other.shardSize);
|
||||
}
|
||||
|
||||
}
|
|
@ -36,7 +36,7 @@ public class SamplerParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SamplerAggregator.SamplerAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public SamplerAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
|
||||
XContentParser.Token token;
|
||||
|
@ -59,7 +59,7 @@ public class SamplerParser implements Aggregator.Parser {
|
|||
}
|
||||
}
|
||||
|
||||
SamplerAggregator.SamplerAggregatorBuilder factory = new SamplerAggregator.SamplerAggregatorBuilder(aggregationName);
|
||||
SamplerAggregatorBuilder factory = new SamplerAggregatorBuilder(aggregationName);
|
||||
if (shardSize != null) {
|
||||
factory.shardSize(shardSize);
|
||||
}
|
||||
|
@ -67,8 +67,8 @@ public class SamplerParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SamplerAggregator.SamplerAggregatorBuilder getFactoryPrototypes() {
|
||||
return SamplerAggregator.SamplerAggregatorBuilder.PROTOTYPE;
|
||||
public SamplerAggregatorBuilder getFactoryPrototypes() {
|
||||
return SamplerAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,29 +19,19 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.avg;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -120,47 +110,6 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
return new InternalAvg(name, 0.0, 0L, formatter, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class AvgAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, AvgAggregatorBuilder> {
|
||||
|
||||
static final AvgAggregatorBuilder PROTOTYPE = new AvgAggregatorBuilder("");
|
||||
|
||||
public AvgAggregatorBuilder(String name) {
|
||||
super(name, InternalAvg.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new AvgAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new AvgAggregator.AvgAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(counts, sums);
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.avg;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class AvgAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, AvgAggregatorBuilder> {
|
||||
|
||||
static final AvgAggregatorBuilder PROTOTYPE = new AvgAggregatorBuilder("");
|
||||
|
||||
public AvgAggregatorBuilder(String name) {
|
||||
super(name, InternalAvg.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new AvgAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new AvgAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -49,14 +49,14 @@ public class AvgParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AvgAggregator.AvgAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected AvgAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new AvgAggregator.AvgAggregatorBuilder(aggregationName);
|
||||
return new AvgAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AvgAggregator.AvgAggregatorBuilder getFactoryPrototypes() {
|
||||
return AvgAggregator.AvgAggregatorBuilder.PROTOTYPE;
|
||||
public AvgAggregatorBuilder getFactoryPrototypes() {
|
||||
return AvgAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,31 +22,22 @@ package org.elasticsearch.search.aggregations.metrics.geobounds;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public final class GeoBoundsAggregator extends MetricsAggregator {
|
||||
|
||||
|
@ -175,68 +166,4 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
|
|||
public void doClose() {
|
||||
Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights);
|
||||
}
|
||||
|
||||
public static class GeoBoundsAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoBoundsAggregatorBuilder> {
|
||||
|
||||
static final GeoBoundsAggregatorBuilder PROTOTYPE = new GeoBoundsAggregatorBuilder("");
|
||||
|
||||
private boolean wrapLongitude = true;
|
||||
|
||||
public GeoBoundsAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoBounds.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set whether to wrap longitudes. Defaults to true.
|
||||
*/
|
||||
public GeoBoundsAggregatorBuilder wrapLongitude(boolean wrapLongitude) {
|
||||
this.wrapLongitude = wrapLongitude;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get whether to wrap longitudes.
|
||||
*/
|
||||
public boolean wrapLongitude() {
|
||||
return wrapLongitude;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoBoundsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new GeoBoundsAggregatorFactory(name, type, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoBoundsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregatorBuilder(name);
|
||||
factory.wrapLongitude = in.readBoolean();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(wrapLongitude);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(WRAP_LONGITUDE_FIELD.getPreferredName(), wrapLongitude);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(wrapLongitude);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoBoundsAggregatorBuilder other = (GeoBoundsAggregatorBuilder) obj;
|
||||
return Objects.equals(wrapLongitude, other.wrapLongitude);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.geobounds;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class GeoBoundsAggregatorBuilder extends ValuesSourceAggregatorBuilder<ValuesSource.GeoPoint, GeoBoundsAggregatorBuilder> {
|
||||
|
||||
static final GeoBoundsAggregatorBuilder PROTOTYPE = new GeoBoundsAggregatorBuilder("");
|
||||
|
||||
private boolean wrapLongitude = true;
|
||||
|
||||
public GeoBoundsAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoBounds.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set whether to wrap longitudes. Defaults to true.
|
||||
*/
|
||||
public GeoBoundsAggregatorBuilder wrapLongitude(boolean wrapLongitude) {
|
||||
this.wrapLongitude = wrapLongitude;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get whether to wrap longitudes.
|
||||
*/
|
||||
public boolean wrapLongitude() {
|
||||
return wrapLongitude;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoBoundsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new GeoBoundsAggregatorFactory(name, type, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoBoundsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregatorBuilder(name);
|
||||
factory.wrapLongitude = in.readBoolean();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(wrapLongitude);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(GeoBoundsAggregator.WRAP_LONGITUDE_FIELD.getPreferredName(), wrapLongitude);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(wrapLongitude);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
GeoBoundsAggregatorBuilder other = (GeoBoundsAggregatorBuilder) obj;
|
||||
return Objects.equals(wrapLongitude, other.wrapLongitude);
|
||||
}
|
||||
|
||||
}
|
|
@ -42,9 +42,9 @@ public class GeoBoundsParser extends GeoPointValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected GeoBoundsAggregator.GeoBoundsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected GeoBoundsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
GeoBoundsAggregator.GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregator.GeoBoundsAggregatorBuilder(aggregationName);
|
||||
GeoBoundsAggregatorBuilder factory = new GeoBoundsAggregatorBuilder(aggregationName);
|
||||
Boolean wrapLongitude = (Boolean) otherOptions.get(GeoBoundsAggregator.WRAP_LONGITUDE_FIELD);
|
||||
if (wrapLongitude != null) {
|
||||
factory.wrapLongitude(wrapLongitude);
|
||||
|
@ -65,8 +65,8 @@ public class GeoBoundsParser extends GeoPointValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public GeoBoundsAggregator.GeoBoundsAggregatorBuilder getFactoryPrototypes() {
|
||||
return GeoBoundsAggregator.GeoBoundsAggregatorBuilder.PROTOTYPE;
|
||||
public GeoBoundsAggregatorBuilder getFactoryPrototypes() {
|
||||
return GeoBoundsAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,27 +22,19 @@ package org.elasticsearch.search.aggregations.metrics.geocentroid;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.spatial.util.GeoEncodingUtils;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -126,46 +118,4 @@ public final class GeoCentroidAggregator extends MetricsAggregator {
|
|||
public void doClose() {
|
||||
Releasables.close(centroids, counts);
|
||||
}
|
||||
|
||||
public static class GeoCentroidAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.GeoPoint, GeoCentroidAggregatorBuilder> {
|
||||
|
||||
static final GeoCentroidAggregatorBuilder PROTOTYPE = new GeoCentroidAggregatorBuilder("");
|
||||
|
||||
public GeoCentroidAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoCentroid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoCentroidAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new GeoCentroidAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoCentroidAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
return new GeoCentroidAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.geocentroid;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.GeoPoint;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class GeoCentroidAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.GeoPoint, GeoCentroidAggregatorBuilder> {
|
||||
|
||||
static final GeoCentroidAggregatorBuilder PROTOTYPE = new GeoCentroidAggregatorBuilder("");
|
||||
|
||||
public GeoCentroidAggregatorBuilder(String name) {
|
||||
super(name, InternalGeoCentroid.TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoCentroidAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new GeoCentroidAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GeoCentroidAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
return new GeoCentroidAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -51,13 +51,13 @@ public class GeoCentroidParser extends GeoPointValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected GeoCentroidAggregator.GeoCentroidAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected GeoCentroidAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new GeoCentroidAggregator.GeoCentroidAggregatorBuilder(aggregationName);
|
||||
return new GeoCentroidAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GeoCentroidAggregator.GeoCentroidAggregatorBuilder getFactoryPrototypes() {
|
||||
return GeoCentroidAggregator.GeoCentroidAggregatorBuilder.PROTOTYPE;
|
||||
public GeoCentroidAggregatorBuilder getFactoryPrototypes() {
|
||||
return GeoCentroidAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,30 +19,20 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.max;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.NumericDoubleValues;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -122,48 +112,6 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class MaxAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MaxAggregatorBuilder> {
|
||||
|
||||
static final MaxAggregatorBuilder PROTOTYPE = new MaxAggregatorBuilder("");
|
||||
|
||||
public MaxAggregatorBuilder(String name) {
|
||||
super(name, InternalMax.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MaxAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MaxAggregator.MaxAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(maxes);
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.max;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class MaxAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MaxAggregatorBuilder> {
|
||||
|
||||
static final MaxAggregatorBuilder PROTOTYPE = new MaxAggregatorBuilder("");
|
||||
|
||||
public MaxAggregatorBuilder(String name) {
|
||||
super(name, InternalMax.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MaxAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MaxAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -49,14 +49,14 @@ public class MaxParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected MaxAggregator.MaxAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected MaxAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new MaxAggregator.MaxAggregatorBuilder(aggregationName);
|
||||
return new MaxAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MaxAggregator.MaxAggregatorBuilder getFactoryPrototypes() {
|
||||
return MaxAggregator.MaxAggregatorBuilder.PROTOTYPE;
|
||||
public MaxAggregatorBuilder getFactoryPrototypes() {
|
||||
return MaxAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,30 +19,20 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.min;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.NumericDoubleValues;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -121,47 +111,6 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class MinAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MinAggregatorBuilder> {
|
||||
|
||||
static final MinAggregatorBuilder PROTOTYPE = new MinAggregatorBuilder("");
|
||||
|
||||
public MinAggregatorBuilder(String name) {
|
||||
super(name, InternalMin.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MinAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MinAggregator.MinAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(mins);
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.min;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class MinAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, MinAggregatorBuilder> {
|
||||
|
||||
static final MinAggregatorBuilder PROTOTYPE = new MinAggregatorBuilder("");
|
||||
|
||||
public MinAggregatorBuilder(String name) {
|
||||
super(name, InternalMin.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new MinAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new MinAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -50,13 +50,13 @@ public class MinParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected MinAggregator.MinAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected MinAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new MinAggregator.MinAggregatorBuilder(aggregationName);
|
||||
return new MinAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MinAggregator.MinAggregatorBuilder getFactoryPrototypes() {
|
||||
return MinAggregator.MinAggregatorBuilder.PROTOTYPE;
|
||||
public MinAggregatorBuilder getFactoryPrototypes() {
|
||||
return MinAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,6 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.scripted;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
import org.elasticsearch.script.LeafSearchScript;
|
||||
import org.elasticsearch.script.Script;
|
||||
|
@ -30,12 +27,9 @@ import org.elasticsearch.script.ScriptContext;
|
|||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
@ -43,7 +37,6 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ScriptedMetricAggregator extends MetricsAggregator {
|
||||
|
||||
|
@ -106,186 +99,4 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
|
|||
return new InternalScriptedMetric(name, null, reduceScript, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class ScriptedMetricAggregatorBuilder extends AggregatorBuilder<ScriptedMetricAggregatorBuilder> {
|
||||
|
||||
static final ScriptedMetricAggregatorBuilder PROTOTYPE = new ScriptedMetricAggregatorBuilder("");
|
||||
|
||||
private Script initScript;
|
||||
private Script mapScript;
|
||||
private Script combineScript;
|
||||
private Script reduceScript;
|
||||
private Map<String, Object> params;
|
||||
|
||||
public ScriptedMetricAggregatorBuilder(String name) {
|
||||
super(name, InternalScriptedMetric.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>init</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder initScript(Script initScript) {
|
||||
if (initScript == null) {
|
||||
throw new IllegalArgumentException("[initScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.initScript = initScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>init</tt> script.
|
||||
*/
|
||||
public Script initScript() {
|
||||
return initScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>map</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder mapScript(Script mapScript) {
|
||||
if (mapScript == null) {
|
||||
throw new IllegalArgumentException("[mapScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.mapScript = mapScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>map</tt> script.
|
||||
*/
|
||||
public Script mapScript() {
|
||||
return mapScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>combine</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder combineScript(Script combineScript) {
|
||||
if (combineScript == null) {
|
||||
throw new IllegalArgumentException("[combineScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.combineScript = combineScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>combine</tt> script.
|
||||
*/
|
||||
public Script combineScript() {
|
||||
return combineScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>reduce</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder reduceScript(Script reduceScript) {
|
||||
if (reduceScript == null) {
|
||||
throw new IllegalArgumentException("[reduceScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.reduceScript = reduceScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>reduce</tt> script.
|
||||
*/
|
||||
public Script reduceScript() {
|
||||
return reduceScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set parameters that will be available in the <tt>init</tt>,
|
||||
* <tt>map</tt> and <tt>combine</tt> phases.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder params(Map<String, Object> params) {
|
||||
if (params == null) {
|
||||
throw new IllegalArgumentException("[params] must not be null: [" + name + "]");
|
||||
}
|
||||
this.params = params;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get parameters that will be available in the <tt>init</tt>,
|
||||
* <tt>map</tt> and <tt>combine</tt> phases.
|
||||
*/
|
||||
public Map<String, Object> params() {
|
||||
return params;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ScriptedMetricAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent,
|
||||
Builder subfactoriesBuilder) throws IOException {
|
||||
return new ScriptedMetricAggregatorFactory(name, type, initScript, mapScript, combineScript, reduceScript, params, context,
|
||||
parent, subfactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException {
|
||||
builder.startObject();
|
||||
if (initScript != null) {
|
||||
builder.field(ScriptedMetricParser.INIT_SCRIPT_FIELD.getPreferredName(), initScript);
|
||||
}
|
||||
|
||||
if (mapScript != null) {
|
||||
builder.field(ScriptedMetricParser.MAP_SCRIPT_FIELD.getPreferredName(), mapScript);
|
||||
}
|
||||
|
||||
if (combineScript != null) {
|
||||
builder.field(ScriptedMetricParser.COMBINE_SCRIPT_FIELD.getPreferredName(), combineScript);
|
||||
}
|
||||
|
||||
if (reduceScript != null) {
|
||||
builder.field(ScriptedMetricParser.REDUCE_SCRIPT_FIELD.getPreferredName(), reduceScript);
|
||||
}
|
||||
if (params != null) {
|
||||
builder.field(ScriptedMetricParser.PARAMS_FIELD.getPreferredName());
|
||||
builder.map(params);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ScriptedMetricAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregatorBuilder(name);
|
||||
factory.initScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.mapScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.combineScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.reduceScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
if (in.readBoolean()) {
|
||||
factory.params = in.readMap();
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalStreamable(initScript);
|
||||
out.writeOptionalStreamable(mapScript);
|
||||
out.writeOptionalStreamable(combineScript);
|
||||
out.writeOptionalStreamable(reduceScript);
|
||||
boolean hasParams = params != null;
|
||||
out.writeBoolean(hasParams);
|
||||
if (hasParams) {
|
||||
out.writeMap(params);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(initScript, mapScript, combineScript, reduceScript, params);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
ScriptedMetricAggregatorBuilder other = (ScriptedMetricAggregatorBuilder) obj;
|
||||
return Objects.equals(initScript, other.initScript)
|
||||
&& Objects.equals(mapScript, other.mapScript)
|
||||
&& Objects.equals(combineScript, other.combineScript)
|
||||
&& Objects.equals(reduceScript, other.reduceScript)
|
||||
&& Objects.equals(params, other.params);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,215 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.aggregations.metrics.scripted;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ScriptedMetricAggregatorBuilder extends AggregatorBuilder<ScriptedMetricAggregatorBuilder> {
|
||||
|
||||
static final ScriptedMetricAggregatorBuilder PROTOTYPE = new ScriptedMetricAggregatorBuilder("");
|
||||
|
||||
private Script initScript;
|
||||
private Script mapScript;
|
||||
private Script combineScript;
|
||||
private Script reduceScript;
|
||||
private Map<String, Object> params;
|
||||
|
||||
public ScriptedMetricAggregatorBuilder(String name) {
|
||||
super(name, InternalScriptedMetric.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>init</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder initScript(Script initScript) {
|
||||
if (initScript == null) {
|
||||
throw new IllegalArgumentException("[initScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.initScript = initScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>init</tt> script.
|
||||
*/
|
||||
public Script initScript() {
|
||||
return initScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>map</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder mapScript(Script mapScript) {
|
||||
if (mapScript == null) {
|
||||
throw new IllegalArgumentException("[mapScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.mapScript = mapScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>map</tt> script.
|
||||
*/
|
||||
public Script mapScript() {
|
||||
return mapScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>combine</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder combineScript(Script combineScript) {
|
||||
if (combineScript == null) {
|
||||
throw new IllegalArgumentException("[combineScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.combineScript = combineScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>combine</tt> script.
|
||||
*/
|
||||
public Script combineScript() {
|
||||
return combineScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the <tt>reduce</tt> script.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder reduceScript(Script reduceScript) {
|
||||
if (reduceScript == null) {
|
||||
throw new IllegalArgumentException("[reduceScript] must not be null: [" + name + "]");
|
||||
}
|
||||
this.reduceScript = reduceScript;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the <tt>reduce</tt> script.
|
||||
*/
|
||||
public Script reduceScript() {
|
||||
return reduceScript;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set parameters that will be available in the <tt>init</tt>,
|
||||
* <tt>map</tt> and <tt>combine</tt> phases.
|
||||
*/
|
||||
public ScriptedMetricAggregatorBuilder params(Map<String, Object> params) {
|
||||
if (params == null) {
|
||||
throw new IllegalArgumentException("[params] must not be null: [" + name + "]");
|
||||
}
|
||||
this.params = params;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get parameters that will be available in the <tt>init</tt>,
|
||||
* <tt>map</tt> and <tt>combine</tt> phases.
|
||||
*/
|
||||
public Map<String, Object> params() {
|
||||
return params;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ScriptedMetricAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent,
|
||||
Builder subfactoriesBuilder) throws IOException {
|
||||
return new ScriptedMetricAggregatorFactory(name, type, initScript, mapScript, combineScript, reduceScript, params, context,
|
||||
parent, subfactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException {
|
||||
builder.startObject();
|
||||
if (initScript != null) {
|
||||
builder.field(ScriptedMetricParser.INIT_SCRIPT_FIELD.getPreferredName(), initScript);
|
||||
}
|
||||
|
||||
if (mapScript != null) {
|
||||
builder.field(ScriptedMetricParser.MAP_SCRIPT_FIELD.getPreferredName(), mapScript);
|
||||
}
|
||||
|
||||
if (combineScript != null) {
|
||||
builder.field(ScriptedMetricParser.COMBINE_SCRIPT_FIELD.getPreferredName(), combineScript);
|
||||
}
|
||||
|
||||
if (reduceScript != null) {
|
||||
builder.field(ScriptedMetricParser.REDUCE_SCRIPT_FIELD.getPreferredName(), reduceScript);
|
||||
}
|
||||
if (params != null) {
|
||||
builder.field(ScriptedMetricParser.PARAMS_FIELD.getPreferredName());
|
||||
builder.map(params);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ScriptedMetricAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregatorBuilder(name);
|
||||
factory.initScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.mapScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.combineScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
factory.reduceScript = in.readOptionalStreamable(Script.SUPPLIER);
|
||||
if (in.readBoolean()) {
|
||||
factory.params = in.readMap();
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalStreamable(initScript);
|
||||
out.writeOptionalStreamable(mapScript);
|
||||
out.writeOptionalStreamable(combineScript);
|
||||
out.writeOptionalStreamable(reduceScript);
|
||||
boolean hasParams = params != null;
|
||||
out.writeBoolean(hasParams);
|
||||
if (hasParams) {
|
||||
out.writeMap(params);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(initScript, mapScript, combineScript, reduceScript, params);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
ScriptedMetricAggregatorBuilder other = (ScriptedMetricAggregatorBuilder) obj;
|
||||
return Objects.equals(initScript, other.initScript)
|
||||
&& Objects.equals(mapScript, other.mapScript)
|
||||
&& Objects.equals(combineScript, other.combineScript)
|
||||
&& Objects.equals(reduceScript, other.reduceScript)
|
||||
&& Objects.equals(params, other.params);
|
||||
}
|
||||
|
||||
}
|
|
@ -52,7 +52,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
public ScriptedMetricAggregatorBuilder parse(String aggregationName, XContentParser parser,
|
||||
QueryParseContext context) throws IOException {
|
||||
Script initScript = null;
|
||||
Script mapScript = null;
|
||||
|
@ -140,7 +140,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
|
|||
throw new ParsingException(parser.getTokenLocation(), "map_script field is required in [" + aggregationName + "].");
|
||||
}
|
||||
|
||||
ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder(aggregationName);
|
||||
ScriptedMetricAggregatorBuilder factory = new ScriptedMetricAggregatorBuilder(aggregationName);
|
||||
if (initScript != null) {
|
||||
factory.initScript(initScript);
|
||||
}
|
||||
|
@ -160,8 +160,8 @@ public class ScriptedMetricParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder getFactoryPrototypes() {
|
||||
return ScriptedMetricAggregator.ScriptedMetricAggregatorBuilder.PROTOTYPE;
|
||||
public ScriptedMetricAggregatorBuilder getFactoryPrototypes() {
|
||||
return ScriptedMetricAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,29 +19,19 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.stats;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -163,47 +153,6 @@ public class StatsAggregator extends NumericMetricsAggregator.MultiValue {
|
|||
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class StatsAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, StatsAggregatorBuilder> {
|
||||
|
||||
static final StatsAggregatorBuilder PROTOTYPE = new StatsAggregatorBuilder("");
|
||||
|
||||
public StatsAggregatorBuilder(String name) {
|
||||
super(name, InternalStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new StatsAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new StatsAggregator.StatsAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(counts, maxes, mins, sums);
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.stats;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class StatsAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, StatsAggregatorBuilder> {
|
||||
|
||||
static final StatsAggregatorBuilder PROTOTYPE = new StatsAggregatorBuilder("");
|
||||
|
||||
public StatsAggregatorBuilder(String name) {
|
||||
super(name, InternalStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new StatsAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new StatsAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -50,13 +50,13 @@ public class StatsParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected StatsAggregator.StatsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected StatsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new StatsAggregator.StatsAggregatorBuilder(aggregationName);
|
||||
return new StatsAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorBuilder<?> getFactoryPrototypes() {
|
||||
return StatsAggregator.StatsAggregatorBuilder.PROTOTYPE;
|
||||
return StatsAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,35 +20,24 @@ package org.elasticsearch.search.aggregations.metrics.stats.extended;
|
|||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -199,63 +188,4 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
|
|||
public void doClose() {
|
||||
Releasables.close(counts, maxes, mins, sumOfSqrs, sums);
|
||||
}
|
||||
|
||||
public static class ExtendedStatsAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, ExtendedStatsAggregatorBuilder> {
|
||||
|
||||
static final ExtendedStatsAggregatorBuilder PROTOTYPE = new ExtendedStatsAggregatorBuilder("");
|
||||
|
||||
private double sigma = 2.0;
|
||||
|
||||
public ExtendedStatsAggregatorBuilder(String name) {
|
||||
super(name, InternalExtendedStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
public ExtendedStatsAggregatorBuilder sigma(double sigma) {
|
||||
if (sigma < 0.0) {
|
||||
throw new IllegalArgumentException("[sigma] must be greater than or equal to 0. Found [" + sigma + "] in [" + name + "]");
|
||||
}
|
||||
this.sigma = sigma;
|
||||
return this;
|
||||
}
|
||||
|
||||
public double sigma() {
|
||||
return sigma;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new ExtendedStatsAggregatorFactory(name, type, config, sigma, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder factory = new ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder(name);
|
||||
factory.sigma = in.readDouble();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(SIGMA_FIELD.getPreferredName(), sigma);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
ExtendedStatsAggregatorBuilder other = (ExtendedStatsAggregatorBuilder) obj;
|
||||
return Objects.equals(sigma, other.sigma);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.stats.extended;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ExtendedStatsAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, ExtendedStatsAggregatorBuilder> {
|
||||
|
||||
static final ExtendedStatsAggregatorBuilder PROTOTYPE = new ExtendedStatsAggregatorBuilder("");
|
||||
|
||||
private double sigma = 2.0;
|
||||
|
||||
public ExtendedStatsAggregatorBuilder(String name) {
|
||||
super(name, InternalExtendedStats.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
public ExtendedStatsAggregatorBuilder sigma(double sigma) {
|
||||
if (sigma < 0.0) {
|
||||
throw new IllegalArgumentException("[sigma] must be greater than or equal to 0. Found [" + sigma + "] in [" + name + "]");
|
||||
}
|
||||
this.sigma = sigma;
|
||||
return this;
|
||||
}
|
||||
|
||||
public double sigma() {
|
||||
return sigma;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new ExtendedStatsAggregatorFactory(name, type, config, sigma, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsAggregatorBuilder innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) throws IOException {
|
||||
ExtendedStatsAggregatorBuilder factory = new ExtendedStatsAggregatorBuilder(name);
|
||||
factory.sigma = in.readDouble();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ExtendedStatsAggregator.SIGMA_FIELD.getPreferredName(), sigma);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
ExtendedStatsAggregatorBuilder other = (ExtendedStatsAggregatorBuilder) obj;
|
||||
return Objects.equals(sigma, other.sigma);
|
||||
}
|
||||
}
|
|
@ -56,9 +56,9 @@ public class ExtendedStatsParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected ExtendedStatsAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder factory = new ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder(aggregationName);
|
||||
ExtendedStatsAggregatorBuilder factory = new ExtendedStatsAggregatorBuilder(aggregationName);
|
||||
Double sigma = (Double) otherOptions.get(ExtendedStatsAggregator.SIGMA_FIELD);
|
||||
if (sigma != null) {
|
||||
factory.sigma(sigma);
|
||||
|
@ -68,6 +68,6 @@ public class ExtendedStatsParser extends NumericValuesSourceParser {
|
|||
|
||||
@Override
|
||||
public AggregatorBuilder<?> getFactoryPrototypes() {
|
||||
return ExtendedStatsAggregator.ExtendedStatsAggregatorBuilder.PROTOTYPE;
|
||||
return ExtendedStatsAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,28 +19,18 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.sum;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -113,47 +103,6 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
return new InternalSum(name, 0.0, formatter, pipelineAggregators(), metaData());
|
||||
}
|
||||
|
||||
public static class SumAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, SumAggregatorBuilder> {
|
||||
|
||||
static final SumAggregatorBuilder PROTOTYPE = new SumAggregatorBuilder("");
|
||||
|
||||
public SumAggregatorBuilder(String name) {
|
||||
super(name, InternalSum.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SumAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new SumAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorBuilder<Numeric, SumAggregatorBuilder> innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new SumAggregator.SumAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doClose() {
|
||||
Releasables.close(sums);
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.sum;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class SumAggregatorBuilder extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource.Numeric, SumAggregatorBuilder> {
|
||||
|
||||
static final SumAggregatorBuilder PROTOTYPE = new SumAggregatorBuilder("");
|
||||
|
||||
public SumAggregatorBuilder(String name) {
|
||||
super(name, InternalSum.TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SumAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new SumAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorBuilder<Numeric, SumAggregatorBuilder> innerReadFrom(String name, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, StreamInput in) {
|
||||
return new SumAggregatorBuilder(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -50,13 +50,13 @@ public class SumParser extends NumericValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SumAggregator.SumAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected SumAggregatorBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new SumAggregator.SumAggregatorBuilder(aggregationName);
|
||||
return new SumAggregatorBuilder(aggregationName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorBuilder<?> getFactoryPrototypes() {
|
||||
return SumAggregator.SumAggregatorBuilder.PROTOTYPE;
|
||||
return SumAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,48 +30,25 @@ import org.apache.lucene.search.TopDocsCollector;
|
|||
import org.apache.lucene.search.TopFieldCollector;
|
||||
import org.apache.lucene.search.TopFieldDocs;
|
||||
import org.apache.lucene.search.TopScoreDocCollector;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.util.LongObjectPagedHashMap;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.AggregationInitializationException;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
|
||||
import org.elasticsearch.search.fetch.FetchPhase;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.source.FetchSourceContext;
|
||||
import org.elasticsearch.search.highlight.HighlightBuilder;
|
||||
import org.elasticsearch.search.internal.InternalSearchHit;
|
||||
import org.elasticsearch.search.internal.InternalSearchHits;
|
||||
import org.elasticsearch.search.internal.SubSearchContext;
|
||||
import org.elasticsearch.search.sort.SortBuilder;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -200,599 +177,4 @@ public class TopHitsAggregator extends MetricsAggregator {
|
|||
protected void doClose() {
|
||||
Releasables.close(topDocsCollectors);
|
||||
}
|
||||
|
||||
public static class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregatorBuilder> {
|
||||
|
||||
static final TopHitsAggregatorBuilder PROTOTYPE = new TopHitsAggregatorBuilder("");
|
||||
|
||||
private int from = 0;
|
||||
private int size = 3;
|
||||
private boolean explain = false;
|
||||
private boolean version = false;
|
||||
private boolean trackScores = false;
|
||||
private List<BytesReference> sorts = null;
|
||||
private HighlightBuilder highlightBuilder;
|
||||
private List<String> fieldNames;
|
||||
private List<String> fieldDataFields;
|
||||
private List<ScriptField> scriptFields;
|
||||
private FetchSourceContext fetchSourceContext;
|
||||
|
||||
public TopHitsAggregatorBuilder(String name) {
|
||||
super(name, InternalTopHits.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* From index to start the search from. Defaults to <tt>0</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder from(int from) {
|
||||
if (from < 0) {
|
||||
throw new IllegalArgumentException("[from] must be greater than or equal to 0. Found [" + from + "] in [" + name + "]");
|
||||
}
|
||||
this.from = from;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the from index to start the search from.
|
||||
**/
|
||||
public int from() {
|
||||
return from;
|
||||
}
|
||||
|
||||
/**
|
||||
* The number of search hits to return. Defaults to <tt>10</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder size(int size) {
|
||||
if (size < 0) {
|
||||
throw new IllegalArgumentException("[size] must be greater than or equal to 0. Found [" + size + "] in [" + name + "]");
|
||||
}
|
||||
this.size = size;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of search hits to return.
|
||||
*/
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort against the given field name and the sort ordering.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param order
|
||||
* The sort ordering
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(String name, SortOrder order) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("sort [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("sort [order] must not be null: [" + name + "]");
|
||||
}
|
||||
sort(SortBuilders.fieldSort(name).order(order));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a sort against the given field name.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field to sort by
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(String name) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("sort [name] must not be null: [" + name + "]");
|
||||
}
|
||||
sort(SortBuilders.fieldSort(name));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort builder.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(SortBuilder sort) {
|
||||
if (sort == null) {
|
||||
throw new IllegalArgumentException("[sort] must not be null: [" + name + "]");
|
||||
}
|
||||
try {
|
||||
if (sorts == null) {
|
||||
sorts = new ArrayList<>();
|
||||
}
|
||||
// NORELEASE when sort has been refactored and made writeable
|
||||
// add the sortBuilcer to the List directly instead of
|
||||
// serialising to XContent
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
builder.startObject();
|
||||
sort.toXContent(builder, EMPTY_PARAMS);
|
||||
builder.endObject();
|
||||
sorts.add(builder.bytes());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort builder.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sorts(List<BytesReference> sorts) {
|
||||
if (sorts == null) {
|
||||
throw new IllegalArgumentException("[sorts] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.sorts == null) {
|
||||
this.sorts = new ArrayList<>();
|
||||
}
|
||||
for (BytesReference sort : sorts) {
|
||||
this.sorts.add(sort);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the bytes representing the sort builders for this request.
|
||||
*/
|
||||
public List<BytesReference> sorts() {
|
||||
return sorts;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds highlight to perform as part of the search.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder highlighter(HighlightBuilder highlightBuilder) {
|
||||
if (highlightBuilder == null) {
|
||||
throw new IllegalArgumentException("[highlightBuilder] must not be null: [" + name + "]");
|
||||
}
|
||||
this.highlightBuilder = highlightBuilder;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the hightlighter builder for this request.
|
||||
*/
|
||||
public HighlightBuilder highlighter() {
|
||||
return highlightBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the response should contain the stored _source for
|
||||
* every hit
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(boolean fetch) {
|
||||
if (this.fetchSourceContext == null) {
|
||||
this.fetchSourceContext = new FetchSourceContext(fetch);
|
||||
} else {
|
||||
this.fetchSourceContext.fetchSource(fetch);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate that _source should be returned with every hit, with an
|
||||
* "include" and/or "exclude" set which can include simple wildcard
|
||||
* elements.
|
||||
*
|
||||
* @param include
|
||||
* An optional include (optionally wildcarded) pattern to
|
||||
* filter the returned _source
|
||||
* @param exclude
|
||||
* An optional exclude (optionally wildcarded) pattern to
|
||||
* filter the returned _source
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable String include, @Nullable String exclude) {
|
||||
fetchSource(include == null ? Strings.EMPTY_ARRAY : new String[] { include },
|
||||
exclude == null ? Strings.EMPTY_ARRAY : new String[] { exclude });
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate that _source should be returned with every hit, with an
|
||||
* "include" and/or "exclude" set which can include simple wildcard
|
||||
* elements.
|
||||
*
|
||||
* @param includes
|
||||
* An optional list of include (optionally wildcarded)
|
||||
* pattern to filter the returned _source
|
||||
* @param excludes
|
||||
* An optional list of exclude (optionally wildcarded)
|
||||
* pattern to filter the returned _source
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable String[] includes, @Nullable String[] excludes) {
|
||||
fetchSourceContext = new FetchSourceContext(includes, excludes);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate how the _source should be fetched.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable FetchSourceContext fetchSourceContext) {
|
||||
if (fetchSourceContext == null) {
|
||||
throw new IllegalArgumentException("[fetchSourceContext] must not be null: [" + name + "]");
|
||||
}
|
||||
this.fetchSourceContext = fetchSourceContext;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link FetchSourceContext} which defines how the _source
|
||||
* should be fetched.
|
||||
*/
|
||||
public FetchSourceContext fetchSource() {
|
||||
return fetchSourceContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a field to load and return (note, it must be stored) as part of
|
||||
* the search request. If none are specified, the source of the document
|
||||
* will be return.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder field(String field) {
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException("[field] must not be null: [" + name + "]");
|
||||
}
|
||||
if (fieldNames == null) {
|
||||
fieldNames = new ArrayList<>();
|
||||
}
|
||||
fieldNames.add(field);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the fields to load and return as part of the search request. If
|
||||
* none are specified, the source of the document will be returned.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fields(List<String> fields) {
|
||||
if (fields == null) {
|
||||
throw new IllegalArgumentException("[fields] must not be null: [" + name + "]");
|
||||
}
|
||||
this.fieldNames = fields;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets no fields to be loaded, resulting in only id and type to be
|
||||
* returned per field.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder noFields() {
|
||||
this.fieldNames = Collections.emptyList();
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the fields to load and return as part of the search request.
|
||||
*/
|
||||
public List<String> fields() {
|
||||
return fieldNames;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a field to load from the field data cache and return as part of
|
||||
* the search request.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fieldDataField(String fieldDataField) {
|
||||
if (fieldDataField == null) {
|
||||
throw new IllegalArgumentException("[fieldDataField] must not be null: [" + name + "]");
|
||||
}
|
||||
if (fieldDataFields == null) {
|
||||
fieldDataFields = new ArrayList<>();
|
||||
}
|
||||
fieldDataFields.add(fieldDataField);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds fields to load from the field data cache and return as part of
|
||||
* the search request.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fieldDataFields(List<String> fieldDataFields) {
|
||||
if (fieldDataFields == null) {
|
||||
throw new IllegalArgumentException("[fieldDataFields] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.fieldDataFields == null) {
|
||||
this.fieldDataFields = new ArrayList<>();
|
||||
}
|
||||
this.fieldDataFields.addAll(fieldDataFields);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the field-data fields.
|
||||
*/
|
||||
public List<String> fieldDataFields() {
|
||||
return fieldDataFields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a script field under the given name with the provided script.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param script
|
||||
* The script
|
||||
*/
|
||||
public TopHitsAggregatorBuilder scriptField(String name, Script script) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("scriptField [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (script == null) {
|
||||
throw new IllegalArgumentException("scriptField [script] must not be null: [" + name + "]");
|
||||
}
|
||||
scriptField(name, script, false);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a script field under the given name with the provided script.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param script
|
||||
* The script
|
||||
*/
|
||||
public TopHitsAggregatorBuilder scriptField(String name, Script script, boolean ignoreFailure) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("scriptField [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (script == null) {
|
||||
throw new IllegalArgumentException("scriptField [script] must not be null: [" + name + "]");
|
||||
}
|
||||
if (scriptFields == null) {
|
||||
scriptFields = new ArrayList<>();
|
||||
}
|
||||
scriptFields.add(new ScriptField(name, script, ignoreFailure));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TopHitsAggregatorBuilder scriptFields(List<ScriptField> scriptFields) {
|
||||
if (scriptFields == null) {
|
||||
throw new IllegalArgumentException("[scriptFields] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.scriptFields == null) {
|
||||
this.scriptFields = new ArrayList<>();
|
||||
}
|
||||
this.scriptFields.addAll(scriptFields);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the script fields.
|
||||
*/
|
||||
public List<ScriptField> scriptFields() {
|
||||
return scriptFields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should each {@link org.elasticsearch.search.SearchHit} be returned
|
||||
* with an explanation of the hit (ranking).
|
||||
*/
|
||||
public TopHitsAggregatorBuilder explain(boolean explain) {
|
||||
this.explain = explain;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether each search hit will be returned with an
|
||||
* explanation of the hit (ranking)
|
||||
*/
|
||||
public boolean explain() {
|
||||
return explain;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should each {@link org.elasticsearch.search.SearchHit} be returned
|
||||
* with a version associated with it.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder version(boolean version) {
|
||||
this.version = version;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the document's version will be included in the
|
||||
* search hits.
|
||||
*/
|
||||
public boolean version() {
|
||||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies when sorting, and controls if scores will be tracked as well.
|
||||
* Defaults to <tt>false</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder trackScores(boolean trackScores) {
|
||||
this.trackScores = trackScores;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether scores will be tracked for this request.
|
||||
*/
|
||||
public boolean trackScores() {
|
||||
return trackScores;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopHitsAggregatorBuilder subAggregations(Builder subFactories) {
|
||||
throw new AggregationInitializationException("Aggregator [" + name + "] of type [" + type + "] cannot accept sub-aggregations");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TopHitsAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subfactoriesBuilder)
|
||||
throws IOException {
|
||||
return new TopHitsAggregatorFactory(name, type, from, size, explain, version, trackScores, sorts, highlightBuilder, fieldNames,
|
||||
fieldDataFields, scriptFields, fetchSourceContext, context, parent, subfactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(SearchSourceBuilder.FROM_FIELD.getPreferredName(), from);
|
||||
builder.field(SearchSourceBuilder.SIZE_FIELD.getPreferredName(), size);
|
||||
builder.field(SearchSourceBuilder.VERSION_FIELD.getPreferredName(), version);
|
||||
builder.field(SearchSourceBuilder.EXPLAIN_FIELD.getPreferredName(), explain);
|
||||
if (fetchSourceContext != null) {
|
||||
builder.field(SearchSourceBuilder._SOURCE_FIELD.getPreferredName(), fetchSourceContext);
|
||||
}
|
||||
if (fieldNames != null) {
|
||||
if (fieldNames.size() == 1) {
|
||||
builder.field(SearchSourceBuilder.FIELDS_FIELD.getPreferredName(), fieldNames.get(0));
|
||||
} else {
|
||||
builder.startArray(SearchSourceBuilder.FIELDS_FIELD.getPreferredName());
|
||||
for (String fieldName : fieldNames) {
|
||||
builder.value(fieldName);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
}
|
||||
if (fieldDataFields != null) {
|
||||
builder.startArray(SearchSourceBuilder.FIELDDATA_FIELDS_FIELD.getPreferredName());
|
||||
for (String fieldDataField : fieldDataFields) {
|
||||
builder.value(fieldDataField);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (scriptFields != null) {
|
||||
builder.startObject(SearchSourceBuilder.SCRIPT_FIELDS_FIELD.getPreferredName());
|
||||
for (ScriptField scriptField : scriptFields) {
|
||||
scriptField.toXContent(builder, params);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
if (sorts != null) {
|
||||
builder.startArray(SearchSourceBuilder.SORT_FIELD.getPreferredName());
|
||||
for (BytesReference sort : sorts) {
|
||||
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(sort);
|
||||
parser.nextToken();
|
||||
builder.copyCurrentStructure(parser);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (trackScores) {
|
||||
builder.field(SearchSourceBuilder.TRACK_SCORES_FIELD.getPreferredName(), true);
|
||||
}
|
||||
if (highlightBuilder != null) {
|
||||
this.highlightBuilder.toXContent(builder, params);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TopHitsAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
TopHitsAggregatorBuilder factory = new TopHitsAggregatorBuilder(name);
|
||||
factory.explain = in.readBoolean();
|
||||
factory.fetchSourceContext = FetchSourceContext.optionalReadFromStream(in);
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<String> fieldDataFields = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
fieldDataFields.add(in.readString());
|
||||
}
|
||||
factory.fieldDataFields = fieldDataFields;
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<String> fieldNames = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
fieldNames.add(in.readString());
|
||||
}
|
||||
factory.fieldNames = fieldNames;
|
||||
}
|
||||
factory.from = in.readVInt();
|
||||
if (in.readBoolean()) {
|
||||
factory.highlightBuilder = HighlightBuilder.PROTOTYPE.readFrom(in);
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<ScriptField> scriptFields = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
scriptFields.add(ScriptField.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory.scriptFields = scriptFields;
|
||||
}
|
||||
factory.size = in.readVInt();
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<BytesReference> sorts = new ArrayList<>();
|
||||
for (int i = 0; i < size; i++) {
|
||||
sorts.add(in.readBytesReference());
|
||||
}
|
||||
factory.sorts = sorts;
|
||||
}
|
||||
factory.trackScores = in.readBoolean();
|
||||
factory.version = in.readBoolean();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(explain);
|
||||
FetchSourceContext.optionalWriteToStream(fetchSourceContext, out);
|
||||
boolean hasFieldDataFields = fieldDataFields != null;
|
||||
out.writeBoolean(hasFieldDataFields);
|
||||
if (hasFieldDataFields) {
|
||||
out.writeVInt(fieldDataFields.size());
|
||||
for (String fieldName : fieldDataFields) {
|
||||
out.writeString(fieldName);
|
||||
}
|
||||
}
|
||||
boolean hasFieldNames = fieldNames != null;
|
||||
out.writeBoolean(hasFieldNames);
|
||||
if (hasFieldNames) {
|
||||
out.writeVInt(fieldNames.size());
|
||||
for (String fieldName : fieldNames) {
|
||||
out.writeString(fieldName);
|
||||
}
|
||||
}
|
||||
out.writeVInt(from);
|
||||
boolean hasHighlighter = highlightBuilder != null;
|
||||
out.writeBoolean(hasHighlighter);
|
||||
if (hasHighlighter) {
|
||||
highlightBuilder.writeTo(out);
|
||||
}
|
||||
boolean hasScriptFields = scriptFields != null;
|
||||
out.writeBoolean(hasScriptFields);
|
||||
if (hasScriptFields) {
|
||||
out.writeVInt(scriptFields.size());
|
||||
for (ScriptField scriptField : scriptFields) {
|
||||
scriptField.writeTo(out);
|
||||
}
|
||||
}
|
||||
out.writeVInt(size);
|
||||
boolean hasSorts = sorts != null;
|
||||
out.writeBoolean(hasSorts);
|
||||
if (hasSorts) {
|
||||
out.writeVInt(sorts.size());
|
||||
for (BytesReference sort : sorts) {
|
||||
out.writeBytesReference(sort);
|
||||
}
|
||||
}
|
||||
out.writeBoolean(trackScores);
|
||||
out.writeBoolean(version);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(explain, fetchSourceContext, fieldDataFields, fieldNames, from, highlightBuilder, scriptFields, size, sorts,
|
||||
trackScores, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
TopHitsAggregatorBuilder other = (TopHitsAggregatorBuilder) obj;
|
||||
return Objects.equals(explain, other.explain)
|
||||
&& Objects.equals(fetchSourceContext, other.fetchSourceContext)
|
||||
&& Objects.equals(fieldDataFields, other.fieldDataFields)
|
||||
&& Objects.equals(fieldNames, other.fieldNames)
|
||||
&& Objects.equals(from, other.from)
|
||||
&& Objects.equals(highlightBuilder, other.highlightBuilder)
|
||||
&& Objects.equals(scriptFields, other.scriptFields)
|
||||
&& Objects.equals(size, other.size)
|
||||
&& Objects.equals(sorts, other.sorts)
|
||||
&& Objects.equals(trackScores, other.trackScores)
|
||||
&& Objects.equals(version, other.version);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,644 @@
|
|||
/*
|
||||
* 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.tophits;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.AggregationInitializationException;
|
||||
import org.elasticsearch.search.aggregations.AggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
|
||||
import org.elasticsearch.search.fetch.source.FetchSourceContext;
|
||||
import org.elasticsearch.search.highlight.HighlightBuilder;
|
||||
import org.elasticsearch.search.sort.SortBuilder;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregatorBuilder> {
|
||||
|
||||
static final TopHitsAggregatorBuilder PROTOTYPE = new TopHitsAggregatorBuilder("");
|
||||
|
||||
private int from = 0;
|
||||
private int size = 3;
|
||||
private boolean explain = false;
|
||||
private boolean version = false;
|
||||
private boolean trackScores = false;
|
||||
private List<BytesReference> sorts = null;
|
||||
private HighlightBuilder highlightBuilder;
|
||||
private List<String> fieldNames;
|
||||
private List<String> fieldDataFields;
|
||||
private List<ScriptField> scriptFields;
|
||||
private FetchSourceContext fetchSourceContext;
|
||||
|
||||
public TopHitsAggregatorBuilder(String name) {
|
||||
super(name, InternalTopHits.TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* From index to start the search from. Defaults to <tt>0</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder from(int from) {
|
||||
if (from < 0) {
|
||||
throw new IllegalArgumentException("[from] must be greater than or equal to 0. Found [" + from + "] in [" + name + "]");
|
||||
}
|
||||
this.from = from;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the from index to start the search from.
|
||||
**/
|
||||
public int from() {
|
||||
return from;
|
||||
}
|
||||
|
||||
/**
|
||||
* The number of search hits to return. Defaults to <tt>10</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder size(int size) {
|
||||
if (size < 0) {
|
||||
throw new IllegalArgumentException("[size] must be greater than or equal to 0. Found [" + size + "] in [" + name + "]");
|
||||
}
|
||||
this.size = size;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of search hits to return.
|
||||
*/
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort against the given field name and the sort ordering.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param order
|
||||
* The sort ordering
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(String name, SortOrder order) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("sort [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("sort [order] must not be null: [" + name + "]");
|
||||
}
|
||||
sort(SortBuilders.fieldSort(name).order(order));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a sort against the given field name.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field to sort by
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(String name) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("sort [name] must not be null: [" + name + "]");
|
||||
}
|
||||
sort(SortBuilders.fieldSort(name));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort builder.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sort(SortBuilder sort) {
|
||||
if (sort == null) {
|
||||
throw new IllegalArgumentException("[sort] must not be null: [" + name + "]");
|
||||
}
|
||||
try {
|
||||
if (sorts == null) {
|
||||
sorts = new ArrayList<>();
|
||||
}
|
||||
// NORELEASE when sort has been refactored and made writeable
|
||||
// add the sortBuilcer to the List directly instead of
|
||||
// serialising to XContent
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
builder.startObject();
|
||||
sort.toXContent(builder, EMPTY_PARAMS);
|
||||
builder.endObject();
|
||||
sorts.add(builder.bytes());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a sort builder.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder sorts(List<BytesReference> sorts) {
|
||||
if (sorts == null) {
|
||||
throw new IllegalArgumentException("[sorts] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.sorts == null) {
|
||||
this.sorts = new ArrayList<>();
|
||||
}
|
||||
for (BytesReference sort : sorts) {
|
||||
this.sorts.add(sort);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the bytes representing the sort builders for this request.
|
||||
*/
|
||||
public List<BytesReference> sorts() {
|
||||
return sorts;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds highlight to perform as part of the search.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder highlighter(HighlightBuilder highlightBuilder) {
|
||||
if (highlightBuilder == null) {
|
||||
throw new IllegalArgumentException("[highlightBuilder] must not be null: [" + name + "]");
|
||||
}
|
||||
this.highlightBuilder = highlightBuilder;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the hightlighter builder for this request.
|
||||
*/
|
||||
public HighlightBuilder highlighter() {
|
||||
return highlightBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the response should contain the stored _source for
|
||||
* every hit
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(boolean fetch) {
|
||||
if (this.fetchSourceContext == null) {
|
||||
this.fetchSourceContext = new FetchSourceContext(fetch);
|
||||
} else {
|
||||
this.fetchSourceContext.fetchSource(fetch);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate that _source should be returned with every hit, with an
|
||||
* "include" and/or "exclude" set which can include simple wildcard
|
||||
* elements.
|
||||
*
|
||||
* @param include
|
||||
* An optional include (optionally wildcarded) pattern to
|
||||
* filter the returned _source
|
||||
* @param exclude
|
||||
* An optional exclude (optionally wildcarded) pattern to
|
||||
* filter the returned _source
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable String include, @Nullable String exclude) {
|
||||
fetchSource(include == null ? Strings.EMPTY_ARRAY : new String[] { include },
|
||||
exclude == null ? Strings.EMPTY_ARRAY : new String[] { exclude });
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate that _source should be returned with every hit, with an
|
||||
* "include" and/or "exclude" set which can include simple wildcard
|
||||
* elements.
|
||||
*
|
||||
* @param includes
|
||||
* An optional list of include (optionally wildcarded)
|
||||
* pattern to filter the returned _source
|
||||
* @param excludes
|
||||
* An optional list of exclude (optionally wildcarded)
|
||||
* pattern to filter the returned _source
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable String[] includes, @Nullable String[] excludes) {
|
||||
fetchSourceContext = new FetchSourceContext(includes, excludes);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate how the _source should be fetched.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fetchSource(@Nullable FetchSourceContext fetchSourceContext) {
|
||||
if (fetchSourceContext == null) {
|
||||
throw new IllegalArgumentException("[fetchSourceContext] must not be null: [" + name + "]");
|
||||
}
|
||||
this.fetchSourceContext = fetchSourceContext;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link FetchSourceContext} which defines how the _source
|
||||
* should be fetched.
|
||||
*/
|
||||
public FetchSourceContext fetchSource() {
|
||||
return fetchSourceContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a field to load and return (note, it must be stored) as part of
|
||||
* the search request. If none are specified, the source of the document
|
||||
* will be return.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder field(String field) {
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException("[field] must not be null: [" + name + "]");
|
||||
}
|
||||
if (fieldNames == null) {
|
||||
fieldNames = new ArrayList<>();
|
||||
}
|
||||
fieldNames.add(field);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the fields to load and return as part of the search request. If
|
||||
* none are specified, the source of the document will be returned.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fields(List<String> fields) {
|
||||
if (fields == null) {
|
||||
throw new IllegalArgumentException("[fields] must not be null: [" + name + "]");
|
||||
}
|
||||
this.fieldNames = fields;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets no fields to be loaded, resulting in only id and type to be
|
||||
* returned per field.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder noFields() {
|
||||
this.fieldNames = Collections.emptyList();
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the fields to load and return as part of the search request.
|
||||
*/
|
||||
public List<String> fields() {
|
||||
return fieldNames;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a field to load from the field data cache and return as part of
|
||||
* the search request.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fieldDataField(String fieldDataField) {
|
||||
if (fieldDataField == null) {
|
||||
throw new IllegalArgumentException("[fieldDataField] must not be null: [" + name + "]");
|
||||
}
|
||||
if (fieldDataFields == null) {
|
||||
fieldDataFields = new ArrayList<>();
|
||||
}
|
||||
fieldDataFields.add(fieldDataField);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds fields to load from the field data cache and return as part of
|
||||
* the search request.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder fieldDataFields(List<String> fieldDataFields) {
|
||||
if (fieldDataFields == null) {
|
||||
throw new IllegalArgumentException("[fieldDataFields] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.fieldDataFields == null) {
|
||||
this.fieldDataFields = new ArrayList<>();
|
||||
}
|
||||
this.fieldDataFields.addAll(fieldDataFields);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the field-data fields.
|
||||
*/
|
||||
public List<String> fieldDataFields() {
|
||||
return fieldDataFields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a script field under the given name with the provided script.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param script
|
||||
* The script
|
||||
*/
|
||||
public TopHitsAggregatorBuilder scriptField(String name, Script script) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("scriptField [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (script == null) {
|
||||
throw new IllegalArgumentException("scriptField [script] must not be null: [" + name + "]");
|
||||
}
|
||||
scriptField(name, script, false);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a script field under the given name with the provided script.
|
||||
*
|
||||
* @param name
|
||||
* The name of the field
|
||||
* @param script
|
||||
* The script
|
||||
*/
|
||||
public TopHitsAggregatorBuilder scriptField(String name, Script script, boolean ignoreFailure) {
|
||||
if (name == null) {
|
||||
throw new IllegalArgumentException("scriptField [name] must not be null: [" + name + "]");
|
||||
}
|
||||
if (script == null) {
|
||||
throw new IllegalArgumentException("scriptField [script] must not be null: [" + name + "]");
|
||||
}
|
||||
if (scriptFields == null) {
|
||||
scriptFields = new ArrayList<>();
|
||||
}
|
||||
scriptFields.add(new ScriptField(name, script, ignoreFailure));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TopHitsAggregatorBuilder scriptFields(List<ScriptField> scriptFields) {
|
||||
if (scriptFields == null) {
|
||||
throw new IllegalArgumentException("[scriptFields] must not be null: [" + name + "]");
|
||||
}
|
||||
if (this.scriptFields == null) {
|
||||
this.scriptFields = new ArrayList<>();
|
||||
}
|
||||
this.scriptFields.addAll(scriptFields);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the script fields.
|
||||
*/
|
||||
public List<ScriptField> scriptFields() {
|
||||
return scriptFields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should each {@link org.elasticsearch.search.SearchHit} be returned
|
||||
* with an explanation of the hit (ranking).
|
||||
*/
|
||||
public TopHitsAggregatorBuilder explain(boolean explain) {
|
||||
this.explain = explain;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether each search hit will be returned with an
|
||||
* explanation of the hit (ranking)
|
||||
*/
|
||||
public boolean explain() {
|
||||
return explain;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should each {@link org.elasticsearch.search.SearchHit} be returned
|
||||
* with a version associated with it.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder version(boolean version) {
|
||||
this.version = version;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the document's version will be included in the
|
||||
* search hits.
|
||||
*/
|
||||
public boolean version() {
|
||||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies when sorting, and controls if scores will be tracked as well.
|
||||
* Defaults to <tt>false</tt>.
|
||||
*/
|
||||
public TopHitsAggregatorBuilder trackScores(boolean trackScores) {
|
||||
this.trackScores = trackScores;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether scores will be tracked for this request.
|
||||
*/
|
||||
public boolean trackScores() {
|
||||
return trackScores;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopHitsAggregatorBuilder subAggregations(Builder subFactories) {
|
||||
throw new AggregationInitializationException("Aggregator [" + name + "] of type [" + type + "] cannot accept sub-aggregations");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TopHitsAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subfactoriesBuilder)
|
||||
throws IOException {
|
||||
return new TopHitsAggregatorFactory(name, type, from, size, explain, version, trackScores, sorts, highlightBuilder, fieldNames,
|
||||
fieldDataFields, scriptFields, fetchSourceContext, context, parent, subfactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(SearchSourceBuilder.FROM_FIELD.getPreferredName(), from);
|
||||
builder.field(SearchSourceBuilder.SIZE_FIELD.getPreferredName(), size);
|
||||
builder.field(SearchSourceBuilder.VERSION_FIELD.getPreferredName(), version);
|
||||
builder.field(SearchSourceBuilder.EXPLAIN_FIELD.getPreferredName(), explain);
|
||||
if (fetchSourceContext != null) {
|
||||
builder.field(SearchSourceBuilder._SOURCE_FIELD.getPreferredName(), fetchSourceContext);
|
||||
}
|
||||
if (fieldNames != null) {
|
||||
if (fieldNames.size() == 1) {
|
||||
builder.field(SearchSourceBuilder.FIELDS_FIELD.getPreferredName(), fieldNames.get(0));
|
||||
} else {
|
||||
builder.startArray(SearchSourceBuilder.FIELDS_FIELD.getPreferredName());
|
||||
for (String fieldName : fieldNames) {
|
||||
builder.value(fieldName);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
}
|
||||
if (fieldDataFields != null) {
|
||||
builder.startArray(SearchSourceBuilder.FIELDDATA_FIELDS_FIELD.getPreferredName());
|
||||
for (String fieldDataField : fieldDataFields) {
|
||||
builder.value(fieldDataField);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (scriptFields != null) {
|
||||
builder.startObject(SearchSourceBuilder.SCRIPT_FIELDS_FIELD.getPreferredName());
|
||||
for (ScriptField scriptField : scriptFields) {
|
||||
scriptField.toXContent(builder, params);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
if (sorts != null) {
|
||||
builder.startArray(SearchSourceBuilder.SORT_FIELD.getPreferredName());
|
||||
for (BytesReference sort : sorts) {
|
||||
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(sort);
|
||||
parser.nextToken();
|
||||
builder.copyCurrentStructure(parser);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (trackScores) {
|
||||
builder.field(SearchSourceBuilder.TRACK_SCORES_FIELD.getPreferredName(), true);
|
||||
}
|
||||
if (highlightBuilder != null) {
|
||||
this.highlightBuilder.toXContent(builder, params);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TopHitsAggregatorBuilder doReadFrom(String name, StreamInput in) throws IOException {
|
||||
TopHitsAggregatorBuilder factory = new TopHitsAggregatorBuilder(name);
|
||||
factory.explain = in.readBoolean();
|
||||
factory.fetchSourceContext = FetchSourceContext.optionalReadFromStream(in);
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<String> fieldDataFields = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
fieldDataFields.add(in.readString());
|
||||
}
|
||||
factory.fieldDataFields = fieldDataFields;
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<String> fieldNames = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
fieldNames.add(in.readString());
|
||||
}
|
||||
factory.fieldNames = fieldNames;
|
||||
}
|
||||
factory.from = in.readVInt();
|
||||
if (in.readBoolean()) {
|
||||
factory.highlightBuilder = HighlightBuilder.PROTOTYPE.readFrom(in);
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<ScriptField> scriptFields = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
scriptFields.add(ScriptField.PROTOTYPE.readFrom(in));
|
||||
}
|
||||
factory.scriptFields = scriptFields;
|
||||
}
|
||||
factory.size = in.readVInt();
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
List<BytesReference> sorts = new ArrayList<>();
|
||||
for (int i = 0; i < size; i++) {
|
||||
sorts.add(in.readBytesReference());
|
||||
}
|
||||
factory.sorts = sorts;
|
||||
}
|
||||
factory.trackScores = in.readBoolean();
|
||||
factory.version = in.readBoolean();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeBoolean(explain);
|
||||
FetchSourceContext.optionalWriteToStream(fetchSourceContext, out);
|
||||
boolean hasFieldDataFields = fieldDataFields != null;
|
||||
out.writeBoolean(hasFieldDataFields);
|
||||
if (hasFieldDataFields) {
|
||||
out.writeVInt(fieldDataFields.size());
|
||||
for (String fieldName : fieldDataFields) {
|
||||
out.writeString(fieldName);
|
||||
}
|
||||
}
|
||||
boolean hasFieldNames = fieldNames != null;
|
||||
out.writeBoolean(hasFieldNames);
|
||||
if (hasFieldNames) {
|
||||
out.writeVInt(fieldNames.size());
|
||||
for (String fieldName : fieldNames) {
|
||||
out.writeString(fieldName);
|
||||
}
|
||||
}
|
||||
out.writeVInt(from);
|
||||
boolean hasHighlighter = highlightBuilder != null;
|
||||
out.writeBoolean(hasHighlighter);
|
||||
if (hasHighlighter) {
|
||||
highlightBuilder.writeTo(out);
|
||||
}
|
||||
boolean hasScriptFields = scriptFields != null;
|
||||
out.writeBoolean(hasScriptFields);
|
||||
if (hasScriptFields) {
|
||||
out.writeVInt(scriptFields.size());
|
||||
for (ScriptField scriptField : scriptFields) {
|
||||
scriptField.writeTo(out);
|
||||
}
|
||||
}
|
||||
out.writeVInt(size);
|
||||
boolean hasSorts = sorts != null;
|
||||
out.writeBoolean(hasSorts);
|
||||
if (hasSorts) {
|
||||
out.writeVInt(sorts.size());
|
||||
for (BytesReference sort : sorts) {
|
||||
out.writeBytesReference(sort);
|
||||
}
|
||||
}
|
||||
out.writeBoolean(trackScores);
|
||||
out.writeBoolean(version);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return Objects.hash(explain, fetchSourceContext, fieldDataFields, fieldNames, from, highlightBuilder, scriptFields, size, sorts,
|
||||
trackScores, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(Object obj) {
|
||||
TopHitsAggregatorBuilder other = (TopHitsAggregatorBuilder) obj;
|
||||
return Objects.equals(explain, other.explain)
|
||||
&& Objects.equals(fetchSourceContext, other.fetchSourceContext)
|
||||
&& Objects.equals(fieldDataFields, other.fieldDataFields)
|
||||
&& Objects.equals(fieldNames, other.fieldNames)
|
||||
&& Objects.equals(from, other.from)
|
||||
&& Objects.equals(highlightBuilder, other.highlightBuilder)
|
||||
&& Objects.equals(scriptFields, other.scriptFields)
|
||||
&& Objects.equals(size, other.size)
|
||||
&& Objects.equals(sorts, other.sorts)
|
||||
&& Objects.equals(trackScores, other.trackScores)
|
||||
&& Objects.equals(version, other.version);
|
||||
}
|
||||
}
|
|
@ -45,9 +45,9 @@ public class TopHitsParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopHitsAggregator.TopHitsAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
public TopHitsAggregatorBuilder parse(String aggregationName, XContentParser parser, QueryParseContext context)
|
||||
throws IOException {
|
||||
TopHitsAggregator.TopHitsAggregatorBuilder factory = new TopHitsAggregator.TopHitsAggregatorBuilder(aggregationName);
|
||||
TopHitsAggregatorBuilder factory = new TopHitsAggregatorBuilder(aggregationName);
|
||||
XContentParser.Token token;
|
||||
String currentFieldName = null;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
|
@ -178,8 +178,8 @@ public class TopHitsParser implements Aggregator.Parser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopHitsAggregator.TopHitsAggregatorBuilder getFactoryPrototypes() {
|
||||
return TopHitsAggregator.TopHitsAggregatorBuilder.PROTOTYPE;
|
||||
public TopHitsAggregatorBuilder getFactoryPrototypes() {
|
||||
return TopHitsAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,27 +19,18 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.valuecount;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.LongArray;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -115,47 +106,4 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
Releasables.close(counts);
|
||||
}
|
||||
|
||||
public static class ValueCountAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource, ValueCountAggregatorBuilder> {
|
||||
|
||||
static final ValueCountAggregatorBuilder PROTOTYPE = new ValueCountAggregatorBuilder("", null);
|
||||
|
||||
public ValueCountAggregatorBuilder(String name, ValueType targetValueType) {
|
||||
super(name, InternalValueCount.TYPE, ValuesSourceType.ANY, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValueCountAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
|
||||
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
|
||||
return new ValueCountAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorBuilder<ValuesSource, ValueCountAggregatorBuilder> innerReadFrom(String name,
|
||||
ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in) {
|
||||
return new ValueCountAggregator.ValueCountAggregatorBuilder(name, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.valuecount;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ValueCountAggregatorBuilder
|
||||
extends ValuesSourceAggregatorBuilder.LeafOnly<ValuesSource, ValueCountAggregatorBuilder> {
|
||||
|
||||
static final ValueCountAggregatorBuilder PROTOTYPE = new ValueCountAggregatorBuilder("", null);
|
||||
|
||||
public ValueCountAggregatorBuilder(String name, ValueType targetValueType) {
|
||||
super(name, InternalValueCount.TYPE, ValuesSourceType.ANY, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValueCountAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
|
||||
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
|
||||
return new ValueCountAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorBuilder<ValuesSource, ValueCountAggregatorBuilder> innerReadFrom(String name,
|
||||
ValuesSourceType valuesSourceType, ValueType targetValueType, StreamInput in) {
|
||||
return new ValueCountAggregatorBuilder(name, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(Object obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -51,13 +51,13 @@ public class ValueCountParser extends AnyValuesSourceParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ValuesSourceAggregatorBuilder<ValuesSource, ValueCountAggregator.ValueCountAggregatorBuilder> createFactory(
|
||||
protected ValuesSourceAggregatorBuilder<ValuesSource, ValueCountAggregatorBuilder> createFactory(
|
||||
String aggregationName, ValuesSourceType valuesSourceType, ValueType targetValueType, Map<ParseField, Object> otherOptions) {
|
||||
return new ValueCountAggregator.ValueCountAggregatorBuilder(aggregationName, targetValueType);
|
||||
return new ValueCountAggregatorBuilder(aggregationName, targetValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueCountAggregator.ValueCountAggregatorBuilder getFactoryPrototypes() {
|
||||
return ValueCountAggregator.ValueCountAggregatorBuilder.PROTOTYPE;
|
||||
public ValueCountAggregatorBuilder getFactoryPrototypes() {
|
||||
return ValueCountAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,19 +20,19 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline;
|
||||
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg.AvgBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max.MaxBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min.MinBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.StatsBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ExtendedStatsBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum.SumBucketPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.serialdiff.SerialDiffPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg.AvgBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max.MaxBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min.MinBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.StatsBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ExtendedStatsBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum.SumBucketPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketselector.BucketSelectorPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.serialdiff.SerialDiffPipelineAggregatorBuilder;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -41,70 +41,70 @@ public final class PipelineAggregatorBuilders {
|
|||
private PipelineAggregatorBuilders() {
|
||||
}
|
||||
|
||||
public static final DerivativePipelineAggregator.DerivativePipelineAggregatorBuilder derivative(String name, String bucketsPath) {
|
||||
return new DerivativePipelineAggregator.DerivativePipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final DerivativePipelineAggregatorBuilder derivative(String name, String bucketsPath) {
|
||||
return new DerivativePipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder maxBucket(String name, String bucketsPath) {
|
||||
return new MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final MaxBucketPipelineAggregatorBuilder maxBucket(String name, String bucketsPath) {
|
||||
return new MaxBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder minBucket(String name, String bucketsPath) {
|
||||
return new MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final MinBucketPipelineAggregatorBuilder minBucket(String name, String bucketsPath) {
|
||||
return new MinBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder avgBucket(String name, String bucketsPath) {
|
||||
return new AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final AvgBucketPipelineAggregatorBuilder avgBucket(String name, String bucketsPath) {
|
||||
return new AvgBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder sumBucket(String name, String bucketsPath) {
|
||||
return new SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final SumBucketPipelineAggregatorBuilder sumBucket(String name, String bucketsPath) {
|
||||
return new SumBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder statsBucket(String name, String bucketsPath) {
|
||||
return new StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final StatsBucketPipelineAggregatorBuilder statsBucket(String name, String bucketsPath) {
|
||||
return new StatsBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder extendedStatsBucket(String name,
|
||||
public static final ExtendedStatsBucketPipelineAggregatorBuilder extendedStatsBucket(String name,
|
||||
String bucketsPath) {
|
||||
return new ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
return new ExtendedStatsBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder percentilesBucket(String name,
|
||||
public static final PercentilesBucketPipelineAggregatorBuilder percentilesBucket(String name,
|
||||
String bucketsPath) {
|
||||
return new PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
return new PercentilesBucketPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final MovAvgPipelineAggregator.MovAvgPipelineAggregatorBuilder movingAvg(String name, String bucketsPath) {
|
||||
return new MovAvgPipelineAggregator.MovAvgPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final MovAvgPipelineAggregatorBuilder movingAvg(String name, String bucketsPath) {
|
||||
return new MovAvgPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final BucketScriptPipelineAggregator.BucketScriptPipelineAggregatorBuilder bucketScript(String name,
|
||||
public static final BucketScriptPipelineAggregatorBuilder bucketScript(String name,
|
||||
Map<String, String> bucketsPathsMap, Script script) {
|
||||
return new BucketScriptPipelineAggregator.BucketScriptPipelineAggregatorBuilder(name, bucketsPathsMap, script);
|
||||
return new BucketScriptPipelineAggregatorBuilder(name, bucketsPathsMap, script);
|
||||
}
|
||||
|
||||
public static final BucketScriptPipelineAggregator.BucketScriptPipelineAggregatorBuilder bucketScript(String name, Script script,
|
||||
public static final BucketScriptPipelineAggregatorBuilder bucketScript(String name, Script script,
|
||||
String... bucketsPaths) {
|
||||
return new BucketScriptPipelineAggregator.BucketScriptPipelineAggregatorBuilder(name, script, bucketsPaths);
|
||||
return new BucketScriptPipelineAggregatorBuilder(name, script, bucketsPaths);
|
||||
}
|
||||
|
||||
public static final BucketSelectorPipelineAggregator.BucketSelectorPipelineAggregatorBuilder bucketSelector(String name,
|
||||
public static final BucketSelectorPipelineAggregatorBuilder bucketSelector(String name,
|
||||
Map<String, String> bucketsPathsMap, Script script) {
|
||||
return new BucketSelectorPipelineAggregator.BucketSelectorPipelineAggregatorBuilder(name, bucketsPathsMap, script);
|
||||
return new BucketSelectorPipelineAggregatorBuilder(name, bucketsPathsMap, script);
|
||||
}
|
||||
|
||||
public static final BucketSelectorPipelineAggregator.BucketSelectorPipelineAggregatorBuilder bucketSelector(String name, Script script,
|
||||
public static final BucketSelectorPipelineAggregatorBuilder bucketSelector(String name, Script script,
|
||||
String... bucketsPaths) {
|
||||
return new BucketSelectorPipelineAggregator.BucketSelectorPipelineAggregatorBuilder(name, script, bucketsPaths);
|
||||
return new BucketSelectorPipelineAggregatorBuilder(name, script, bucketsPaths);
|
||||
}
|
||||
|
||||
public static final CumulativeSumPipelineAggregator.CumulativeSumPipelineAggregatorBuilder cumulativeSum(String name,
|
||||
public static final CumulativeSumPipelineAggregatorBuilder cumulativeSum(String name,
|
||||
String bucketsPath) {
|
||||
return new CumulativeSumPipelineAggregator.CumulativeSumPipelineAggregatorBuilder(name, bucketsPath);
|
||||
return new CumulativeSumPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
|
||||
public static final SerialDiffPipelineAggregator.SerialDiffPipelineAggregatorBuilder diff(String name, String bucketsPath) {
|
||||
return new SerialDiffPipelineAggregator.SerialDiffPipelineAggregatorBuilder(name, bucketsPath);
|
||||
public static final SerialDiffPipelineAggregatorBuilder diff(String name, String bucketsPath) {
|
||||
return new SerialDiffPipelineAggregatorBuilder(name, bucketsPath);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,13 +30,13 @@ public class AvgBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
protected AvgBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
Map<String, Object> unparsedParams) {
|
||||
return new AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
return new AvgBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return AvgBucketPipelineAggregator.AvgBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public AvgBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return AvgBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,17 +20,12 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
|
@ -89,57 +84,4 @@ public class AvgBucketPipelineAggregator extends BucketMetricsPipelineAggregator
|
|||
return new InternalSimpleValue(name(), avgValue, formatter, pipelineAggregators, metadata);
|
||||
}
|
||||
|
||||
public static class AvgBucketPipelineAggregatorBuilder extends BucketMetricsPipelineAggregatorBuilder<AvgBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final AvgBucketPipelineAggregatorBuilder PROTOTYPE = new AvgBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public AvgBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private AvgBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new AvgBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in) throws IOException {
|
||||
return new AvgBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<AvgBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.avg;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class AvgBucketPipelineAggregatorBuilder extends BucketMetricsPipelineAggregatorBuilder<AvgBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final AvgBucketPipelineAggregatorBuilder PROTOTYPE = new AvgBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public AvgBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private AvgBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, AvgBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new AvgBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AvgBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in) throws IOException {
|
||||
return new AvgBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<AvgBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -31,14 +31,14 @@ public class MaxBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
protected MaxBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
Map<String, Object> unparsedParams) {
|
||||
return new MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
return new MaxBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return MaxBucketPipelineAggregator.MaxBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public MaxBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return MaxBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,16 +20,11 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
@ -96,59 +91,4 @@ public class MaxBucketPipelineAggregator extends BucketMetricsPipelineAggregator
|
|||
return new InternalBucketMetricValue(name(), keys, maxValue, formatter, Collections.emptyList(), metaData());
|
||||
}
|
||||
|
||||
public static class MaxBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<MaxBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final MaxBucketPipelineAggregatorBuilder PROTOTYPE = new MaxBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public MaxBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private MaxBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new MaxBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new MaxBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<MaxBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.max;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class MaxBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<MaxBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final MaxBucketPipelineAggregatorBuilder PROTOTYPE = new MaxBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public MaxBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private MaxBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, MaxBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new MaxBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MaxBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new MaxBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<MaxBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -31,14 +31,14 @@ public class MinBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
protected MinBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
Map<String, Object> unparsedParams) {
|
||||
return new MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
return new MinBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return MinBucketPipelineAggregator.MinBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public MinBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return MinBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,16 +20,11 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
@ -95,61 +90,6 @@ public class MinBucketPipelineAggregator extends BucketMetricsPipelineAggregator
|
|||
java.util.Map<String, Object> metadata) {
|
||||
String[] keys = minBucketKeys.toArray(new String[minBucketKeys.size()]);
|
||||
return new InternalBucketMetricValue(name(), keys, minValue, formatter, Collections.emptyList(), metaData());
|
||||
};
|
||||
|
||||
public static class MinBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<MinBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final MinBucketPipelineAggregatorBuilder PROTOTYPE = new MinBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public MinBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private MinBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new MinBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new MinBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<MinBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.min;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class MinBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<MinBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final MinBucketPipelineAggregatorBuilder PROTOTYPE = new MinBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public MinBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private MinBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, MinBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new MinBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MinBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new MinBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<MinBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -37,7 +37,7 @@ public class PercentilesBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
protected PercentilesBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
String bucketsPath, Map<String, Object> unparsedParams) throws ParseException {
|
||||
|
||||
double[] percents = null;
|
||||
|
@ -63,8 +63,8 @@ public class PercentilesBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
}
|
||||
|
||||
PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder factory = new
|
||||
PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
PercentilesBucketPipelineAggregatorBuilder factory = new
|
||||
PercentilesBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
if (percents != null) {
|
||||
factory.percents(percents);
|
||||
}
|
||||
|
@ -72,7 +72,7 @@ public class PercentilesBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return PercentilesBucketPipelineAggregator.PercentilesBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public PercentilesBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return PercentilesBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,25 +22,19 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile;
|
|||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class PercentilesBucketPipelineAggregator extends BucketMetricsPipelineAggregator {
|
||||
|
||||
|
@ -124,98 +118,4 @@ public class PercentilesBucketPipelineAggregator extends BucketMetricsPipelineAg
|
|||
out.writeDoubleArray(percents);
|
||||
}
|
||||
|
||||
public static class PercentilesBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<PercentilesBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final PercentilesBucketPipelineAggregatorBuilder PROTOTYPE = new PercentilesBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
private double[] percents = new double[] { 1.0, 5.0, 25.0, 50.0, 75.0, 95.0, 99.0 };
|
||||
|
||||
public PercentilesBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private PercentilesBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the percentages to calculate percentiles for in this aggregation
|
||||
*/
|
||||
public double[] percents() {
|
||||
return percents;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the percentages to calculate percentiles for in this aggregation
|
||||
*/
|
||||
public PercentilesBucketPipelineAggregatorBuilder percents(double[] percents) {
|
||||
if (percents == null) {
|
||||
throw new IllegalArgumentException("[percents] must not be null: [" + name + "]");
|
||||
}
|
||||
for (Double p : percents) {
|
||||
if (p == null || p < 0.0 || p > 100.0) {
|
||||
throw new IllegalArgumentException(PercentilesBucketParser.PERCENTS.getPreferredName()
|
||||
+ " must only contain non-null doubles from 0.0-100.0 inclusive");
|
||||
}
|
||||
}
|
||||
this.percents = percents;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new PercentilesBucketPipelineAggregator(name, percents, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
|
||||
for (Double p : percents) {
|
||||
if (p == null || p < 0.0 || p > 100.0) {
|
||||
throw new IllegalStateException(PercentilesBucketParser.PERCENTS.getPreferredName()
|
||||
+ " must only contain non-null doubles from 0.0-100.0 inclusive");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
if (percents != null) {
|
||||
builder.field(PercentilesBucketParser.PERCENTS.getPreferredName(), percents);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PercentilesBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
PercentilesBucketPipelineAggregatorBuilder factory = new PercentilesBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
factory.percents = in.readDoubleArray();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDoubleArray(percents);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Arrays.hashCode(percents);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<PercentilesBucketPipelineAggregatorBuilder> obj) {
|
||||
PercentilesBucketPipelineAggregatorBuilder other = (PercentilesBucketPipelineAggregatorBuilder) obj;
|
||||
return Objects.deepEquals(percents, other.percents);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,128 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.percentile;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class PercentilesBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<PercentilesBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final PercentilesBucketPipelineAggregatorBuilder PROTOTYPE = new PercentilesBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
private double[] percents = new double[] { 1.0, 5.0, 25.0, 50.0, 75.0, 95.0, 99.0 };
|
||||
|
||||
public PercentilesBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private PercentilesBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, PercentilesBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the percentages to calculate percentiles for in this aggregation
|
||||
*/
|
||||
public double[] percents() {
|
||||
return percents;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the percentages to calculate percentiles for in this aggregation
|
||||
*/
|
||||
public PercentilesBucketPipelineAggregatorBuilder percents(double[] percents) {
|
||||
if (percents == null) {
|
||||
throw new IllegalArgumentException("[percents] must not be null: [" + name + "]");
|
||||
}
|
||||
for (Double p : percents) {
|
||||
if (p == null || p < 0.0 || p > 100.0) {
|
||||
throw new IllegalArgumentException(PercentilesBucketParser.PERCENTS.getPreferredName()
|
||||
+ " must only contain non-null doubles from 0.0-100.0 inclusive");
|
||||
}
|
||||
}
|
||||
this.percents = percents;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new PercentilesBucketPipelineAggregator(name, percents, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
|
||||
for (Double p : percents) {
|
||||
if (p == null || p < 0.0 || p > 100.0) {
|
||||
throw new IllegalStateException(PercentilesBucketParser.PERCENTS.getPreferredName()
|
||||
+ " must only contain non-null doubles from 0.0-100.0 inclusive");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
if (percents != null) {
|
||||
builder.field(PercentilesBucketParser.PERCENTS.getPreferredName(), percents);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PercentilesBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
PercentilesBucketPipelineAggregatorBuilder factory = new PercentilesBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
factory.percents = in.readDoubleArray();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDoubleArray(percents);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Arrays.hashCode(percents);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<PercentilesBucketPipelineAggregatorBuilder> obj) {
|
||||
PercentilesBucketPipelineAggregatorBuilder other = (PercentilesBucketPipelineAggregatorBuilder) obj;
|
||||
return Objects.deepEquals(percents, other.percents);
|
||||
}
|
||||
|
||||
}
|
|
@ -30,13 +30,13 @@ public class StatsBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
protected StatsBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
String bucketsPath, Map<String, Object> unparsedParams) {
|
||||
return new StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
return new StatsBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return StatsBucketPipelineAggregator.StatsBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public StatsBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return StatsBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,16 +20,11 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
|
@ -95,59 +90,4 @@ public class StatsBucketPipelineAggregator extends BucketMetricsPipelineAggregat
|
|||
return new InternalStatsBucket(name(), count, sum, min, max, formatter, pipelineAggregators, metadata);
|
||||
}
|
||||
|
||||
public static class StatsBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<StatsBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final StatsBucketPipelineAggregatorBuilder PROTOTYPE = new StatsBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public StatsBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private StatsBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new StatsBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new StatsBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<StatsBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.stats;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.Parser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class StatsBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<StatsBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final StatsBucketPipelineAggregatorBuilder PROTOTYPE = new StatsBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public StatsBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private StatsBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, StatsBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new StatsBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected StatsBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new StatsBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<StatsBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -34,7 +34,7 @@ public class ExtendedStatsBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
protected ExtendedStatsBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName,
|
||||
String bucketsPath, Map<String, Object> unparsedParams) throws ParseException {
|
||||
|
||||
Double sigma = null;
|
||||
|
@ -49,8 +49,8 @@ public class ExtendedStatsBucketParser extends BucketMetricsParser {
|
|||
+ param.getClass().getSimpleName() + "` provided instead", 0);
|
||||
}
|
||||
}
|
||||
ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder factory =
|
||||
new ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
ExtendedStatsBucketPipelineAggregatorBuilder factory =
|
||||
new ExtendedStatsBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
if (sigma != null) {
|
||||
factory.sigma(sigma);
|
||||
}
|
||||
|
@ -58,7 +58,7 @@ public class ExtendedStatsBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return ExtendedStatsBucketPipelineAggregator.ExtendedStatsBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public ExtendedStatsBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return ExtendedStatsBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,23 +20,17 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ExtendedStatsBucketPipelineAggregator extends BucketMetricsPipelineAggregator {
|
||||
|
||||
|
@ -101,90 +95,4 @@ public class ExtendedStatsBucketPipelineAggregator extends BucketMetricsPipeline
|
|||
return new InternalExtendedStatsBucket(name(), count, sum, min, max, sumOfSqrs, sigma, formatter, pipelineAggregators, metadata);
|
||||
}
|
||||
|
||||
public static class ExtendedStatsBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<ExtendedStatsBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final ExtendedStatsBucketPipelineAggregatorBuilder PROTOTYPE = new ExtendedStatsBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
private double sigma = 2.0;
|
||||
|
||||
public ExtendedStatsBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private ExtendedStatsBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the value of sigma to use when calculating the standard deviation
|
||||
* bounds
|
||||
*/
|
||||
public ExtendedStatsBucketPipelineAggregatorBuilder sigma(double sigma) {
|
||||
if (sigma < 0.0) {
|
||||
throw new IllegalArgumentException(ExtendedStatsBucketParser.SIGMA.getPreferredName() + " must be a non-negative double");
|
||||
}
|
||||
this.sigma = sigma;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value of sigma to use when calculating the standard deviation
|
||||
* bounds
|
||||
*/
|
||||
public double sigma() {
|
||||
return sigma;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new ExtendedStatsBucketPipelineAggregator(name, bucketsPaths, sigma, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
|
||||
if (sigma < 0.0 ) {
|
||||
throw new IllegalStateException(ExtendedStatsBucketParser.SIGMA.getPreferredName()
|
||||
+ " must be a non-negative double");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ExtendedStatsBucketParser.SIGMA.getPreferredName(), sigma);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
ExtendedStatsBucketPipelineAggregatorBuilder factory = new ExtendedStatsBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
factory.sigma = in.readDouble();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<ExtendedStatsBucketPipelineAggregatorBuilder> obj) {
|
||||
ExtendedStatsBucketPipelineAggregatorBuilder other = (ExtendedStatsBucketPipelineAggregatorBuilder) obj;
|
||||
return Objects.equals(sigma, other.sigma);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,120 @@
|
|||
/*
|
||||
* 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.pipeline.bucketmetrics.stats.extended;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.Parser;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ExtendedStatsBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<ExtendedStatsBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final ExtendedStatsBucketPipelineAggregatorBuilder PROTOTYPE = new ExtendedStatsBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
private double sigma = 2.0;
|
||||
|
||||
public ExtendedStatsBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private ExtendedStatsBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, ExtendedStatsBucketPipelineAggregator.TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the value of sigma to use when calculating the standard deviation
|
||||
* bounds
|
||||
*/
|
||||
public ExtendedStatsBucketPipelineAggregatorBuilder sigma(double sigma) {
|
||||
if (sigma < 0.0) {
|
||||
throw new IllegalArgumentException(ExtendedStatsBucketParser.SIGMA.getPreferredName() + " must be a non-negative double");
|
||||
}
|
||||
this.sigma = sigma;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value of sigma to use when calculating the standard deviation
|
||||
* bounds
|
||||
*/
|
||||
public double sigma() {
|
||||
return sigma;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new ExtendedStatsBucketPipelineAggregator(name, bucketsPaths, sigma, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
|
||||
if (sigma < 0.0 ) {
|
||||
throw new IllegalStateException(ExtendedStatsBucketParser.SIGMA.getPreferredName()
|
||||
+ " must be a non-negative double");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ExtendedStatsBucketParser.SIGMA.getPreferredName(), sigma);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExtendedStatsBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
ExtendedStatsBucketPipelineAggregatorBuilder factory = new ExtendedStatsBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
factory.sigma = in.readDouble();
|
||||
return factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return Objects.hash(sigma);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<ExtendedStatsBucketPipelineAggregatorBuilder> obj) {
|
||||
ExtendedStatsBucketPipelineAggregatorBuilder other = (ExtendedStatsBucketPipelineAggregatorBuilder) obj;
|
||||
return Objects.equals(sigma, other.sigma);
|
||||
}
|
||||
|
||||
}
|
|
@ -30,13 +30,13 @@ public class SumBucketParser extends BucketMetricsParser {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
protected SumBucketPipelineAggregatorBuilder buildFactory(String pipelineAggregatorName, String bucketsPath,
|
||||
Map<String, Object> unparsedParams) {
|
||||
return new SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
return new SumBucketPipelineAggregatorBuilder(pipelineAggregatorName, bucketsPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return SumBucketPipelineAggregator.SumBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
public SumBucketPipelineAggregatorBuilder getFactoryPrototype() {
|
||||
return SumBucketPipelineAggregatorBuilder.PROTOTYPE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,17 +20,12 @@
|
|||
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
|
||||
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
|
||||
|
||||
|
@ -85,59 +80,4 @@ public class SumBucketPipelineAggregator extends BucketMetricsPipelineAggregator
|
|||
return new InternalSimpleValue(name(), sum, formatter, pipelineAggregators, metadata);
|
||||
}
|
||||
|
||||
public static class SumBucketPipelineAggregatorBuilder
|
||||
extends BucketMetricsPipelineAggregatorBuilder<SumBucketPipelineAggregatorBuilder> {
|
||||
|
||||
static final SumBucketPipelineAggregatorBuilder PROTOTYPE = new SumBucketPipelineAggregatorBuilder("", "");
|
||||
|
||||
public SumBucketPipelineAggregatorBuilder(String name, String bucketsPath) {
|
||||
this(name, new String[] { bucketsPath });
|
||||
}
|
||||
|
||||
private SumBucketPipelineAggregatorBuilder(String name, String[] bucketsPaths) {
|
||||
super(name, TYPE.name(), bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PipelineAggregator createInternal(Map<String, Object> metaData) throws IOException {
|
||||
return new SumBucketPipelineAggregator(name, bucketsPaths, gapPolicy(), formatter(), metaData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doValidate(AggregatorFactory<?> parent, AggregatorFactory<?>[] aggFactories,
|
||||
List<PipelineAggregatorBuilder<?>> pipelineAggregatorFactories) {
|
||||
if (bucketsPaths.length != 1) {
|
||||
throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName()
|
||||
+ " must contain a single entry for aggregation [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SumBucketPipelineAggregatorBuilder innerReadFrom(String name, String[] bucketsPaths, StreamInput in)
|
||||
throws IOException {
|
||||
return new SumBucketPipelineAggregatorBuilder(name, bucketsPaths);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
// Do nothing, no extra state to write to stream
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int innerHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean innerEquals(BucketMetricsPipelineAggregatorBuilder<SumBucketPipelineAggregatorBuilder> other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue