Remove AggregationContext. (#22124)

This class is just a wrapper around `SearchContext`, so let's use
`SearchContext` directly. The change is mechanical, except the
`ValuesSourceConfig` class, where I moved the logic to get a `ValuesSource`
given a config.
This commit is contained in:
Adrien Grand 2016-12-13 09:09:40 +01:00 committed by GitHub
parent a4e8b5d952
commit 049fd3991c
141 changed files with 584 additions and 651 deletions

View File

@ -55,7 +55,6 @@ import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.dfs.DfsPhase;
import org.elasticsearch.search.dfs.DfsSearchResult;
@ -727,8 +726,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
context.terminateAfter(source.terminateAfter());
if (source.aggregations() != null) {
try {
AggregationContext aggContext = new AggregationContext(context);
AggregatorFactories factories = source.aggregations().build(aggContext, null);
AggregatorFactories factories = source.aggregations().build(context, null);
factories.validate();
context.aggregations(new SearchContextAggregations(factories));
} catch (IOException e) {

View File

@ -22,7 +22,7 @@ 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.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -122,12 +122,12 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
}
@Override
public final AggregatorFactory<?> build(AggregationContext context, AggregatorFactory<?> parent) throws IOException {
public final AggregatorFactory<?> build(SearchContext context, AggregatorFactory<?> parent) throws IOException {
AggregatorFactory<?> factory = doBuild(context, parent, factoriesBuilder);
return factory;
}
protected abstract AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent,
protected abstract AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subfactoriesBuilder) throws IOException;
@Override

View File

@ -24,7 +24,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -63,7 +63,7 @@ public abstract class AggregationBuilder
}
/** Internal: build an {@link AggregatorFactory} based on the configuration of this builder. */
protected abstract AggregatorFactory<?> build(AggregationContext context, AggregatorFactory<?> parent) throws IOException;
protected abstract AggregatorFactory<?> build(SearchContext context, AggregatorFactory<?> parent) throws IOException;
/** Associate metadata with this {@link AggregationBuilder}. */
public abstract AggregationBuilder setMetaData(Map<String, Object> metaData);

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.SearchPhase;
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.profile.query.CollectorResult;
import org.elasticsearch.search.profile.query.InternalProfileCollector;
@ -51,9 +50,6 @@ public class AggregationPhase implements SearchPhase {
@Override
public void preProcess(SearchContext context) {
if (context.aggregations() != null) {
AggregationContext aggregationContext = new AggregationContext(context);
context.aggregations().aggregationContext(aggregationContext);
List<Aggregator> collectors = new ArrayList<>();
Aggregator[] aggregators;
try {

View File

@ -28,7 +28,7 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -77,9 +77,9 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
public abstract String name();
/**
* Return the {@link AggregationContext} attached with this {@link Aggregator}.
* Return the {@link SearchContext} attached with this {@link Aggregator}.
*/
public abstract AggregationContext context();
public abstract SearchContext context();
/**
* Return the parent aggregator.

View File

@ -25,7 +25,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.aggregations.bucket.BestBucketsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import org.elasticsearch.search.query.QueryPhaseExecutionException;
@ -45,7 +45,7 @@ public abstract class AggregatorBase extends Aggregator {
protected final String name;
protected final Aggregator parent;
protected final AggregationContext context;
protected final SearchContext context;
private final Map<String, Object> metaData;
protected final Aggregator[] subAggregators;
@ -66,7 +66,7 @@ public abstract class AggregatorBase extends Aggregator {
* @param parent The parent aggregator (may be {@code null} for top level aggregators)
* @param metaData The metaData associated with this aggregator
*/
protected AggregatorBase(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent,
protected AggregatorBase(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
this.name = name;
this.pipelineAggregators = pipelineAggregators;
@ -76,11 +76,11 @@ public abstract class AggregatorBase extends Aggregator {
this.breakerService = context.bigArrays().breakerService();
assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
this.subAggregators = factories.createSubAggregators(this);
context.searchContext().addReleasable(this, Lifetime.PHASE);
context.addReleasable(this, Lifetime.PHASE);
// Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
collectableSubAggregators = new BucketCollector() {
void badState(){
throw new QueryPhaseExecutionException(AggregatorBase.this.context.searchContext(),
throw new QueryPhaseExecutionException(AggregatorBase.this.context,
"preCollection not called on new Aggregator before use", null);
}
@Override
@ -245,7 +245,7 @@ public abstract class AggregatorBase extends Aggregator {
* @return The current aggregation context.
*/
@Override
public AggregationContext context() {
public SearchContext context() {
return context;
}

View File

@ -24,9 +24,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.profile.Profilers;
import org.elasticsearch.search.profile.aggregation.ProfilingAggregator;
@ -81,7 +81,7 @@ public class AggregatorFactories {
// aggs
final boolean collectsFromSingleBucket = false;
Aggregator factory = factories[i].create(parent, collectsFromSingleBucket);
Profilers profilers = factory.context().searchContext().getProfilers();
Profilers profilers = factory.context().getProfilers();
if (profilers != null) {
factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler());
}
@ -97,7 +97,7 @@ public class AggregatorFactories {
// top-level aggs only get called with bucket 0
final boolean collectsFromSingleBucket = true;
Aggregator factory = factories[i].create(null, collectsFromSingleBucket);
Profilers profilers = factory.context().searchContext().getProfilers();
Profilers profilers = factory.context().getProfilers();
if (profilers != null) {
factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler());
}
@ -193,7 +193,7 @@ public class AggregatorFactories {
return this;
}
public AggregatorFactories build(AggregationContext context, AggregatorFactory<?> parent) throws IOException {
public AggregatorFactories build(SearchContext context, AggregatorFactory<?> parent) throws IOException {
if (aggregationBuilders.isEmpty() && pipelineAggregatorBuilders.isEmpty()) {
return EMPTY;
}

View File

@ -26,8 +26,9 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -42,13 +43,13 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
ObjectArray<Aggregator> aggregators;
ObjectArray<LeafBucketCollector> collectors;
MultiBucketAggregatorWrapper(BigArrays bigArrays, AggregationContext context, Aggregator parent, AggregatorFactory<?> factory,
MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context, Aggregator parent, AggregatorFactory<?> factory,
Aggregator first) {
this.bigArrays = bigArrays;
this.parent = parent;
this.factory = factory;
this.first = first;
context.searchContext().addReleasable(this, Lifetime.PHASE);
context.addReleasable(this, Lifetime.PHASE);
aggregators = bigArrays.newObjectArray(1);
aggregators.set(0, first);
collectors = bigArrays.newObjectArray(1);
@ -64,7 +65,7 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
}
@Override
public AggregationContext context() {
public SearchContext context() {
return first.context();
}
@ -166,7 +167,7 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
protected final AggregatorFactory<?> parent;
protected final AggregatorFactories factories;
protected final Map<String, Object> metaData;
protected final AggregationContext context;
protected final SearchContext context;
/**
* Constructs a new aggregator factory.
@ -178,7 +179,7 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
* @throws IOException
* if an error occurs creating the factory
*/
public AggregatorFactory(String name, Type type, AggregationContext context, AggregatorFactory<?> parent,
public AggregatorFactory(String name, Type type, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
this.name = name;
this.type = type;
@ -238,7 +239,7 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
* {@link Aggregator}s that only know how to collect bucket <tt>0</tt>, this
* returns an aggregator that can collect any bucket.
*/
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory<?> factory, final AggregationContext context,
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory<?> factory, final SearchContext context,
final Aggregator parent) throws IOException {
final Aggregator first = factory.create(parent, true);
final BigArrays bigArrays = context.bigArrays();

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -33,12 +33,12 @@ import java.util.Map;
*/
public abstract class NonCollectingAggregator extends AggregatorBase {
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories,
protected NonCollectingAggregator(String name, SearchContext context, Aggregator parent, AggregatorFactories subFactories,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, subFactories, context, parent, pipelineAggregators, metaData);
}
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent,
protected NonCollectingAggregator(String name, SearchContext context, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
this(name, context, parent, AggregatorFactories.EMPTY, pipelineAggregators, metaData);
}

View File

@ -18,8 +18,6 @@
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.search.aggregations.support.AggregationContext;
/**
* The aggregation context that is part of the search context.
*/
@ -27,7 +25,6 @@ public class SearchContextAggregations {
private final AggregatorFactories factories;
private Aggregator[] aggregators;
private AggregationContext aggregationContext;
/**
* Creates a new aggregation context with the parsed aggregator factories
@ -44,14 +41,6 @@ public class SearchContextAggregations {
return aggregators;
}
public AggregationContext aggregationContext() {
return aggregationContext;
}
public void aggregationContext(AggregationContext aggregationContext) {
this.aggregationContext = aggregationContext;
}
/**
* Registers all the created aggregators (top level aggregators) for the search execution context.
*

View File

@ -31,7 +31,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.BucketCollector;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -58,7 +58,7 @@ public class BestBucketsDeferringCollector extends DeferringBucketCollector {
final List<Entry> entries = new ArrayList<>();
BucketCollector collector;
final AggregationContext aggContext;
final SearchContext searchContext;
LeafReaderContext context;
PackedLongValues.Builder docDeltas;
PackedLongValues.Builder buckets;
@ -67,8 +67,8 @@ public class BestBucketsDeferringCollector extends DeferringBucketCollector {
LongHash selectedBuckets;
/** Sole constructor. */
public BestBucketsDeferringCollector(AggregationContext context) {
this.aggContext = context;
public BestBucketsDeferringCollector(SearchContext context) {
this.searchContext = context;
}
@Override
@ -147,8 +147,8 @@ public class BestBucketsDeferringCollector extends DeferringBucketCollector {
boolean needsScores = collector.needsScores();
Weight weight = null;
if (needsScores) {
weight = aggContext.searchContext().searcher()
.createNormalizedWeight(aggContext.searchContext().query(), true);
weight = searchContext.searcher()
.createNormalizedWeight(searchContext.query(), true);
}
for (Entry entry : entries) {
final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context);

View File

@ -28,7 +28,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -40,7 +40,7 @@ public abstract class BucketsAggregator extends AggregatorBase {
private final BigArrays bigArrays;
private IntArray docCounts;
public BucketsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent,
public BucketsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, pipelineAggregators, metaData);
bigArrays = context.bigArrays();

View File

@ -24,7 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.BucketCollector;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -82,7 +82,7 @@ public abstract class DeferringBucketCollector extends BucketCollector {
}
@Override
public AggregationContext context() {
public SearchContext context() {
return in.context();
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -33,7 +33,7 @@ import java.util.Map;
public abstract class SingleBucketAggregator extends BucketsAggregator {
protected SingleBucketAggregator(String name, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent,
SearchContext aggregationContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
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.Bytes.ParentChild;
@ -40,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuil
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.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -81,16 +81,16 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
}
@Override
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(AggregationContext context,
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(SearchContext 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) {
protected ValuesSourceConfig<ParentChild> resolveConfig(SearchContext context) {
ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
DocumentMapper childDocMapper = aggregationContext.searchContext().mapperService().documentMapper(childType);
DocumentMapper childDocMapper = context.mapperService().documentMapper(childType);
if (childDocMapper != null) {
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
@ -98,11 +98,11 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
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);
DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
if (parentDocMapper != null) {
parentFilter = parentDocMapper.typeFilter();
childFilter = childDocMapper.typeFilter();
ParentChildIndexFieldData parentChildIndexFieldData = aggregationContext.searchContext().fieldData()
ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData()
.getForField(parentFieldMapper.fieldType());
config.fieldContext(new FieldContext(parentFieldMapper.fieldType().name(), parentChildIndexFieldData,
parentFieldMapper.fieldType()));

View File

@ -27,11 +27,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
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.Bytes.ParentChild;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -45,7 +45,7 @@ public class ChildrenAggregatorFactory
private final Query childFilter;
public ChildrenAggregatorFactory(String name, Type type, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
Query parentFilter, AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Query parentFilter, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.parentType = parentType;
@ -70,7 +70,7 @@ public class ChildrenAggregatorFactory
protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
long maxOrd = valuesSource.globalMaxOrd(context.searchContext().searcher(), parentType);
long maxOrd = valuesSource.globalMaxOrd(context.searcher(), parentType);
return new ParentToChildrenAggregator(name, factories, context, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd,
pipelineAggregators, metaData);
}

View File

@ -38,8 +38,8 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -67,18 +67,18 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
private final LongObjectPagedHashMap<long[]> parentOrdToOtherBuckets;
private boolean multipleBucketsPerParentOrd = false;
public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext,
public ParentToChildrenAggregator(String name, AggregatorFactories factories, SearchContext context,
Aggregator parent, String parentType, Query childFilter, Query parentFilter,
ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource,
long maxOrd, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
this.parentType = parentType;
// these two filters are cached in the parser
this.childFilter = aggregationContext.searchContext().searcher().createNormalizedWeight(childFilter, false);
this.parentFilter = aggregationContext.searchContext().searcher().createNormalizedWeight(parentFilter, false);
this.parentOrdToBuckets = aggregationContext.bigArrays().newLongArray(maxOrd, false);
this.childFilter = context.searcher().createNormalizedWeight(childFilter, false);
this.parentFilter = context.searcher().createNormalizedWeight(parentFilter, false);
this.parentOrdToBuckets = context.bigArrays().newLongArray(maxOrd, false);
this.parentOrdToBuckets.fill(0, maxOrd, -1);
this.parentOrdToOtherBuckets = new LongObjectPagedHashMap<>(aggregationContext.bigArrays());
this.parentOrdToOtherBuckets = new LongObjectPagedHashMap<>(context.bigArrays());
this.valuesSource = valuesSource;
}
@ -132,7 +132,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator {
@Override
protected void doPostCollection() throws IOException {
IndexReader indexReader = context().searchContext().searcher().getIndexReader();
IndexReader indexReader = context().searcher().getIndexReader();
for (LeafReaderContext ctx : indexReader.leaves()) {
Scorer childDocsScorer = childFilter.scorer(ctx);
if (childDocsScorer == null) {

View File

@ -28,7 +28,7 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -69,10 +69,10 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
}
@Override
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent,
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
// TODO this sucks we need a rewrite phase for aggregations too
final QueryBuilder rewrittenFilter = QueryBuilder.rewriteQuery(filter, context.searchContext().getQueryShardContext());
final QueryBuilder rewrittenFilter = QueryBuilder.rewriteQuery(filter, context.getQueryShardContext());
return new FilterAggregatorFactory(name, type, rewrittenFilter, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -45,10 +45,10 @@ public class FilterAggregator extends SingleBucketAggregator {
public FilterAggregator(String name,
Weight filter,
AggregatorFactories factories,
AggregationContext aggregationContext,
SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
this.filter = filter;
}

View File

@ -28,7 +28,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -38,11 +38,11 @@ public class FilterAggregatorFactory extends AggregatorFactory<FilterAggregatorF
private final Weight weight;
public FilterAggregatorFactory(String name, Type type, QueryBuilder filterBuilder, AggregationContext context,
public FilterAggregatorFactory(String name, Type type, QueryBuilder filterBuilder, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactoriesBuilder, metaData);
IndexSearcher contextSearcher = context.searchContext().searcher();
Query filter = filterBuilder.toQuery(context.searchContext().getQueryShardContext());
IndexSearcher contextSearcher = context.searcher();
Query filter = filterBuilder.toQuery(context.getQueryShardContext());
weight = contextSearcher.createNormalizedWeight(filter, false);
}

View File

@ -32,7 +32,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -169,12 +169,12 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
}
@Override
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
List<KeyedFilter> rewrittenFilters = new ArrayList<>();
for(KeyedFilter kf : filters) {
rewrittenFilters.add(new KeyedFilter(kf.key(), QueryBuilder.rewriteQuery(kf.filter(),
context.searchContext().getQueryShardContext())));
context.getQueryShardContext())));
}
return new FiltersAggregatorFactory(name, type, rewrittenFilters, keyed, otherBucket, otherBucketKey, context, parent,
subFactoriesBuilder, metaData);

View File

@ -29,7 +29,6 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -39,7 +38,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -123,10 +122,10 @@ public class FiltersAggregator extends BucketsAggregator {
private final int totalNumKeys;
public FiltersAggregator(String name, AggregatorFactories factories, String[] keys, Weight[] filters, boolean keyed, String otherBucketKey,
AggregationContext aggregationContext,
SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
this.keyed = keyed;
this.keys = keys;
this.filters = filters;

View File

@ -28,7 +28,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -43,19 +43,19 @@ public class FiltersAggregatorFactory extends AggregatorFactory<FiltersAggregato
private final String otherBucketKey;
public FiltersAggregatorFactory(String name, Type type, List<KeyedFilter> filters, boolean keyed, boolean otherBucket,
String otherBucketKey, AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
String otherBucketKey, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
this.keyed = keyed;
this.otherBucket = otherBucket;
this.otherBucketKey = otherBucketKey;
IndexSearcher contextSearcher = context.searchContext().searcher();
IndexSearcher contextSearcher = context.searcher();
weights = new Weight[filters.size()];
keys = new String[filters.size()];
for (int i = 0; i < filters.size(); ++i) {
KeyedFilter keyedFilter = filters.get(i);
this.keys[i] = keyedFilter.key();
Query filter = keyedFilter.filter().toFilter(context.searchContext().getQueryShardContext());
Query filter = keyedFilter.filter().toFilter(context.getQueryShardContext());
this.weights[i] = contextSearcher.createNormalizedWeight(filter, false);
}
}

View File

@ -37,7 +37,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
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.ValuesSourceAggregationBuilder;
@ -45,6 +44,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -129,7 +129,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
int shardSize = this.shardSize;
@ -139,7 +139,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
if (shardSize < 0) {
// Use default heuristic to avoid any wrong-ranking caused by
// distributed counting
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.searchContext().numberOfShards());
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.numberOfShards());
}
if (requiredSize <= 0 || shardSize <= 0) {

View File

@ -29,7 +29,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -50,7 +50,7 @@ public class GeoHashGridAggregator extends BucketsAggregator {
private final LongHash bucketOrds;
public GeoHashGridAggregator(String name, AggregatorFactories factories, GeoGridAggregationBuilder.CellIdSource valuesSource,
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
int requiredSize, int shardSize, SearchContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;

View File

@ -27,11 +27,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder.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 org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Collections;
@ -45,7 +45,7 @@ public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<
private final int shardSize;
public GeoHashGridAggregatorFactory(String name, Type type, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
int shardSize, AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
int shardSize, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.precision = precision;

View File

@ -27,7 +27,7 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -52,7 +52,7 @@ public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalA
}
@Override
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
return new GlobalAggregatorFactory(name, type, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -25,7 +25,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -33,7 +33,7 @@ import java.util.Map;
public class GlobalAggregator extends SingleBucketAggregator {
public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, List<PipelineAggregator> pipelineAggregators,
public GlobalAggregator(String name, AggregatorFactories subFactories, SearchContext aggregationContext, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, subFactories, aggregationContext, null, pipelineAggregators, metaData);
}

View File

@ -25,7 +25,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -33,7 +33,7 @@ import java.util.Map;
public class GlobalAggregatorFactory extends AggregatorFactory<GlobalAggregatorFactory> {
public GlobalAggregatorFactory(String name, Type type, AggregationContext context, AggregatorFactory<?> parent,
public GlobalAggregatorFactory(String name, Type type, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
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.ValuesSource.Numeric;
@ -41,6 +40,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.HashMap;
@ -320,13 +320,13 @@ public class DateHistogramAggregationBuilder
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
Rounding rounding = createRounding();
ExtendedBounds roundedBounds = null;
if (this.extendedBounds != null) {
// parse any string bounds to longs and round
roundedBounds = this.extendedBounds.parseAndValidate(name, context.searchContext(), config.format()).round(rounding);
roundedBounds = this.extendedBounds.parseAndValidate(name, context, config.format()).round(rounding);
}
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
rounding, roundedBounds, context, parent, subFactoriesBuilder, metaData);

View File

@ -33,8 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -65,7 +65,7 @@ class DateHistogramAggregator extends BucketsAggregator {
public DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, InternalOrder order,
boolean keyed,
long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource,
DocValueFormat formatter, AggregationContext aggregationContext,
DocValueFormat formatter, SearchContext aggregationContext,
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);

View File

@ -25,17 +25,16 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
public final class DateHistogramAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, DateHistogramAggregatorFactory> {
@ -50,7 +49,7 @@ public final class DateHistogramAggregatorFactory
public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval,
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
Rounding rounding, ExtendedBounds extendedBounds, AggregationContext context, AggregatorFactory<?> parent,
Rounding rounding, ExtendedBounds extendedBounds, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.interval = interval;

View File

@ -31,7 +31,6 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
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.ValuesSource.Numeric;
@ -40,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -265,7 +265,7 @@ public class HistogramAggregationBuilder
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, minBound, maxBound,
context, parent, subFactoriesBuilder, metaData);

View File

@ -34,8 +34,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo;
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.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -64,10 +64,10 @@ class HistogramAggregator extends BucketsAggregator {
public HistogramAggregator(String name, AggregatorFactories factories, double interval, double offset,
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
@Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter,
AggregationContext aggregationContext, Aggregator parent,
SearchContext context, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
if (interval <= 0) {
throw new IllegalArgumentException("interval must be positive, got: " + interval);
}
@ -81,7 +81,7 @@ class HistogramAggregator extends BucketsAggregator {
this.valuesSource = valuesSource;
this.formatter = formatter;
bucketOrds = new LongHash(1, aggregationContext.bigArrays());
bucketOrds = new LongHash(1, context.bigArrays());
}
@Override

View File

@ -24,11 +24,11 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -44,7 +44,7 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double interval, double offset,
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
AggregationContext context, AggregatorFactory<?> parent,
SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.interval = interval;

View File

@ -27,7 +27,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSourceAggregationBuilder;
@ -35,6 +34,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -74,7 +74,7 @@ public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new MissingAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -27,8 +27,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -39,7 +39,7 @@ public class MissingAggregator extends SingleBucketAggregator {
private final ValuesSource valuesSource;
public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;

View File

@ -24,10 +24,10 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -35,7 +35,7 @@ import java.util.Map;
public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, MissingAggregatorFactory> {
public MissingAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, AggregationContext context,
public MissingAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -31,7 +31,7 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -78,9 +78,9 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
}
@Override
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
ObjectMapper childObjectMapper = context.searchContext().getObjectMapper(path);
ObjectMapper childObjectMapper = context.getObjectMapper(path);
if (childObjectMapper == null) {
// in case the path has been unmapped:
return new NestedAggregatorFactory(name, type, null, null, context, parent, subFactoriesBuilder, metaData);
@ -90,11 +90,11 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested");
}
try {
ObjectMapper parentObjectMapper = context.searchContext().getQueryShardContext().nestedScope().nextLevel(childObjectMapper);
ObjectMapper parentObjectMapper = context.getQueryShardContext().nestedScope().nextLevel(childObjectMapper);
return new NestedAggregatorFactory(name, type, parentObjectMapper, childObjectMapper, context, parent, subFactoriesBuilder,
metaData);
} finally {
context.searchContext().getQueryShardContext().nestedScope().previousLevel();
context.getQueryShardContext().nestedScope().previousLevel();
}
}

View File

@ -38,7 +38,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -52,11 +52,11 @@ public class NestedAggregator extends SingleBucketAggregator {
private final Query childFilter;
public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper,
AggregationContext aggregationContext, Aggregator parentAggregator,
SearchContext context, Aggregator parentAggregator,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parentAggregator, pipelineAggregators, metaData);
super(name, factories, context, parentAggregator, pipelineAggregators, metaData);
Query parentFilter = parentObjectMapper != null ? parentObjectMapper.nestedTypeFilter() : Queries.newNonNestedFilter();
this.parentFilter = context.searchContext().bitsetFilterCache().getBitSetProducer(parentFilter);
this.parentFilter = context.bitsetFilterCache().getBitSetProducer(parentFilter);
this.childFilter = childObjectMapper.nestedTypeFilter();
}

View File

@ -20,15 +20,14 @@
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.ObjectMapper;
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.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -40,7 +39,7 @@ public class NestedAggregatorFactory extends AggregatorFactory<NestedAggregatorF
private final ObjectMapper childObjectMapper;
public NestedAggregatorFactory(String name, Type type, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
this.parentObjectMapper = parentObjectMapper;
@ -61,7 +60,7 @@ public class NestedAggregatorFactory extends AggregatorFactory<NestedAggregatorF
private static final class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
public Unmapped(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}

View File

@ -33,7 +33,7 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -82,16 +82,16 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
}
@Override
protected AggregatorFactory<?> doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
if (findNestedAggregatorFactory(parent) == null) {
throw new SearchParseException(context.searchContext(),
throw new SearchParseException(context,
"Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation", null);
}
ObjectMapper parentObjectMapper = null;
if (path != null) {
parentObjectMapper = context.searchContext().getObjectMapper(path);
parentObjectMapper = context.getObjectMapper(path);
if (parentObjectMapper == null) {
return new ReverseNestedAggregatorFactory(name, type, true, null, context, parent, subFactoriesBuilder, metaData);
}
@ -100,7 +100,7 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
}
}
NestedScope nestedScope = context.searchContext().getQueryShardContext().nestedScope();
NestedScope nestedScope = context.getQueryShardContext().nestedScope();
try {
nestedScope.nextLevel(parentObjectMapper);
return new ReverseNestedAggregatorFactory(name, type, false, parentObjectMapper, context, parent, subFactoriesBuilder,

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.nested;
import com.carrotsearch.hppc.LongIntHashMap;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Query;
@ -34,7 +35,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -48,15 +49,15 @@ public class ReverseNestedAggregator extends SingleBucketAggregator {
private final BitSetProducer parentBitsetProducer;
public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
if (objectMapper == null) {
parentFilter = Queries.newNonNestedFilter();
} else {
parentFilter = objectMapper.nestedTypeFilter();
}
parentBitsetProducer = context.searchContext().bitsetFilterCache().getBitSetProducer(parentFilter);
parentBitsetProducer = context.bitsetFilterCache().getBitSetProducer(parentFilter);
}
@Override

View File

@ -20,16 +20,14 @@
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.search.SearchParseException;
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.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -41,7 +39,7 @@ public class ReverseNestedAggregatorFactory extends AggregatorFactory<ReverseNes
private final ObjectMapper parentObjectMapper;
public ReverseNestedAggregatorFactory(String name, Type type, boolean unmapped, ObjectMapper parentObjectMapper,
AggregationContext context, AggregatorFactory<?> parent,
SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
@ -61,7 +59,7 @@ public class ReverseNestedAggregatorFactory extends AggregatorFactory<ReverseNes
private static final class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
public Unmapped(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}

View File

@ -26,11 +26,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -44,7 +44,7 @@ public class AbstractRangeAggregatorFactory<AF extends AbstractRangeAggregatorFa
private final boolean keyed;
public AbstractRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, R[] ranges, boolean keyed,
InternalRange.Factory<?, ?> rangeFactory, AggregationContext context, AggregatorFactory<?> parent,
InternalRange.Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.ranges = ranges;

View File

@ -25,11 +25,11 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -63,10 +63,10 @@ public abstract class AbstractRangeBuilder<AB extends AbstractRangeBuilder<AB, R
* Resolve any strings in the ranges so we have a number value for the from
* and to of each range. The ranges are also sorted before being returned.
*/
protected Range[] processRanges(AggregationContext context, ValuesSourceConfig<Numeric> config) {
protected Range[] processRanges(SearchContext context, ValuesSourceConfig<Numeric> config) {
Range[] ranges = new Range[this.ranges.size()];
for (int i = 0; i < ranges.length; i++) {
ranges[i] = this.ranges.get(i).process(config.format(), context.searchContext());
ranges[i] = this.ranges.get(i).process(config.format(), context);
}
sortRanges(ranges);
return ranges;

View File

@ -39,8 +39,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
/** A range aggregator for values that are stored in SORTED_SET doc values. */
public final class BinaryRangeAggregator extends BucketsAggregator {
@ -78,10 +78,10 @@ public final class BinaryRangeAggregator extends BucketsAggregator {
public BinaryRangeAggregator(String name, AggregatorFactories factories,
ValuesSource.Bytes valuesSource, DocValueFormat format,
List<Range> ranges, boolean keyed, AggregationContext aggregationContext,
List<Range> ranges, boolean keyed, SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.format = format;
this.keyed = keyed;

View File

@ -27,10 +27,10 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
public class BinaryRangeAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Bytes, BinaryRangeAggregatorFactory> {
@ -41,7 +41,7 @@ public class BinaryRangeAggregatorFactory
public BinaryRangeAggregatorFactory(String name, Type type,
ValuesSourceConfig<ValuesSource.Bytes> config,
List<BinaryRangeAggregator.Range> ranges, boolean keyed,
AggregationContext context,
SearchContext context,
AggregatorFactory<?> parent, Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);

View File

@ -28,10 +28,10 @@ import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -138,7 +138,7 @@ public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregati
}
@Override
protected RangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected RangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
// We need to call processRanges here so they are parsed before we make the decision of whether to cache the request
Range[] ranges = processRanges(context, config);

View File

@ -38,7 +38,6 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
@ -206,10 +205,10 @@ public class RangeAggregator extends BucketsAggregator {
final double[] maxTo;
public RangeAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
InternalRange.Factory rangeFactory, Range[] ranges, boolean keyed, AggregationContext aggregationContext,
InternalRange.Factory rangeFactory, Range[] ranges, boolean keyed, SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
assert valuesSource != null;
this.valuesSource = valuesSource;
this.format = format;
@ -338,7 +337,7 @@ public class RangeAggregator extends BucketsAggregator {
@SuppressWarnings("unchecked")
public Unmapped(String name, R[] ranges, boolean keyed, DocValueFormat format,
AggregationContext context,
SearchContext context,
Aggregator parent, InternalRange.Factory factory, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {

View File

@ -25,8 +25,8 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -34,7 +34,7 @@ import java.util.Map;
public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory<RangeAggregatorFactory, RangeAggregator.Range> {
public RangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
Factory<?, ?> rangeFactory, AggregationContext context, AggregatorFactory<?> parent,
Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -30,10 +30,10 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.internal.SearchContext;
import org.joda.time.DateTime;
import java.io.IOException;
@ -283,7 +283,7 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
}
@Override
protected DateRangeAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected DateRangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
// We need to call processRanges here so they are parsed and we know whether `now` has been used before we make
// the decision of whether to cache the request

View File

@ -26,8 +26,8 @@ import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeAggregato
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -35,7 +35,7 @@ import java.util.Map;
public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory<DateRangeAggregatorFactory, Range> {
public DateRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
Factory<?, ?> rangeFactory, AggregationContext context, AggregatorFactory<?> parent,
Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -38,12 +38,12 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -385,7 +385,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(AggregationContext context,
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
Range[] ranges = this.ranges.toArray(new Range[this.range().size()]);

View File

@ -22,8 +22,8 @@ package org.elasticsearch.search.aggregations.bucket.range.geodistance;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
@ -37,10 +37,10 @@ import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceAggregationBuilder.Range;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -57,7 +57,7 @@ public class GeoDistanceRangeAggregatorFactory
private final boolean keyed;
public GeoDistanceRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, GeoPoint origin,
Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, AggregationContext context,
Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.origin = origin;

View File

@ -40,7 +40,6 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.range.BinaryRangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.BinaryRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
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.ValuesSourceAggregationBuilder;
@ -48,6 +47,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.net.InetAddress;
@ -369,7 +369,7 @@ public final class IpRangeAggregationBuilder
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.Bytes, ?> innerBuild(
AggregationContext context, ValuesSourceConfig<ValuesSource.Bytes> config,
SearchContext context, ValuesSourceConfig<ValuesSource.Bytes> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
List<BinaryRangeAggregator.Range> ranges = new ArrayList<>();

View File

@ -28,13 +28,13 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -137,7 +137,7 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context,
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new DiversifiedAggregatorFactory(name, TYPE, config, shardSize, maxDocsPerValue, executionHint, context, parent,
subFactoriesBuilder, metaData);

View File

@ -24,15 +24,15 @@ 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.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -45,7 +45,7 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
private final String executionHint;
public DiversifiedAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
String executionHint, AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
String executionHint, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.shardSize = shardSize;
@ -65,7 +65,7 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, context.searchContext().parseFieldMatcher());
execution = ExecutionMode.fromString(executionHint, context.parseFieldMatcher());
}
// In some cases using ordinals is just not supported: override

View File

@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -49,10 +49,10 @@ public class DiversifiedBytesHashSamplerAggregator extends SamplerAggregator {
private int maxDocsPerValue;
public DiversifiedBytesHashSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
ValuesSource valuesSource,
int maxDocsPerValue) throws IOException {
super(name, shardSize, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.maxDocsPerValue = maxDocsPerValue;
}

View File

@ -34,8 +34,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -48,12 +48,12 @@ public class DiversifiedMapSamplerAggregator extends SamplerAggregator {
private BytesRefHash bucketOrds;
public DiversifiedMapSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
ValuesSource valuesSource, int maxDocsPerValue) throws IOException {
super(name, shardSize, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.maxDocsPerValue = maxDocsPerValue;
bucketOrds = new BytesRefHash(shardSize, aggregationContext.bigArrays());
bucketOrds = new BytesRefHash(shardSize, context.bigArrays());
}

View File

@ -31,8 +31,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -44,9 +44,9 @@ public class DiversifiedNumericSamplerAggregator extends SamplerAggregator {
private int maxDocsPerValue;
public DiversifiedNumericSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
ValuesSource.Numeric valuesSource, int maxDocsPerValue) throws IOException {
super(name, shardSize, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.maxDocsPerValue = maxDocsPerValue;
}

View File

@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
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.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -45,9 +45,9 @@ public class DiversifiedOrdinalsSamplerAggregator extends SamplerAggregator {
private int maxDocsPerValue;
public DiversifiedOrdinalsSamplerAggregator(String name, int shardSize, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, int maxDocsPerValue) throws IOException {
super(name, shardSize, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.maxDocsPerValue = maxDocsPerValue;
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -75,7 +75,7 @@ public class SamplerAggregationBuilder extends AbstractAggregationBuilder<Sample
}
@Override
protected SamplerAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
protected SamplerAggregatorFactory doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
return new SamplerAggregatorFactory(name, type, shardSize, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -31,8 +31,8 @@ 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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -61,7 +61,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
@Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new DiversifiedMapSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData,
@ -79,7 +79,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
@Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new DiversifiedBytesHashSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators,
@ -98,7 +98,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
@Override
Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new DiversifiedOrdinalsSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxDocsPerValue);
@ -127,7 +127,7 @@ public class SamplerAggregator extends SingleBucketAggregator {
}
abstract Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
abstract boolean needsGlobalOrdinals();
@ -142,9 +142,9 @@ public class SamplerAggregator extends SingleBucketAggregator {
protected final int shardSize;
protected BestDocsDeferringCollector bdd;
public SamplerAggregator(String name, int shardSize, AggregatorFactories factories, AggregationContext aggregationContext,
public SamplerAggregator(String name, int shardSize, AggregatorFactories factories, SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
super(name, factories, context, parent, pipelineAggregators, metaData);
this.shardSize = shardSize;
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -34,7 +34,7 @@ public class SamplerAggregatorFactory extends AggregatorFactory<SamplerAggregato
private final int shardSize;
public SamplerAggregatorFactory(String name, Type type, int shardSize, AggregationContext context, AggregatorFactory<?> parent,
public SamplerAggregatorFactory(String name, Type type, int shardSize, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
this.shardSize = shardSize;

View File

@ -32,9 +32,9 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.internal.ContextIndexSearcher;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -55,11 +55,11 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories,
ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, DocValueFormat format,
BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
AggregationContext aggregationContext, Aggregator parent,
SearchContext context, Aggregator parent,
SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, null, format, bucketCountThresholds, includeExclude, aggregationContext, parent,
super(name, factories, valuesSource, null, format, bucketCountThresholds, includeExclude, context, parent,
SubAggCollectionMode.DEPTH_FIRST, false, pipelineAggregators, metaData);
this.significanceHeuristic = significanceHeuristic;
this.termsAggFactory = termsAggFactory;
@ -143,7 +143,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
@Override
public SignificantStringTerms buildEmptyAggregation() {
// We need to account for the significance of a miss in our global stats - provide corpus size as context
ContextIndexSearcher searcher = context.searchContext().searcher();
ContextIndexSearcher searcher = context.searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(),
@ -161,12 +161,12 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
DocValueFormat format, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SearchContext context, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, bucketCountThresholds, includeExclude, aggregationContext, parent, significanceHeuristic,
super(name, factories, valuesSource, format, bucketCountThresholds, includeExclude, context, parent, significanceHeuristic,
termsAggFactory, pipelineAggregators, metaData);
bucketOrds = new LongHash(1, aggregationContext.bigArrays());
bucketOrds = new LongHash(1, context.bigArrays());
}
@Override

View File

@ -30,9 +30,9 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.internal.ContextIndexSearcher;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -44,12 +44,12 @@ import static java.util.Collections.emptyList;
public class SignificantLongTermsAggregator extends LongTermsAggregator {
public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource,
DocValueFormat format, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent,
DocValueFormat format, BucketCountThresholds bucketCountThresholds, SearchContext context, Aggregator parent,
SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
IncludeExclude.LongFilter includeExclude,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, null, bucketCountThresholds, aggregationContext, parent,
super(name, factories, valuesSource, format, null, bucketCountThresholds, context, parent,
SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, pipelineAggregators, metaData);
this.significanceHeuristic = significanceHeuristic;
this.termsAggFactory = termsAggFactory;
@ -116,7 +116,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
@Override
public SignificantLongTerms buildEmptyAggregation() {
// We need to account for the significance of a miss in our global stats - provide corpus size as context
ContextIndexSearcher searcher = context.searchContext().searcher();
ContextIndexSearcher searcher = context.searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantLongTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(),

View File

@ -31,9 +31,9 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.bucket.terms.StringTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.internal.ContextIndexSearcher;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -52,7 +52,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
private final SignificanceHeuristic significanceHeuristic;
public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, DocValueFormat format,
BucketCountThresholds bucketCountThresholds, IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext,
BucketCountThresholds bucketCountThresholds, IncludeExclude.StringFilter includeExclude, SearchContext aggregationContext,
Aggregator parent, SignificanceHeuristic significanceHeuristic, SignificantTermsAggregatorFactory termsAggFactory,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -126,7 +126,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
@Override
public SignificantStringTerms buildEmptyAggregation() {
// We need to account for the significance of a miss in our global stats - provide corpus size as context
ContextIndexSearcher searcher = context.searchContext().searcher();
ContextIndexSearcher searcher = context.searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(),

View File

@ -39,7 +39,6 @@ import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilde
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.ValuesSourceAggregationBuilder;
@ -47,6 +46,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -264,9 +264,9 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context.searchContext());
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context);
return new SignificantTermsAggregatorFactory(name, type, config, includeExclude, executionHint, filterBuilder,
bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -19,14 +19,14 @@
package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
@ -41,15 +41,14 @@ 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.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@ -75,15 +74,15 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
public SignificantTermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, IncludeExclude includeExclude,
String executionHint, QueryBuilder filterBuilder, TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic, AggregationContext context, AggregatorFactory<?> parent,
SignificanceHeuristic significanceHeuristic, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.includeExclude = includeExclude;
this.executionHint = executionHint;
this.filter = filterBuilder == null
? null
: filterBuilder.toQuery(context.searchContext().getQueryShardContext());
IndexSearcher searcher = context.searchContext().searcher();
: filterBuilder.toQuery(context.getQueryShardContext());
IndexSearcher searcher = context.searcher();
this.supersetNumDocs = filter == null
// Important - need to use the doc count that includes deleted docs
// or we have this issue: https://github.com/elastic/elasticsearch/issues/7951
@ -91,7 +90,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
: searcher.count(filter);
this.bucketCountThresholds = bucketCountThresholds;
this.significanceHeuristic = significanceHeuristic;
setFieldInfo(context.searchContext());
setFieldInfo(context);
}
@ -113,7 +112,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
if (termsEnum != null) {
return termsEnum;
}
IndexReader reader = context.searchContext().searcher().getIndexReader();
IndexReader reader = context.searcher().getIndexReader();
if (numberOfAggregatorsCreated > 1) {
termsEnum = new FreqTermsEnum(reader, field, true, false, filter, context.bigArrays());
} else {
@ -123,7 +122,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
private long getBackgroundFrequency(String value) throws IOException {
Query query = fieldType.termQuery(value, context.searchContext().getQueryShardContext());
Query query = fieldType.termQuery(value, context.getQueryShardContext());
if (query instanceof TermQuery) {
// for types that use the inverted index, we prefer using a caching terms
// enum that will do a better job at reusing index inputs
@ -142,7 +141,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
.add(filter, Occur.FILTER)
.build();
}
return context.searchContext().searcher().count(query);
return context.searcher().count(query);
}
public long getBackgroundFrequency(BytesRef termBytes) throws IOException {
@ -191,13 +190,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
// such are impossible to differentiate from non-significant terms
// at that early stage.
bucketCountThresholds.setShardSize(2 * BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
context.searchContext().numberOfShards()));
context.numberOfShards()));
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, context.searchContext().parseFieldMatcher());
execution = ExecutionMode.fromString(executionHint, context.parseFieldMatcher());
}
if (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals)) {
execution = ExecutionMode.MAP;
@ -251,7 +250,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, DocValueFormat format,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SearchContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter(format);
@ -265,7 +264,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, DocValueFormat format,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SearchContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(format);
@ -280,7 +279,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, DocValueFormat format,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SearchContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(format);
@ -307,7 +306,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, DocValueFormat format,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SearchContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;

View File

@ -24,7 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -36,7 +36,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
protected final boolean showTermDocCountError;
public AbstractStringTermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent,
public AbstractStringTermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);

View File

@ -27,9 +27,9 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -39,7 +39,7 @@ import java.util.stream.Collectors;
public class DoubleTermsAggregator extends LongTermsAggregator {
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent,
Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode,

View File

@ -26,7 +26,6 @@ import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LongBitSet;
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.IntArray;
@ -44,8 +43,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -72,10 +71,10 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent,
IncludeExclude.OrdinalsFilter includeExclude, SearchContext context, Aggregator parent,
SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, order, format, bucketCountThresholds, collectionMode, showTermDocCountError,
super(name, factories, context, parent, order, format, bucketCountThresholds, collectionMode, showTermDocCountError,
pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.includeExclude = includeExclude;
@ -262,12 +261,12 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, Terms.Order order,
DocValueFormat format, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode,
SearchContext context, Aggregator parent, SubAggCollectionMode collectionMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, factories, valuesSource, order, format, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode,
super(name, factories, valuesSource, order, format, bucketCountThresholds, includeExclude, context, parent, collectionMode,
showTermDocCountError, pipelineAggregators, metaData);
bucketOrds = new LongHash(1, aggregationContext.bigArrays());
bucketOrds = new LongHash(1, context.bigArrays());
}
@Override
@ -335,10 +334,10 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
Terms.Order order, DocValueFormat format,
BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent,
BucketCountThresholds bucketCountThresholds, SearchContext context, Aggregator parent,
SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, order, format, bucketCountThresholds, null, aggregationContext, parent, collectionMode,
super(name, factories, valuesSource, order, format, bucketCountThresholds, null, context, parent, collectionMode,
showTermDocCountError, pipelineAggregators, metaData);
assert factories == null || factories.countAggregators() == 0;
this.segmentDocCounts = context.bigArrays().newIntArray(1, true);

View File

@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriority
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter;
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.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -50,7 +50,7 @@ public class LongTermsAggregator extends TermsAggregator {
private LongFilter longFilter;
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent,
Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);
@ -107,7 +107,7 @@ public class LongTermsAggregator extends TermsAggregator {
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
// we need to fill-in the blanks
for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) {
for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
final SortedNumericDocValues values = getValues(valuesSource, ctx);
for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) {
values.setDocument(docId);

View File

@ -33,8 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -52,15 +52,15 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext,
IncludeExclude.StringFilter includeExclude, SearchContext context,
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, order, format, bucketCountThresholds, collectionMode, showTermDocCountError,
super(name, factories, context, parent, order, format, bucketCountThresholds, collectionMode, showTermDocCountError,
pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.includeExclude = includeExclude;
bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays());
bucketOrds = new BytesRefHash(1, context.bigArrays());
}
@Override
@ -110,7 +110,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
// we need to fill-in the blanks
for (LeafReaderContext ctx : context.searchContext().searcher().getTopReaderContext().leaves()) {
for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
// brute force
for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) {

View File

@ -35,7 +35,6 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
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.ValuesSourceAggregationBuilder;
@ -43,6 +42,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -285,7 +285,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new TermsAggregatorFactory(name, type, config, order, includeExclude, executionHint, collectMode,
bucketCountThresholds, showTermDocCountError, context, parent, subFactoriesBuilder, metaData);

View File

@ -33,8 +33,8 @@ import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.Aggregation;
import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.CompoundOrder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.HashSet;
@ -172,7 +172,7 @@ public abstract class TermsAggregator extends BucketsAggregator {
protected final Set<Aggregator> aggsUsedForSorting = new HashSet<>();
protected final SubAggCollectionMode collectMode;
public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent,
public TermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
BucketCountThresholds bucketCountThresholds, Terms.Order order, DocValueFormat format, SubAggCollectionMode collectMode,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, pipelineAggregators, metaData);

View File

@ -25,20 +25,20 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
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.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -55,7 +55,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
public TermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
IncludeExclude includeExclude, String executionHint, SubAggCollectionMode collectMode,
TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError, AggregationContext context,
TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.order = order;
@ -98,13 +98,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
// heuristic to avoid any wrong-ranking caused by distributed
// counting
bucketCountThresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize(),
context.searchContext().numberOfShards()));
context.numberOfShards()));
}
bucketCountThresholds.ensureValidity();
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null;
if (executionHint != null) {
execution = ExecutionMode.fromString(executionHint, context.searchContext().parseFieldMatcher());
execution = ExecutionMode.fromString(executionHint, context.parseFieldMatcher());
}
// In some cases, using ordinals is just not supported: override it
@ -116,7 +116,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
final double ratio;
if (execution == null || execution.needsGlobalOrdinals()) {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = context.searchContext().searcher();
IndexSearcher indexSearcher = context.searcher();
maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher);
ratio = maxOrd / ((double) indexSearcher.getIndexReader().numDocs());
} else {
@ -227,12 +227,12 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter(format);
return new StringTermsAggregator(name, factories, valuesSource, order, format, bucketCountThresholds, filter,
aggregationContext, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
context, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
}
@Override
@ -246,12 +246,12 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(format);
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, order,
format, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError,
format, bucketCountThresholds, filter, context, parent, subAggCollectMode, showTermDocCountError,
pipelineAggregators, metaData);
}
@ -266,12 +266,12 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(format);
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource,
order, format, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError,
order, format, bucketCountThresholds, filter, context, parent, subAggCollectMode, showTermDocCountError,
pipelineAggregators, metaData);
}
@ -285,7 +285,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (includeExclude != null || factories.countAggregators() > 0
@ -293,10 +293,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
// segment to global ord mapping
|| valuesSource.getClass() != ValuesSource.Bytes.FieldData.class) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, format, bucketCountThresholds, includeExclude,
aggregationContext, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
context, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
}
return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories,
(ValuesSource.Bytes.WithOrdinals) valuesSource, order, format, bucketCountThresholds, aggregationContext, parent,
(ValuesSource.Bytes.WithOrdinals) valuesSource, order, format, bucketCountThresholds, context, parent,
subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData);
}
@ -323,7 +323,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode,
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException;

View File

@ -23,7 +23,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -31,7 +31,7 @@ import java.util.Map;
public abstract class MetricsAggregator extends AggregatorBase {
protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
protected MetricsAggregator(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, AggregatorFactories.EMPTY, context, parent, pipelineAggregators, metaData);
}

View File

@ -20,7 +20,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -28,14 +28,14 @@ import java.util.Map;
public abstract class NumericMetricsAggregator extends MetricsAggregator {
private NumericMetricsAggregator(String name, AggregationContext context, Aggregator parent,
private NumericMetricsAggregator(String name, SearchContext context, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
public abstract static class SingleValue extends NumericMetricsAggregator {
protected SingleValue(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
protected SingleValue(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}
@ -45,7 +45,7 @@ public abstract class NumericMetricsAggregator extends MetricsAggregator {
public abstract static class MultiValue extends NumericMetricsAggregator {
protected MultiValue(String name, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
protected MultiValue(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSource.Numeric;
@ -36,6 +35,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuil
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -70,7 +70,7 @@ public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected AvgAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected AvgAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new AvgAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -31,8 +31,8 @@ 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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -46,7 +46,7 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
DoubleArray sums;
DocValueFormat format;
public AvgAggregator(String name, ValuesSource.Numeric valuesSource, DocValueFormat formatter, AggregationContext context,
public AvgAggregator(String name, ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;

View File

@ -24,11 +24,11 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -36,7 +36,7 @@ import java.util.Map;
public class AvgAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AvgAggregatorFactory> {
public AvgAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, AggregationContext context,
public AvgAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -29,13 +29,13 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -122,7 +122,7 @@ public final class CardinalityAggregationBuilder
}
@Override
protected CardinalityAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
protected CardinalityAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new CardinalityAggregatorFactory(name, type, config, precisionThreshold, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.metrics.cardinality;
import com.carrotsearch.hppc.BitMixer;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedNumericDocValues;
@ -40,8 +41,8 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -62,7 +63,7 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
private Collector collector;
public CardinalityAggregator(String name, ValuesSource valuesSource, int precision,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.precision = precision;

View File

@ -24,10 +24,10 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -38,7 +38,7 @@ public class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<
private final Long precisionThreshold;
public CardinalityAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Long precisionThreshold,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.precisionThreshold = precisionThreshold;

View File

@ -28,13 +28,13 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -89,7 +89,7 @@ public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<
}
@Override
protected GeoBoundsAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
protected GeoBoundsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new GeoBoundsAggregatorFactory(name, type, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -52,7 +52,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
DoubleArray negLefts;
DoubleArray negRights;
protected GeoBoundsAggregator(String name, AggregationContext aggregationContext, Aggregator parent,
protected GeoBoundsAggregator(String name, SearchContext aggregationContext, Aggregator parent,
ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, aggregationContext, parent, pipelineAggregators, metaData);

View File

@ -24,10 +24,10 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -38,7 +38,7 @@ public class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
private final boolean wrapLongitude;
public GeoBoundsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.wrapLongitude = wrapLongitude;

View File

@ -28,13 +28,13 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -70,7 +70,7 @@ public class GeoCentroidAggregationBuilder
}
@Override
protected GeoCentroidAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
protected GeoCentroidAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new GeoCentroidAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -47,10 +47,10 @@ public final class GeoCentroidAggregator extends MetricsAggregator {
LongArray centroids;
LongArray counts;
protected GeoCentroidAggregator(String name, AggregationContext aggregationContext, Aggregator parent,
protected GeoCentroidAggregator(String name, SearchContext context, Aggregator parent,
ValuesSource.GeoPoint valuesSource, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, aggregationContext, parent, pipelineAggregators, metaData);
super(name, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final BigArrays bigArrays = context.bigArrays();

View File

@ -24,10 +24,10 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -36,7 +36,7 @@ import java.util.Map;
public class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, GeoCentroidAggregatorFactory> {
public GeoCentroidAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSource.Numeric;
@ -36,6 +35,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuil
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -70,7 +70,7 @@ public class MaxAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected MaxAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected MaxAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new MaxAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -24,16 +24,16 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.Aggregator;
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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -47,7 +47,7 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
DoubleArray maxes;
public MaxAggregator(String name, ValuesSource.Numeric valuesSource, DocValueFormat formatter,
AggregationContext context,
SearchContext context,
Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);

View File

@ -24,11 +24,11 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -36,7 +36,7 @@ import java.util.Map;
public class MaxAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, MaxAggregatorFactory> {
public MaxAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, AggregationContext context,
public MaxAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -29,7 +29,6 @@ import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder;
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;
@ -37,6 +36,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuil
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -71,7 +71,7 @@ public class MinAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected MinAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected MinAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new MinAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -24,16 +24,16 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.Aggregator;
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.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -47,7 +47,7 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
DoubleArray mins;
public MinAggregator(String name, ValuesSource.Numeric valuesSource, DocValueFormat formatter,
AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;

View File

@ -24,11 +24,11 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
@ -36,7 +36,7 @@ import java.util.Map;
public class MinAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, MinAggregatorFactory> {
public MinAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, AggregationContext context,
public MinAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentileRanksAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentileRanksAggregatorFactory;
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;
@ -40,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -222,7 +222,7 @@ public class PercentileRanksAggregationBuilder extends LeafOnly<ValuesSource.Num
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
switch (method) {
case TDIGEST:

View File

@ -31,7 +31,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.HDRPercentilesAggregatorFactory;
import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestPercentilesAggregatorFactory;
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;
@ -40,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -228,7 +228,7 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
switch (method) {
case TDIGEST:

Some files were not shown because too many files have changed in this diff Show More