Replace the SearchContext with QueryShardContext when building aggregator factories (#46527)

This commit replaces the `SearchContext` with the `QueryShardContext` when building aggregator factories. Aggregator factories are part of the `SearchContext` so they shouldn't require a `SearchContext` to create them.
The main changes here are the signatures of `AggregationBuilder#build` that now takes a `QueryShardContext` and `AggregatorFactory#createInternal` that passes the `SearchContext` to build the `Aggregator`.

Relates #46523
This commit is contained in:
Jim Ferenczi 2019-09-11 16:12:34 +02:00 committed by jimczi
parent 27c15f137e
commit 23bf310c84
130 changed files with 1291 additions and 957 deletions

View File

@ -22,6 +22,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -32,7 +33,6 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -80,9 +80,11 @@ public class MatrixStatsAggregationBuilder
}
@Override
protected MatrixStatsAggregatorFactory innerBuild(SearchContext context, Map<String, ValuesSourceConfig<Numeric>> configs,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
return new MatrixStatsAggregatorFactory(name, configs, multiValueMode, context, parent, subFactoriesBuilder, metaData);
protected MatrixStatsAggregatorFactory innerBuild(QueryShardContext queryShardContext,
Map<String, ValuesSourceConfig<Numeric>> configs,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
return new MatrixStatsAggregatorFactory(name, configs, multiValueMode, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.search.aggregations.matrix.stats;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -37,23 +38,32 @@ final class MatrixStatsAggregatorFactory extends ArrayValuesSourceAggregatorFact
private final MultiValueMode multiValueMode;
MatrixStatsAggregatorFactory(String name,
Map<String, ValuesSourceConfig<ValuesSource.Numeric>> configs, MultiValueMode multiValueMode,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, configs, context, parent, subFactoriesBuilder, metaData);
Map<String, ValuesSourceConfig<ValuesSource.Numeric>> configs,
MultiValueMode multiValueMode,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, configs, queryShardContext, parent, subFactoriesBuilder, metaData);
this.multiValueMode = multiValueMode;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
return new MatrixStatsAggregator(name, null, context, parent, multiValueMode, pipelineAggregators, metaData);
return new MatrixStatsAggregator(name, null, searchContext, parent, multiValueMode, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Map<String, ValuesSource.Numeric> valuesSources, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MatrixStatsAggregator(name, valuesSources, context, parent, multiValueMode, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Map<String, ValuesSource.Numeric> valuesSources,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MatrixStatsAggregator(name, valuesSources, searchContext, parent, multiValueMode, pipelineAggregators, metaData);
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
@ -34,7 +35,6 @@ import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -239,28 +239,28 @@ public abstract class ArrayValuesSourceAggregationBuilder<VS extends ValuesSourc
}
@Override
protected final ArrayValuesSourceAggregatorFactory<VS> doBuild(SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
Map<String, ValuesSourceConfig<VS>> configs = resolveConfig(context);
ArrayValuesSourceAggregatorFactory<VS> factory = innerBuild(context, configs, parent, subFactoriesBuilder);
protected final ArrayValuesSourceAggregatorFactory<VS> doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
Map<String, ValuesSourceConfig<VS>> configs = resolveConfig(queryShardContext);
ArrayValuesSourceAggregatorFactory<VS> factory = innerBuild(queryShardContext, configs, parent, subFactoriesBuilder);
return factory;
}
protected Map<String, ValuesSourceConfig<VS>> resolveConfig(SearchContext context) {
protected Map<String, ValuesSourceConfig<VS>> resolveConfig(QueryShardContext queryShardContext) {
HashMap<String, ValuesSourceConfig<VS>> configs = new HashMap<>();
for (String field : fields) {
ValuesSourceConfig<VS> config = config(context, field, null);
ValuesSourceConfig<VS> config = config(queryShardContext, field, null);
configs.put(field, config);
}
return configs;
}
protected abstract ArrayValuesSourceAggregatorFactory<VS> innerBuild(SearchContext context,
protected abstract ArrayValuesSourceAggregatorFactory<VS> innerBuild(QueryShardContext queryShardContext,
Map<String, ValuesSourceConfig<VS>> configs,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException;
public ValuesSourceConfig<VS> config(SearchContext context, String field, Script script) {
public ValuesSourceConfig<VS> config(QueryShardContext queryShardContext, String field, Script script) {
ValueType valueType = this.valueType != null ? this.valueType : targetValueType;
@ -282,7 +282,7 @@ public abstract class ArrayValuesSourceAggregationBuilder<VS extends ValuesSourc
return config.format(resolveFormat(format, valueType));
}
MappedFieldType fieldType = context.smartNameFieldType(field);
MappedFieldType fieldType = queryShardContext.getMapperService().fullName(field);
if (fieldType == null) {
ValuesSourceType valuesSourceType = valueType != null ? valueType.getValuesSourceType() : this.valuesSourceType;
ValuesSourceConfig<VS> config = new ValuesSourceConfig<>(valuesSourceType);
@ -291,7 +291,7 @@ public abstract class ArrayValuesSourceAggregationBuilder<VS extends ValuesSourc
return config.unmapped(true);
}
IndexFieldData<?> indexFieldData = context.getForField(fieldType);
IndexFieldData<?> indexFieldData = queryShardContext.getForField(fieldType);
ValuesSourceConfig<VS> config;
if (valuesSourceType == ValuesSourceType.ANY) {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.support;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -36,35 +37,44 @@ public abstract class ArrayValuesSourceAggregatorFactory<VS extends ValuesSource
protected Map<String, ValuesSourceConfig<VS>> configs;
public ArrayValuesSourceAggregatorFactory(String name, Map<String, ValuesSourceConfig<VS>> configs,
SearchContext context, AggregatorFactory parent,
QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactoriesBuilder, metaData);
super(name, queryShardContext, parent, subFactoriesBuilder, metaData);
this.configs = configs;
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
HashMap<String, VS> valuesSources = new HashMap<>();
for (Map.Entry<String, ValuesSourceConfig<VS>> config : configs.entrySet()) {
VS vs = config.getValue().toValuesSource(context.getQueryShardContext());
VS vs = config.getValue().toValuesSource(queryShardContext);
if (vs != null) {
valuesSources.put(config.getKey(), vs);
}
}
if (valuesSources.isEmpty()) {
return createUnmapped(parent, pipelineAggregators, metaData);
return createUnmapped(searchContext, parent, pipelineAggregators, metaData);
}
return doCreateInternal(valuesSources, parent, collectsFromSingleBucket, pipelineAggregators, metaData);
return doCreateInternal(valuesSources, searchContext, parent,
collectsFromSingleBucket, pipelineAggregators, metaData);
}
protected abstract Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
protected abstract Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
protected abstract Aggregator doCreateInternal(Map<String, VS> valuesSources, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
protected abstract Aggregator doCreateInternal(Map<String, VS> valuesSources,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
}

View File

@ -570,9 +570,9 @@ public class PainlessExecuteAction extends ActionType<PainlessExecuteAction.Resp
ParsedDocument parsedDocument = indexService.mapperService().documentMapper().parse(sourceToParse);
indexWriter.addDocuments(parsedDocument.docs());
try (IndexReader indexReader = DirectoryReader.open(indexWriter)) {
final long absoluteStartMillis = System.currentTimeMillis();
final IndexSearcher searcher = new IndexSearcher(indexReader);
searcher.setQueryCache(null);
final long absoluteStartMillis = System.currentTimeMillis();
QueryShardContext context =
indexService.newQueryShardContext(0, searcher, () -> absoluteStartMillis, null);
return handler.apply(context, indexReader.leaves().get(0));

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.join.mapper.ParentIdFieldMapper;
import org.elasticsearch.join.mapper.ParentJoinFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilder;
@ -39,7 +40,6 @@ 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.Map;
@ -95,29 +95,29 @@ public class ChildrenAggregationBuilder
}
@Override
protected ValuesSourceAggregatorFactory<WithOrdinals> innerBuild(SearchContext context,
ValuesSourceConfig<WithOrdinals> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new ChildrenAggregatorFactory(name, config, childFilter, parentFilter, context, parent,
protected ValuesSourceAggregatorFactory<WithOrdinals> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<WithOrdinals> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new ChildrenAggregatorFactory(name, config, childFilter, parentFilter, queryShardContext, parent,
subFactoriesBuilder, metaData);
}
@Override
protected ValuesSourceConfig<WithOrdinals> resolveConfig(SearchContext context) {
protected ValuesSourceConfig<WithOrdinals> resolveConfig(QueryShardContext queryShardContext) {
ValuesSourceConfig<WithOrdinals> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
joinFieldResolveConfig(context, config);
joinFieldResolveConfig(queryShardContext, config);
return config;
}
private void joinFieldResolveConfig(SearchContext context, ValuesSourceConfig<WithOrdinals> config) {
ParentJoinFieldMapper parentJoinFieldMapper = ParentJoinFieldMapper.getMapper(context.mapperService());
private void joinFieldResolveConfig(QueryShardContext queryShardContext, ValuesSourceConfig<WithOrdinals> config) {
ParentJoinFieldMapper parentJoinFieldMapper = ParentJoinFieldMapper.getMapper(queryShardContext.getMapperService());
ParentIdFieldMapper parentIdFieldMapper = parentJoinFieldMapper.getParentIdFieldMapper(childType, false);
if (parentIdFieldMapper != null) {
parentFilter = parentIdFieldMapper.getParentFilter();
childFilter = parentIdFieldMapper.getChildFilter(childType);
MappedFieldType fieldType = parentIdFieldMapper.fieldType();
final SortedSetDVOrdinalsIndexFieldData fieldData = context.getForField(fieldType);
final SortedSetDVOrdinalsIndexFieldData fieldData = queryShardContext.getForField(fieldType);
config.fieldContext(new FieldContext(fieldType.name(), fieldData, fieldType));
} else {
config.unmapped(true);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.join.aggregations;
import org.apache.lucene.search.Query;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -44,7 +45,7 @@ public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory<Wit
ValuesSourceConfig<WithOrdinals> config,
Query childFilter,
Query parentFilter,
SearchContext context,
QueryShardContext context,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
@ -55,9 +56,9 @@ public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory<Wit
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
@ -67,17 +68,17 @@ public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory<Wit
@Override
protected Aggregator doCreateInternal(WithOrdinals valuesSource,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
SearchContext searchContext, Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
long maxOrd = valuesSource.globalMaxOrd(context.searcher());
long maxOrd = valuesSource.globalMaxOrd(searchContext.searcher());
if (collectsFromSingleBucket) {
return new ParentToChildrenAggregator(name, factories, context, parent, childFilter,
return new ParentToChildrenAggregator(name, factories, searchContext, parent, childFilter,
parentFilter, valuesSource, maxOrd, pipelineAggregators, metaData);
} else {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.join.mapper.ParentIdFieldMapper;
import org.elasticsearch.join.mapper.ParentJoinFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilder;
@ -39,7 +40,6 @@ 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.Map;
@ -95,29 +95,29 @@ public class ParentAggregationBuilder
}
@Override
protected ValuesSourceAggregatorFactory<WithOrdinals> innerBuild(SearchContext context,
ValuesSourceConfig<WithOrdinals> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new ParentAggregatorFactory(name, config, childFilter, parentFilter, context, parent,
protected ValuesSourceAggregatorFactory<WithOrdinals> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<WithOrdinals> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new ParentAggregatorFactory(name, config, childFilter, parentFilter, queryShardContext, parent,
subFactoriesBuilder, metaData);
}
@Override
protected ValuesSourceConfig<WithOrdinals> resolveConfig(SearchContext context) {
protected ValuesSourceConfig<WithOrdinals> resolveConfig(QueryShardContext queryShardContext) {
ValuesSourceConfig<WithOrdinals> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
joinFieldResolveConfig(context, config);
joinFieldResolveConfig(queryShardContext, config);
return config;
}
private void joinFieldResolveConfig(SearchContext context, ValuesSourceConfig<WithOrdinals> config) {
ParentJoinFieldMapper parentJoinFieldMapper = ParentJoinFieldMapper.getMapper(context.mapperService());
private void joinFieldResolveConfig(QueryShardContext queryShardContext, ValuesSourceConfig<WithOrdinals> config) {
ParentJoinFieldMapper parentJoinFieldMapper = ParentJoinFieldMapper.getMapper(queryShardContext.getMapperService());
ParentIdFieldMapper parentIdFieldMapper = parentJoinFieldMapper.getParentIdFieldMapper(childType, false);
if (parentIdFieldMapper != null) {
parentFilter = parentIdFieldMapper.getParentFilter();
childFilter = parentIdFieldMapper.getChildFilter(childType);
MappedFieldType fieldType = parentIdFieldMapper.fieldType();
final SortedSetDVOrdinalsIndexFieldData fieldData = context.getForField(fieldType);
final SortedSetDVOrdinalsIndexFieldData fieldData = queryShardContext.getForField(fieldType);
config.fieldContext(new FieldContext(fieldType.name(), fieldData, fieldType));
} else {
config.unmapped(true);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.join.aggregations;
import org.apache.lucene.search.Query;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -44,20 +45,20 @@ public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory<WithO
ValuesSourceConfig<WithOrdinals> config,
Query childFilter,
Query parentFilter,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.childFilter = childFilter;
this.parentFilter = parentFilter;
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalParent(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
@ -67,17 +68,17 @@ public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory<WithO
@Override
protected Aggregator doCreateInternal(WithOrdinals valuesSource,
Aggregator children,
SearchContext searchContext, Aggregator children,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
long maxOrd = valuesSource.globalMaxOrd(context.searcher());
long maxOrd = valuesSource.globalMaxOrd(searchContext.searcher());
if (collectsFromSingleBucket) {
return new ChildrenToParentAggregator(name, factories, context, children, childFilter,
return new ChildrenToParentAggregator(name, factories, searchContext, children, childFilter,
parentFilter, valuesSource, maxOrd, pipelineAggregators, metaData);
} else {
return asMultiBucketAggregator(this, context, children);
return asMultiBucketAggregator(this, searchContext, children);
}
}
}

View File

@ -130,8 +130,8 @@ public class QueryShardContext extends QueryRewriteContext {
public QueryShardContext(QueryShardContext source) {
this(source.shardId, source.indexSettings, source.bigArrays, source.bitsetFilterCache, source.indexFieldDataService,
source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(),
source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.fullyQualifiedIndex);
source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(),
source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.fullyQualifiedIndex);
}
private QueryShardContext(int shardId,
@ -152,13 +152,13 @@ public class QueryShardContext extends QueryRewriteContext {
this.shardId = shardId;
this.similarityService = similarityService;
this.mapperService = mapperService;
this.bigArrays = bigArrays;
this.bitsetFilterCache = bitsetFilterCache;
this.indexFieldDataService = indexFieldDataLookup;
this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields();
this.nestedScope = new NestedScope();
this.scriptService = scriptService;
this.indexSettings = indexSettings;
this.bigArrays = bigArrays;
this.searcher = searcher;
this.fullyQualifiedIndex = fullyQualifiedIndex;
}
@ -435,7 +435,7 @@ public class QueryShardContext extends QueryRewriteContext {
/** Return the current {@link IndexReader}, or {@code null} if no index reader is available,
* for instance if this rewrite context is used to index queries (percolation). */
public IndexReader getIndexReader() {
return searcher != null ? searcher.getIndexReader() : null;
return searcher == null ? null : searcher.getIndexReader();
}
/** Return the current {@link IndexSearcher}, or {@code null} if no index reader is available,

View File

@ -180,8 +180,8 @@ final class DefaultSearchContext extends SearchContext {
this.relativeTimeSupplier = relativeTimeSupplier;
this.timeout = timeout;
this.minNodeVersion = minNodeVersion;
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher, request::nowInMillis,
shardTarget.getClusterAlias());
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher,
request::nowInMillis, shardTarget.getClusterAlias());
queryShardContext.setTypes(request.types());
queryBoost = request.indexBoost();
}

View File

@ -787,7 +787,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
context.terminateAfter(source.terminateAfter());
if (source.aggregations() != null) {
try {
AggregatorFactories factories = source.aggregations().build(context, null);
AggregatorFactories factories = source.aggregations().build(queryShardContext, null);
context.aggregations(new SearchContextAggregations(factories, multiBucketConsumerService.create()));
} catch (IOException e) {
throw new AggregationInitializationException("Failed to create aggregators", e);

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Collections;
@ -135,13 +135,13 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
}
@Override
public final AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException {
AggregatorFactory factory = doBuild(context, parent, factoriesBuilder);
public final AggregatorFactory build(QueryShardContext queryShardContext, AggregatorFactory parent) throws IOException {
AggregatorFactory factory = doBuild(queryShardContext, parent, factoriesBuilder);
return factory;
}
protected abstract AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subfactoriesBuilder) throws IOException;
protected abstract AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subfactoriesBuilder) throws IOException;
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {

View File

@ -25,7 +25,7 @@ import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Collection;
@ -63,7 +63,7 @@ public abstract class AggregationBuilder
}
/** Internal: build an {@link AggregatorFactory} based on the configuration of this builder. */
protected abstract AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException;
protected abstract AggregatorFactory build(QueryShardContext queryShardContext, AggregatorFactory parent) throws IOException;
/** Associate metadata with this {@link AggregationBuilder}. */
@Override

View File

@ -52,7 +52,7 @@ public class AggregationPhase implements SearchPhase {
Aggregator[] aggregators;
try {
AggregatorFactories factories = context.aggregations().factories();
aggregators = factories.createTopLevelAggregators();
aggregators = factories.createTopLevelAggregators(context);
for (int i = 0; i < aggregators.length; i++) {
if (aggregators[i] instanceof GlobalAggregator == false) {
collectors.add(aggregators[i]);

View File

@ -74,7 +74,7 @@ public abstract class AggregatorBase extends Aggregator {
this.context = context;
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);
this.subAggregators = factories.createSubAggregators(context, this);
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() {

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -187,7 +188,7 @@ public class AggregatorFactories {
* Create all aggregators so that they can be consumed with multiple
* buckets.
*/
public Aggregator[] createSubAggregators(Aggregator parent) throws IOException {
public Aggregator[] createSubAggregators(SearchContext searchContext, Aggregator parent) throws IOException {
Aggregator[] aggregators = new Aggregator[countAggregators()];
for (int i = 0; i < factories.length; ++i) {
// TODO: sometimes even sub aggregations always get called with bucket 0, eg. if
@ -195,7 +196,7 @@ public class AggregatorFactories {
// propagate the fact that only bucket 0 will be collected with single-bucket
// aggs
final boolean collectsFromSingleBucket = false;
Aggregator factory = factories[i].create(parent, collectsFromSingleBucket);
Aggregator factory = factories[i].create(searchContext, parent, collectsFromSingleBucket);
Profilers profilers = factory.context().getProfilers();
if (profilers != null) {
factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler());
@ -205,13 +206,13 @@ public class AggregatorFactories {
return aggregators;
}
public Aggregator[] createTopLevelAggregators() throws IOException {
public Aggregator[] createTopLevelAggregators(SearchContext searchContext) throws IOException {
// These aggregators are going to be used with a single bucket ordinal, no need to wrap the PER_BUCKET ones
Aggregator[] aggregators = new Aggregator[factories.length];
for (int i = 0; i < factories.length; i++) {
// top-level aggs only get called with bucket 0
final boolean collectsFromSingleBucket = true;
Aggregator factory = factories[i].create(null, collectsFromSingleBucket);
Aggregator factory = factories[i].create(searchContext, null, collectsFromSingleBucket);
Profilers profilers = factory.context().getProfilers();
if (profilers != null) {
factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler());
@ -314,7 +315,7 @@ public class AggregatorFactories {
return this;
}
public AggregatorFactories build(SearchContext context, AggregatorFactory parent) throws IOException {
public AggregatorFactories build(QueryShardContext queryShardContext, AggregatorFactory parent) throws IOException {
if (aggregationBuilders.isEmpty() && pipelineAggregatorBuilders.isEmpty()) {
return EMPTY;
}
@ -329,7 +330,7 @@ public class AggregatorFactories {
int i = 0;
for (AggregationBuilder agg : aggregationBuilders) {
aggFactories[i] = agg.build(context, parent);
aggFactories[i] = agg.build(queryShardContext, parent);
++i;
}
return new AggregatorFactories(aggFactories, orderedpipelineAggregators);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.ScoreMode;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
@ -43,8 +44,8 @@ public abstract class AggregatorFactory {
ObjectArray<Aggregator> aggregators;
ObjectArray<LeafBucketCollector> collectors;
MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context, Aggregator parent, AggregatorFactory factory,
Aggregator first) {
MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context,
Aggregator parent, AggregatorFactory factory, Aggregator first) {
this.bigArrays = bigArrays;
this.parent = parent;
this.factory = factory;
@ -126,7 +127,7 @@ public abstract class AggregatorFactory {
aggregators = bigArrays.grow(aggregators, bucket + 1);
Aggregator aggregator = aggregators.get(bucket);
if (aggregator == null) {
aggregator = factory.create(parent, true);
aggregator = factory.create(context(), parent, true);
aggregator.preCollection();
aggregators.set(bucket, aggregator);
}
@ -170,7 +171,8 @@ public abstract class AggregatorFactory {
protected final AggregatorFactory parent;
protected final AggregatorFactories factories;
protected final Map<String, Object> metaData;
protected final SearchContext context;
protected final QueryShardContext queryShardContext;
/**
* Constructs a new aggregator factory.
@ -180,12 +182,12 @@ public abstract class AggregatorFactory {
* @throws IOException
* if an error occurs creating the factory
*/
public AggregatorFactory(String name, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
public AggregatorFactory(String name, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
this.name = name;
this.context = context;
this.queryShardContext = queryShardContext;
this.parent = parent;
this.factories = subFactoriesBuilder.build(context, this);
this.factories = subFactoriesBuilder.build(queryShardContext, this);
this.metaData = metaData;
}
@ -196,12 +198,18 @@ public abstract class AggregatorFactory {
public void doValidate() {
}
protected abstract Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException;
protected abstract Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException;
/**
* Creates the aggregator
*
*
* @param searchContext
* The search context
* @param parent
* The parent aggregator (if this is a top level factory, the
* parent will be {@code null})
@ -213,8 +221,8 @@ public abstract class AggregatorFactory {
*
* @return The created aggregator
*/
public final Aggregator create(Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
return createInternal(parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData);
public final Aggregator create(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
return createInternal(searchContext, parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData);
}
public AggregatorFactory getParent() {
@ -226,11 +234,11 @@ public abstract class AggregatorFactory {
* {@link Aggregator}s that only know how to collect bucket {@code 0}, this
* returns an aggregator that can collect any bucket.
*/
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext context,
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext searchContext,
final Aggregator parent) throws IOException {
final Aggregator first = factory.create(parent, true);
final BigArrays bigArrays = context.bigArrays();
return new MultiBucketAggregatorWrapper(bigArrays, context, parent, factory, first);
final Aggregator first = factory.create(searchContext, parent, true);
final BigArrays bigArrays = searchContext.bigArrays();
return new MultiBucketAggregatorWrapper(bigArrays, searchContext, parent, factory, first);
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.Rewriteable;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
@ -34,7 +35,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.MultiBucketAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.adjacency.AdjacencyMatrixAggregator.KeyedFilter;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -196,9 +196,9 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
int maxFilters = context.indexShard().indexSettings().getMaxAdjacencyMatrixFilters();
int maxFilters = queryShardContext.getIndexSettings().getMaxAdjacencyMatrixFilters();
if (filters.size() > maxFilters){
throw new IllegalArgumentException(
"Number of filters is too large, must be less than or equal to: [" + maxFilters + "] but was ["
@ -209,10 +209,10 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
List<KeyedFilter> rewrittenFilters = new ArrayList<>(filters.size());
for (KeyedFilter kf : filters) {
rewrittenFilters.add(new KeyedFilter(kf.key(), Rewriteable.rewrite(kf.filter(), context.getQueryShardContext(), true)));
rewrittenFilters.add(new KeyedFilter(kf.key(), Rewriteable.rewrite(kf.filter(), queryShardContext, true)));
}
return new AdjacencyMatrixAggregatorFactory(name, rewrittenFilters, separator, context, parent,
return new AdjacencyMatrixAggregatorFactory(name, rewrittenFilters, separator, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Weight;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -40,26 +41,29 @@ public class AdjacencyMatrixAggregatorFactory extends AggregatorFactory {
private final Weight[] weights;
private final String separator;
public AdjacencyMatrixAggregatorFactory(String name, List<KeyedFilter> filters, String separator,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
IndexSearcher contextSearcher = context.searcher();
public AdjacencyMatrixAggregatorFactory(String name, List<KeyedFilter> filters, String separator,
QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactories, metaData);
IndexSearcher contextSearcher = queryShardContext.searcher();
this.separator = separator;
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().toQuery(context.getQueryShardContext());
Query filter = keyedFilter.filter().toQuery(queryShardContext);
this.weights[i] = contextSearcher.createWeight(contextSearcher.rewrite(filter), ScoreMode.COMPLETE_NO_SCORES, 1f);
}
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AdjacencyMatrixAggregator(name, factories, separator, keys, weights, context, parent,
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AdjacencyMatrixAggregator(name, factories, separator, keys, weights, searchContext, parent,
pipelineAggregators, metaData);
}

View File

@ -25,12 +25,12 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -195,8 +195,8 @@ public class CompositeAggregationBuilder extends AbstractAggregationBuilder<Comp
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subfactoriesBuilder) throws IOException {
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subfactoriesBuilder) throws IOException {
AggregatorFactory invalid = checkParentIsNullOrNested(parent);
if (invalid != null) {
throw new IllegalArgumentException("[composite] aggregation cannot be used with a parent aggregation of" +
@ -204,7 +204,7 @@ public class CompositeAggregationBuilder extends AbstractAggregationBuilder<Comp
}
CompositeValuesSourceConfig[] configs = new CompositeValuesSourceConfig[sources.size()];
for (int i = 0; i < configs.length; i++) {
configs[i] = sources.get(i).build(context);
configs[i] = sources.get(i).build(queryShardContext);
if (configs[i].valuesSource().needsScores()) {
throw new IllegalArgumentException("[sources] cannot access _score");
}
@ -235,7 +235,7 @@ public class CompositeAggregationBuilder extends AbstractAggregationBuilder<Comp
} else {
afterKey = null;
}
return new CompositeAggregationFactory(name, context, parent, subfactoriesBuilder, metaData, size, configs, afterKey);
return new CompositeAggregationFactory(name, queryShardContext, parent, subfactoriesBuilder, metaData, size, configs, afterKey);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.composite;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,19 +35,19 @@ class CompositeAggregationFactory extends AggregatorFactory {
private final CompositeValuesSourceConfig[] sources;
private final CompositeKey afterKey;
CompositeAggregationFactory(String name, SearchContext context, AggregatorFactory parent,
CompositeAggregationFactory(String name, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData,
int size, CompositeValuesSourceConfig[] sources, CompositeKey afterKey) throws IOException {
super(name, context, parent, subFactoriesBuilder, metaData);
super(name, queryShardContext, parent, subFactoriesBuilder, metaData);
this.size = size;
this.sources = sources;
this.afterKey = afterKey;
}
@Override
protected Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket,
protected Aggregator createInternal(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new CompositeAggregator(name, factories, context, parent, pipelineAggregators, metaData,
return new CompositeAggregator(name, factories, searchContext, parent, pipelineAggregators, metaData,
size, sources, afterKey);
}
}

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.ToXContentFragment;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
@ -289,15 +289,15 @@ public abstract class CompositeValuesSourceBuilder<AB extends CompositeValuesSou
/**
* Creates a {@link CompositeValuesSourceConfig} for this source.
*
* @param context The search context for this source.
* @param queryShardContext The shard context for this source.
* @param config The {@link ValuesSourceConfig} for this source.
*/
protected abstract CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig<?> config) throws IOException;
protected abstract CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<?> config) throws IOException;
public final CompositeValuesSourceConfig build(SearchContext context) throws IOException {
ValuesSourceConfig<?> config = ValuesSourceConfig.resolve(context.getQueryShardContext(),
public final CompositeValuesSourceConfig build(QueryShardContext queryShardContext) throws IOException {
ValuesSourceConfig<?> config = ValuesSourceConfig.resolve(queryShardContext,
valueType, field, script, null,null, format);
return innerBuild(context, config);
return innerBuild(queryShardContext, config);
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
@ -36,7 +37,6 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateIntervalWrappe
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.time.ZoneId;
@ -216,9 +216,9 @@ public class DateHistogramValuesSourceBuilder
}
@Override
protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig<?> config) throws IOException {
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<?> config) throws IOException {
Rounding rounding = dateHistogramInterval.createRounding(timeZone());
ValuesSource orig = config.toValuesSource(context.getQueryShardContext());
ValuesSource orig = config.toValuesSource(queryShardContext);
if (orig == null) {
orig = ValuesSource.Numeric.EMPTY;
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.bucket.geogrid.CellIdSource;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileGridAggregationBuilder;
@ -33,7 +34,6 @@ import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -103,8 +103,8 @@ public class GeoTileGridValuesSourceBuilder extends CompositeValuesSourceBuilder
}
@Override
protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig<?> config) throws IOException {
ValuesSource orig = config.toValuesSource(context.getQueryShardContext());
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<?> config) throws IOException {
ValuesSource orig = config.toValuesSource(queryShardContext);
if (orig == null) {
orig = ValuesSource.GeoPoint.EMPTY;
}

View File

@ -25,11 +25,11 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -110,8 +110,8 @@ public class HistogramValuesSourceBuilder extends CompositeValuesSourceBuilder<H
}
@Override
protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig<?> config) throws IOException {
ValuesSource orig = config.toValuesSource(context.getQueryShardContext());
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<?> config) throws IOException {
ValuesSource orig = config.toValuesSource(queryShardContext);
if (orig == null) {
orig = ValuesSource.Numeric.EMPTY;
}

View File

@ -26,10 +26,10 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.script.Script;
import java.io.IOException;
@ -70,8 +70,8 @@ public class TermsValuesSourceBuilder extends CompositeValuesSourceBuilder<Terms
}
@Override
protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig<?> config) throws IOException {
ValuesSource vs = config.toValuesSource(context.getQueryShardContext());
protected CompositeValuesSourceConfig innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<?> config) throws IOException {
ValuesSource vs = config.toValuesSource(queryShardContext);
if (vs == null) {
// The field is unmapped so we use a value source that can parse any type of values.
// This is needed because the after values are parsed even when there are no values to process.

View File

@ -25,12 +25,12 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.Rewriteable;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -93,9 +93,9 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
return new FilterAggregatorFactory(name, filter, context, parent, subFactoriesBuilder, metaData);
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
return new FilterAggregatorFactory(name, filter, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -24,6 +24,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Weight;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -40,10 +41,10 @@ public class FilterAggregatorFactory extends AggregatorFactory {
private Weight weight;
private Query filter;
public FilterAggregatorFactory(String name, QueryBuilder filterBuilder, SearchContext context,
public FilterAggregatorFactory(String name, QueryBuilder filterBuilder, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactoriesBuilder, metaData);
filter = filterBuilder.toQuery(context.getQueryShardContext());
super(name, queryShardContext, parent, subFactoriesBuilder, metaData);
filter = filterBuilder.toQuery(queryShardContext);
}
/**
@ -57,7 +58,7 @@ public class FilterAggregatorFactory extends AggregatorFactory {
*/
public Weight getWeight() {
if (weight == null) {
IndexSearcher contextSearcher = context.searcher();
IndexSearcher contextSearcher = queryShardContext.searcher();
try {
weight = contextSearcher.createWeight(contextSearcher.rewrite(filter), ScoreMode.COMPLETE_NO_SCORES, 1f);
} catch (IOException e) {
@ -68,9 +69,12 @@ public class FilterAggregatorFactory extends AggregatorFactory {
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new FilterAggregator(name, () -> this.getWeight(), factories, context, parent, pipelineAggregators, metaData);
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new FilterAggregator(name, () -> this.getWeight(), factories, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.Rewriteable;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
@ -34,7 +35,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.MultiBucketAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -220,9 +220,9 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
return new FiltersAggregatorFactory(name, filters, keyed, otherBucket, otherBucketKey, context, parent,
return new FiltersAggregatorFactory(name, filters, keyed, otherBucket, otherBucketKey, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Weight;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -45,9 +46,9 @@ public class FiltersAggregatorFactory extends AggregatorFactory {
private final String otherBucketKey;
public FiltersAggregatorFactory(String name, List<KeyedFilter> filters, boolean keyed, boolean otherBucket,
String otherBucketKey, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
String otherBucketKey, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactories, metaData);
this.keyed = keyed;
this.otherBucket = otherBucket;
this.otherBucketKey = otherBucketKey;
@ -56,7 +57,7 @@ public class FiltersAggregatorFactory extends AggregatorFactory {
for (int i = 0; i < filters.size(); ++i) {
KeyedFilter keyedFilter = filters.get(i);
this.keys[i] = keyedFilter.key();
this.filters[i] = keyedFilter.filter().toQuery(context.getQueryShardContext());
this.filters[i] = keyedFilter.filter().toQuery(queryShardContext);
}
}
@ -69,10 +70,10 @@ public class FiltersAggregatorFactory extends AggregatorFactory {
* Note that as aggregations are initialsed and executed in a serial manner,
* no concurrency considerations are necessary here.
*/
public Weight[] getWeights() {
public Weight[] getWeights(SearchContext searchContext) {
if (weights == null) {
try {
IndexSearcher contextSearcher = context.searcher();
IndexSearcher contextSearcher = searchContext.searcher();
weights = new Weight[filters.length];
for (int i = 0; i < filters.length; ++i) {
this.weights[i] = contextSearcher.createWeight(contextSearcher.rewrite(filters[i]), ScoreMode.COMPLETE_NO_SCORES, 1);
@ -85,10 +86,13 @@ public class FiltersAggregatorFactory extends AggregatorFactory {
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new FiltersAggregator(name, factories, keys, () -> getWeights(), keyed, otherBucket ? otherBucketKey : null, context, parent,
pipelineAggregators, metaData);
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new FiltersAggregator(name, factories, keys, () -> getWeights(searchContext), keyed,
otherBucket ? otherBucketKey : null, searchContext, parent, pipelineAggregators, metaData);
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
@ -37,7 +38,6 @@ 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.Map;
@ -110,7 +110,7 @@ public abstract class GeoGridAggregationBuilder extends ValuesSourceAggregationB
*/
protected abstract ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> createFactory(
String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, int precision, int requiredSize, int shardSize,
SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder, Map<String, Object> metaData
QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder, Map<String, Object> metaData
) throws IOException;
public int precision() {
@ -144,8 +144,9 @@ public abstract class GeoGridAggregationBuilder extends ValuesSourceAggregationB
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory parent, Builder subFactoriesBuilder)
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
int shardSize = this.shardSize;
@ -165,7 +166,7 @@ public abstract class GeoGridAggregationBuilder extends ValuesSourceAggregationB
if (shardSize < requiredSize) {
shardSize = requiredSize;
}
return createFactory(name, config, precision, requiredSize, shardSize, context, parent,
return createFactory(name, config, precision, requiredSize, shardSize, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -23,13 +23,13 @@ import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.Map;
@ -61,10 +61,9 @@ public class GeoHashGridAggregationBuilder extends GeoGridAggregationBuilder {
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> createFactory(
String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, int precision, int requiredSize, int shardSize,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData
) throws IOException {
return new GeoHashGridAggregatorFactory(name, config, precision, requiredSize, shardSize, context, parent,
QueryShardContext queryShardContext, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
return new GeoHashGridAggregatorFactory(name, config, precision, requiredSize, shardSize, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.geometry.utils.Geohash;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -44,20 +45,22 @@ public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<
private final int shardSize;
GeoHashGridAggregatorFactory(String name, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
int shardSize, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.precision = precision;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize,
Collections.emptyList(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -66,13 +69,17 @@ public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(final GeoPoint valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
CellIdSource cellIdSource = new CellIdSource(valuesSource, precision, Geohash::longEncode);
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, context, parent,
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, searchContext, parent,
pipelineAggregators, metaData);
}
}

View File

@ -22,13 +22,13 @@ package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.Map;
@ -60,10 +60,10 @@ public class GeoTileGridAggregationBuilder extends GeoGridAggregationBuilder {
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> createFactory(
String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, int precision, int requiredSize, int shardSize,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
QueryShardContext queryShardContext, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData
) throws IOException {
return new GeoTileGridAggregatorFactory(name, config, precision, requiredSize, shardSize, context, parent,
return new GeoTileGridAggregatorFactory(name, config, precision, requiredSize, shardSize, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -43,21 +44,22 @@ public class GeoTileGridAggregatorFactory extends ValuesSourceAggregatorFactory<
private final int shardSize;
GeoTileGridAggregatorFactory(String name, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
int shardSize, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData
) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
int shardSize, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.precision = precision;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new InternalGeoTileGrid(name, requiredSize,
Collections.emptyList(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -66,13 +68,17 @@ public class GeoTileGridAggregatorFactory extends ValuesSourceAggregatorFactory<
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(final GeoPoint valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
CellIdSource cellIdSource = new CellIdSource(valuesSource, precision, GeoTileUtils::longEncode);
return new GeoTileGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, context, parent,
return new GeoTileGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, searchContext, parent,
pipelineAggregators, metaData);
}
}

View File

@ -23,11 +23,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -61,9 +61,9 @@ public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalA
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
return new GlobalAggregatorFactory(name, context, parent, subFactoriesBuilder, metaData);
return new GlobalAggregatorFactory(name, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -32,14 +33,20 @@ import java.util.Map;
public class GlobalAggregatorFactory extends AggregatorFactory {
public GlobalAggregatorFactory(String name, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
public GlobalAggregatorFactory(String name,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactories, metaData);
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (parent != null) {
throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " + "sub-aggregation [" + name
+ "]. Global aggregations can only be defined as top level aggregations");
@ -47,6 +54,6 @@ public class GlobalAggregatorFactory extends AggregatorFactory {
if (collectsFromSingleBucket == false) {
throw new IllegalStateException();
}
return new GlobalAggregator(name, factories, context, pipelineAggregators, metaData);
return new GlobalAggregator(name, factories, searchContext, pipelineAggregators, metaData);
}
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -41,7 +42,6 @@ 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.time.ZoneId;
@ -184,22 +184,22 @@ public class AutoDateHistogramAggregationBuilder
}
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
RoundingInfo[] roundings = buildRoundings(timeZone(), getMinimumIntervalExpression());
int maxRoundingInterval = Arrays.stream(roundings,0, roundings.length-1)
.map(rounding -> rounding.innerIntervals)
.flatMapToInt(Arrays::stream)
.boxed()
.reduce(Integer::max).get();
Settings settings = context.getQueryShardContext().getIndexSettings().getNodeSettings();
Settings settings = queryShardContext.getIndexSettings().getNodeSettings();
int maxBuckets = MultiBucketConsumerService.MAX_BUCKET_SETTING.get(settings);
int bucketCeiling = maxBuckets / maxRoundingInterval;
if (numBuckets > bucketCeiling) {
throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName()+
" must be less than " + bucketCeiling);
}
return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent,
return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, queryShardContext, parent,
subFactoriesBuilder,
metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -44,34 +45,42 @@ public final class AutoDateHistogramAggregatorFactory
ValuesSourceConfig<Numeric> config,
int numBuckets,
RoundingInfo[] roundingInfos,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.numBuckets = numBuckets;
this.roundingInfos = roundingInfos;
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
return createAggregator(valuesSource, parent, pipelineAggregators, metaData);
return createAggregator(valuesSource, searchContext, parent, pipelineAggregators, metaData);
}
private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, valuesSource, config.format(), context, parent,
pipelineAggregators,
metaData);
private Aggregator createAggregator(ValuesSource.Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos,
valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return createAggregator(null, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return createAggregator(null, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -52,7 +52,6 @@ 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.time.Instant;
@ -491,11 +490,13 @@ public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuil
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
final ZoneId tz = timeZone();
final Rounding rounding = dateHistogramInterval.createRounding(tz);
final ZoneId rewrittenTimeZone = rewriteTimeZone(context.getQueryShardContext());
final ZoneId rewrittenTimeZone = rewriteTimeZone(queryShardContext);
final Rounding shardRounding;
if (tz == rewrittenTimeZone) {
shardRounding = rounding;
@ -506,10 +507,10 @@ public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuil
ExtendedBounds roundedBounds = null;
if (this.extendedBounds != null) {
// parse any string bounds to longs and round
roundedBounds = this.extendedBounds.parseAndValidate(name, context, config.format()).round(rounding);
roundedBounds = this.extendedBounds.parseAndValidate(name, queryShardContext, config.format()).round(rounding);
}
return new DateHistogramAggregatorFactory(name, config, offset, order, keyed, minDocCount,
rounding, shardRounding, roundedBounds, context, parent, subFactoriesBuilder, metaData);
rounding, shardRounding, roundedBounds, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.Rounding;
import org.elasticsearch.index.mapper.RangeType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -48,10 +49,10 @@ public final class DateHistogramAggregatorFactory
public DateHistogramAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config,
long offset, BucketOrder order, boolean keyed, long minDocCount,
Rounding rounding, Rounding shardRounding, ExtendedBounds extendedBounds, SearchContext context,
Rounding rounding, Rounding shardRounding, ExtendedBounds extendedBounds, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.offset = offset;
this.order = order;
this.keyed = keyed;
@ -75,20 +76,24 @@ public final class DateHistogramAggregatorFactory
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
if (valuesSource instanceof ValuesSource.Numeric) {
return createAggregator((ValuesSource.Numeric) valuesSource, parent, pipelineAggregators, metaData);
return createAggregator((ValuesSource.Numeric) valuesSource, searchContext, parent, pipelineAggregators, metaData);
} else if (valuesSource instanceof ValuesSource.Range) {
ValuesSource.Range rangeValueSource = (ValuesSource.Range) valuesSource;
if (rangeValueSource.rangeType() != RangeType.DATE) {
throw new IllegalArgumentException("Expected date range type but found range type [" + rangeValueSource.rangeType().name
+ "]");
}
return createRangeAggregator((ValuesSource.Range) valuesSource, parent, pipelineAggregators, metaData);
return createRangeAggregator((ValuesSource.Range) valuesSource, searchContext, parent, pipelineAggregators, metaData);
}
else {
throw new IllegalArgumentException("Expected one of [Date, Range] values source, found ["
@ -96,22 +101,27 @@ public final class DateHistogramAggregatorFactory
}
}
private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
private Aggregator createAggregator(ValuesSource.Numeric valuesSource, SearchContext searchContext,
Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new DateHistogramAggregator(name, factories, rounding, shardRounding, offset, order, keyed, minDocCount, extendedBounds,
valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
private Aggregator createRangeAggregator(ValuesSource.Range valuesSource, Aggregator parent,
private Aggregator createRangeAggregator(ValuesSource.Range valuesSource,
SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new DateRangeHistogramAggregator(name, factories, rounding, shardRounding, offset, order, keyed, minDocCount, extendedBounds,
valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return createAggregator(null, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return createAggregator(null, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -31,9 +31,8 @@ import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Objects;
@ -148,20 +147,20 @@ public class ExtendedBounds implements ToXContentFragment, Writeable {
/**
* Parse the bounds and perform any delayed validation. Returns the result of the parsing.
*/
ExtendedBounds parseAndValidate(String aggName, SearchContext context, DocValueFormat format) {
ExtendedBounds parseAndValidate(String aggName, QueryShardContext queryShardContext, DocValueFormat format) {
Long min = this.min;
Long max = this.max;
assert format != null;
if (minAsStr != null) {
min = format.parseLong(minAsStr, false, context.getQueryShardContext()::nowInMillis);
min = format.parseLong(minAsStr, false, queryShardContext::nowInMillis);
}
if (maxAsStr != null) {
// TODO: Should we rather pass roundUp=true?
max = format.parseLong(maxAsStr, false, context.getQueryShardContext()::nowInMillis);
max = format.parseLong(maxAsStr, false, queryShardContext::nowInMillis);
}
if (min != null && max != null && min.compareTo(max) > 0) {
throw new SearchParseException(context, "[extended_bounds.min][" + min + "] cannot be greater than " +
"[extended_bounds.max][" + max + "] for histogram aggregation [" + aggName + "]", null);
throw new IllegalArgumentException("[extended_bounds.min][" + min + "] cannot be greater than " +
"[extended_bounds.max][" + max + "] for histogram aggregation [" + aggName + "]");
}
return new ExtendedBounds(min, max, minAsStr, maxAsStr);
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -40,7 +41,6 @@ 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;
@ -302,10 +302,12 @@ public class HistogramAggregationBuilder extends ValuesSourceAggregationBuilder<
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new HistogramAggregatorFactory(name, config, interval, offset, order, keyed, minDocCount, minBound, maxBound,
context, parent, subFactoriesBuilder, metaData);
queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -54,11 +55,20 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
+ missing + "]");
}
public HistogramAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, double interval, double offset,
BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
public HistogramAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource> config,
double interval,
double offset,
BucketOrder order,
boolean keyed,
long minDocCount,
double minBound,
double maxBound,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.interval = interval;
this.offset = offset;
this.order = order;
@ -73,14 +83,18 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
if (valuesSource instanceof ValuesSource.Numeric) {
return new NumericHistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
(ValuesSource.Numeric) valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
(ValuesSource.Numeric) valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
} else if (valuesSource instanceof ValuesSource.Range) {
ValuesSource.Range rangeValueSource = (ValuesSource.Range) valuesSource;
if (rangeValueSource.rangeType().isNumeric() == false) {
@ -88,7 +102,7 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
+ rangeValueSource.rangeType().name + "]");
}
return new RangeHistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
(ValuesSource.Range) valuesSource, config.format(), context, parent, pipelineAggregators,
(ValuesSource.Range) valuesSource, config.format(), searchContext, parent, pipelineAggregators,
metaData);
}
else {
@ -98,9 +112,11 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new NumericHistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
null, config.format(), context, parent, pipelineAggregators, metaData);
null, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,7 +36,6 @@ 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.Map;
@ -84,9 +84,11 @@ public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new MissingAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new MissingAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,21 +35,27 @@ import java.util.Map;
public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource> {
public MissingAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, SearchContext context,
public MissingAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected MissingAggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, null, context, parent, pipelineAggregators, metaData);
protected MissingAggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, null, searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected MissingAggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, valuesSource, context, parent, pipelineAggregators, metaData);
protected MissingAggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, valuesSource, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -25,12 +25,12 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -87,23 +87,25 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
ObjectMapper childObjectMapper = context.getObjectMapper(path);
protected AggregatorFactory doBuild(QueryShardContext queryShardContext,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
ObjectMapper childObjectMapper = queryShardContext.getObjectMapper(path);
if (childObjectMapper == null) {
// in case the path has been unmapped:
return new NestedAggregatorFactory(name, null, null, context, parent, subFactoriesBuilder, metaData);
return new NestedAggregatorFactory(name, null, null, queryShardContext,
parent, subFactoriesBuilder, metaData);
}
if (childObjectMapper.nested().isNested() == false) {
throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested");
}
try {
ObjectMapper parentObjectMapper = context.getQueryShardContext().nestedScope().nextLevel(childObjectMapper);
return new NestedAggregatorFactory(name, parentObjectMapper, childObjectMapper, context, parent, subFactoriesBuilder,
metaData);
ObjectMapper parentObjectMapper = queryShardContext.nestedScope().nextLevel(childObjectMapper);
return new NestedAggregatorFactory(name, parentObjectMapper, childObjectMapper, queryShardContext,
parent, subFactoriesBuilder, metaData);
} finally {
context.getQueryShardContext().nestedScope().previousLevel();
queryShardContext.nestedScope().previousLevel();
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -38,27 +39,33 @@ public class NestedAggregatorFactory extends AggregatorFactory {
private final ObjectMapper childObjectMapper;
NestedAggregatorFactory(String name, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
QueryShardContext queryShardContext, AggregatorFactory parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactories, metaData);
this.parentObjectMapper = parentObjectMapper;
this.childObjectMapper = childObjectMapper;
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (childObjectMapper == null) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
return new Unmapped(name, searchContext, parent, pipelineAggregators, metaData);
}
return new NestedAggregator(name, factories, parentObjectMapper, childObjectMapper, context, parent,
return new NestedAggregator(name, factories, parentObjectMapper, childObjectMapper, searchContext, parent,
pipelineAggregators, metaData, collectsFromSingleBucket);
}
private static final class Unmapped extends NonCollectingAggregator {
Unmapped(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
Unmapped(String name,
SearchContext context,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}

View File

@ -25,14 +25,13 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.NestedScope;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -92,28 +91,27 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
if (findNestedAggregatorFactory(parent) == null) {
throw new SearchParseException(context,
"Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation", null);
throw new IllegalArgumentException("Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation");
}
ObjectMapper parentObjectMapper = null;
if (path != null) {
parentObjectMapper = context.getObjectMapper(path);
parentObjectMapper = queryShardContext.getObjectMapper(path);
if (parentObjectMapper == null) {
return new ReverseNestedAggregatorFactory(name, true, null, context, parent, subFactoriesBuilder, metaData);
return new ReverseNestedAggregatorFactory(name, true, null, queryShardContext, parent, subFactoriesBuilder, metaData);
}
if (parentObjectMapper.nested().isNested() == false) {
throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested");
}
}
NestedScope nestedScope = context.getQueryShardContext().nestedScope();
NestedScope nestedScope = queryShardContext.nestedScope();
try {
nestedScope.nextLevel(parentObjectMapper);
return new ReverseNestedAggregatorFactory(name, false, parentObjectMapper, context, parent, subFactoriesBuilder,
return new ReverseNestedAggregatorFactory(name, false, parentObjectMapper, queryShardContext, parent, subFactoriesBuilder,
metaData);
} finally {
nestedScope.previousLevel();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.index.mapper.ObjectMapper;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -38,28 +39,35 @@ public class ReverseNestedAggregatorFactory extends AggregatorFactory {
private final ObjectMapper parentObjectMapper;
public ReverseNestedAggregatorFactory(String name, boolean unmapped, ObjectMapper parentObjectMapper,
SearchContext context, AggregatorFactory parent,
QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
super(name, queryShardContext, parent, subFactories, metaData);
this.unmapped = unmapped;
this.parentObjectMapper = parentObjectMapper;
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (unmapped) {
return new Unmapped(name, context, parent, pipelineAggregators, metaData);
return new Unmapped(name, searchContext, parent, pipelineAggregators, metaData);
} else {
return new ReverseNestedAggregator(name, factories, parentObjectMapper, context, parent, pipelineAggregators, metaData);
return new ReverseNestedAggregator(name, factories, parentObjectMapper,
searchContext, parent, pipelineAggregators, metaData);
}
}
private static final class Unmapped extends NonCollectingAggregator {
Unmapped(String name, SearchContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
Unmapped(String name,
SearchContext context,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -41,25 +42,37 @@ public class AbstractRangeAggregatorFactory<R extends Range> extends ValuesSourc
private final R[] ranges;
private final boolean keyed;
public AbstractRangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, R[] ranges, boolean keyed,
InternalRange.Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
public AbstractRangeAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
R[] ranges,
boolean keyed,
InternalRange.Factory<?, ?> rangeFactory,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.ranges = ranges;
this.keyed = keyed;
this.rangeFactory = rangeFactory;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new Unmapped<>(name, ranges, keyed, config.format(), context, parent, rangeFactory, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new Unmapped<>(name, ranges, keyed, config.format(), searchContext, parent, rangeFactory, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, context, parent,
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, searchContext, parent,
pipelineAggregators, metaData);
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -40,30 +41,30 @@ public class BinaryRangeAggregatorFactory
public BinaryRangeAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.Bytes> config,
List<BinaryRangeAggregator.Range> ranges, boolean keyed,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent, Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.ranges = ranges;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new BinaryRangeAggregator(name, factories, null, config.format(),
ranges, keyed, context, parent, pipelineAggregators, metaData);
ranges, keyed, searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Bytes valuesSource,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
SearchContext searchContext, Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new BinaryRangeAggregator(name, factories, valuesSource, config.format(),
ranges, keyed, context, parent, pipelineAggregators, metaData);
ranges, keyed, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -29,7 +30,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactory;
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;
import java.time.ZonedDateTime;
@ -288,8 +288,8 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
}
@Override
protected DateRangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected DateRangeAggregatorFactory innerBuild(QueryShardContext queryShardContext, 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
RangeAggregator.Range[] ranges = processRanges(range -> {
@ -300,23 +300,23 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
String fromAsString = range.getFromAsString();
String toAsString = range.getToAsString();
if (fromAsString != null) {
from = parser.parseDouble(fromAsString, false, context.getQueryShardContext()::nowInMillis);
from = parser.parseDouble(fromAsString, false, queryShardContext::nowInMillis);
} else if (Double.isFinite(from)) {
// from/to provided as double should be converted to string and parsed regardless to support
// different formats like `epoch_millis` vs. `epoch_second` with numeric input
from = parser.parseDouble(Long.toString((long) from), false, context.getQueryShardContext()::nowInMillis);
from = parser.parseDouble(Long.toString((long) from), false, queryShardContext::nowInMillis);
}
if (toAsString != null) {
to = parser.parseDouble(toAsString, false, context.getQueryShardContext()::nowInMillis);
to = parser.parseDouble(toAsString, false, queryShardContext::nowInMillis);
} else if (Double.isFinite(to)) {
to = parser.parseDouble(Long.toString((long) to), false, context.getQueryShardContext()::nowInMillis);
to = parser.parseDouble(Long.toString((long) to), false, queryShardContext::nowInMillis);
}
return new RangeAggregator.Range(range.getKey(), from, fromAsString, to, toAsString);
});
if (ranges.length == 0) {
throw new IllegalArgumentException("No [ranges] specified for the [" + this.getName() + "] aggregation");
}
return new DateRangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
return new DateRangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, queryShardContext, parent, subFactoriesBuilder,
metaData);
}
}

View File

@ -19,21 +19,27 @@
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory<RangeAggregator.Range> {
public DateRangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, RangeAggregator.Range[] ranges, boolean keyed,
InternalRange.Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
public DateRangeAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
RangeAggregator.Range[] ranges,
boolean keyed,
InternalRange.Factory<?, ?> rangeFactory,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, ranges, keyed, rangeFactory, queryShardContext, parent, subFactoriesBuilder, metaData);
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.common.xcontent.XContentParserUtils;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -39,7 +40,6 @@ 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.ValuesSourceParserHelper;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -411,14 +411,15 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
Range[] ranges = this.ranges.toArray(new Range[this.range().size()]);
if (ranges.length == 0) {
throw new IllegalArgumentException("No [ranges] specified for the [" + this.getName() + "] aggregation");
}
return new GeoDistanceRangeAggregatorFactory(name, config, origin, ranges, unit, distanceType, keyed, context, parent,
return new GeoDistanceRangeAggregatorFactory(name, config, origin, ranges, unit, distanceType, keyed, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -53,9 +54,9 @@ public class GeoDistanceRangeAggregatorFactory
private final boolean keyed;
public GeoDistanceRangeAggregatorFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, GeoPoint origin,
Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, SearchContext context,
Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.origin = origin;
this.ranges = ranges;
this.unit = unit;
@ -64,17 +65,23 @@ public class GeoDistanceRangeAggregatorFactory
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new RangeAggregator.Unmapped<>(name, ranges, keyed, config.format(), context, parent, rangeFactory, pipelineAggregators,
metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new RangeAggregator.Unmapped<>(name, ranges, keyed, config.format(), searchContext, parent,
rangeFactory, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
return new RangeAggregator(name, factories, distanceSource, config.format(), rangeFactory, ranges, keyed, context,
return new RangeAggregator(name, factories, distanceSource, config.format(), rangeFactory, ranges, keyed, searchContext,
parent,
pipelineAggregators, metaData);
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -42,7 +43,6 @@ 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;
@ -365,8 +365,8 @@ public final class IpRangeAggregationBuilder
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.Bytes> innerBuild(
SearchContext context, ValuesSourceConfig<ValuesSource.Bytes> config,
AggregatorFactory parent, Builder subFactoriesBuilder)
QueryShardContext queryShardContext, ValuesSourceConfig<ValuesSource.Bytes> config,
AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
List<BinaryRangeAggregator.Range> ranges = new ArrayList<>();
if(this.ranges.size() == 0){
@ -376,7 +376,7 @@ public final class IpRangeAggregationBuilder
ranges.add(new BinaryRangeAggregator.Range(range.key, toBytesRef(range.from), toBytesRef(range.to)));
}
return new BinaryRangeAggregatorFactory(name, config, ranges,
keyed, context, parent, subFactoriesBuilder, metaData);
keyed, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -22,6 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -30,7 +31,6 @@ import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
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;
import java.util.Map;
@ -146,8 +146,8 @@ public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregati
}
@Override
protected RangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected RangeAggregatorFactory innerBuild(QueryShardContext queryShardContext, 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(range -> {
DocValueFormat parser = config.format();
@ -155,17 +155,17 @@ public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregati
Double from = range.from;
Double to = range.to;
if (range.fromAsStr != null) {
from = parser.parseDouble(range.fromAsStr, false, context.getQueryShardContext()::nowInMillis);
from = parser.parseDouble(range.fromAsStr, false, queryShardContext::nowInMillis);
}
if (range.toAsStr != null) {
to = parser.parseDouble(range.toAsStr, false, context.getQueryShardContext()::nowInMillis);
to = parser.parseDouble(range.toAsStr, false, queryShardContext::nowInMillis);
}
return new Range(range.key, from, range.fromAsStr, to, range.toAsStr);
});
if (ranges.length == 0) {
throw new IllegalArgumentException("No [ranges] specified for the [" + this.getName() + "] aggregation");
}
return new RangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
return new RangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, queryShardContext, parent, subFactoriesBuilder,
metaData);
}

View File

@ -19,13 +19,13 @@
package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
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.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -33,9 +33,9 @@ import java.util.Map;
public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory<RangeAggregator.Range> {
public RangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
Factory<?, ?> rangeFactory, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, ranges, keyed, rangeFactory, queryShardContext, parent, subFactoriesBuilder, metaData);
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -33,7 +34,6 @@ 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.Map;
@ -148,9 +148,11 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new DiversifiedAggregatorFactory(name, config, shardSize, maxDocsPerValue, executionHint, context, parent,
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new DiversifiedAggregatorFactory(name, config, shardSize, maxDocsPerValue, executionHint, queryShardContext, parent,
subFactoriesBuilder, metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -44,20 +45,24 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
private final String executionHint;
DiversifiedAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
String executionHint, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
String executionHint, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.shardSize = shardSize;
this.maxDocsPerValue = maxDocsPerValue;
this.executionHint = executionHint;
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (valuesSource instanceof ValuesSource.Numeric) {
return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData,
return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, searchContext, parent, pipelineAggregators, metaData,
(Numeric) valuesSource, maxDocsPerValue);
}
@ -75,7 +80,7 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
if ((execution.needsGlobalOrdinals()) && (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals))) {
execution = ExecutionMode.MAP;
}
return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, pipelineAggregators,
return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, searchContext, parent, pipelineAggregators,
metaData);
}
@ -84,11 +89,13 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final UnmappedSampler aggregation = new UnmappedSampler(name, pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, factories, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, factories, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;

View File

@ -24,11 +24,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -84,9 +84,9 @@ public class SamplerAggregationBuilder extends AbstractAggregationBuilder<Sample
}
@Override
protected SamplerAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder)
protected SamplerAggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent, Builder subFactoriesBuilder)
throws IOException {
return new SamplerAggregatorFactory(name, shardSize, context, parent, subFactoriesBuilder, metaData);
return new SamplerAggregatorFactory(name, shardSize, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -33,16 +34,19 @@ public class SamplerAggregatorFactory extends AggregatorFactory {
private final int shardSize;
SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
SamplerAggregatorFactory(String name, int shardSize, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactories, metaData);
this.shardSize = shardSize;
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new SamplerAggregator(name, shardSize, factories, context, parent, pipelineAggregators, metaData);
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new SamplerAggregator(name, shardSize, factories, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -46,7 +47,6 @@ 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.Map;
@ -284,11 +284,13 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context);
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(queryShardContext);
return new SignificantTermsAggregatorFactory(name, config, includeExclude, executionHint, filterBuilder,
bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder, metaData);
bucketCountThresholds, executionHeuristic, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.lucene.index.FilterableTermsEnum;
import org.elasticsearch.common.lucene.index.FreqTermsEnum;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
@ -81,11 +82,11 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
QueryBuilder filterBuilder,
TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
if (!config.unmapped()) {
this.fieldType = config.fieldContext().fieldType();
@ -96,8 +97,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
this.executionHint = executionHint;
this.filter = filterBuilder == null
? null
: filterBuilder.toQuery(context.getQueryShardContext());
IndexSearcher searcher = context.searcher();
: filterBuilder.toQuery(queryShardContext);
IndexSearcher searcher = queryShardContext.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
@ -118,9 +119,9 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
if (termsEnum != null) {
return termsEnum;
}
IndexReader reader = context.searcher().getIndexReader();
IndexReader reader = queryShardContext.getIndexReader();
if (numberOfAggregatorsCreated > 1) {
termsEnum = new FreqTermsEnum(reader, field, true, false, filter, context.bigArrays());
termsEnum = new FreqTermsEnum(reader, field, true, false, filter, queryShardContext.bigArrays());
} else {
termsEnum = new FilterableTermsEnum(reader, indexedFieldName, PostingsEnum.NONE, filter);
}
@ -128,7 +129,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
private long getBackgroundFrequency(String value) throws IOException {
Query query = fieldType.termQuery(value, context.getQueryShardContext());
Query query = fieldType.termQuery(value, queryShardContext);
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
@ -147,7 +148,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
.add(filter, Occur.FILTER)
.build();
}
return context.searcher().count(query);
return queryShardContext.searcher().count(query);
}
public long getBackgroundFrequency(BytesRef termBytes) throws IOException {
@ -161,11 +162,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -174,10 +177,14 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
numberOfAggregatorsCreated++;
@ -218,7 +225,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
+ "include/exclude clauses");
}
return execution.create(name, factories, valuesSource, format, bucketCountThresholds, includeExclude, context, parent,
return execution.create(name, factories, valuesSource, format, bucketCountThresholds, includeExclude, searchContext, parent,
significanceHeuristic, this, pipelineAggregators, metaData);
}
@ -238,7 +245,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
longFilter = includeExclude.convertToLongFilter(config.format());
}
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
bucketCountThresholds, context, parent, significanceHeuristic, this, longFilter, pipelineAggregators,
bucketCountThresholds, searchContext, parent, significanceHeuristic, this, longFilter, pipelineAggregators,
metaData);
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
@ -40,7 +41,6 @@ import org.elasticsearch.search.aggregations.bucket.terms.IncludeExclude;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Arrays;
@ -340,12 +340,12 @@ public class SignificantTextAggregationBuilder extends AbstractAggregationBuilde
}
@Override
protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context);
protected AggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(queryShardContext);
return new SignificantTextAggregatorFactory(name, includeExclude, filterBuilder,
bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder,
bucketCountThresholds, executionHeuristic, queryShardContext, parent, subFactoriesBuilder,
fieldName, sourceFieldNames, filterDuplicateText, metaData);
}

View File

@ -33,6 +33,7 @@ import org.elasticsearch.common.lucene.index.FilterableTermsEnum;
import org.elasticsearch.common.lucene.index.FreqTermsEnum;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -65,16 +66,23 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
private final DocValueFormat format = DocValueFormat.RAW;
private final boolean filterDuplicateText;
public SignificantTextAggregatorFactory(String name, IncludeExclude includeExclude,
QueryBuilder filterBuilder, TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, String fieldName, String [] sourceFieldNames,
boolean filterDuplicateText, Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactoriesBuilder, metaData);
public SignificantTextAggregatorFactory(String name,
IncludeExclude includeExclude,
QueryBuilder filterBuilder,
TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
String fieldName,
String [] sourceFieldNames,
boolean filterDuplicateText,
Map<String, Object> metaData) throws IOException {
super(name, queryShardContext, parent, subFactoriesBuilder, metaData);
// Note that if the field is unmapped (its field type is null), we don't fail,
// and just use the given field name as a placeholder.
this.fieldType = context.getQueryShardContext().fieldMapper(fieldName);
this.fieldType = queryShardContext.fieldMapper(fieldName);
this.indexedFieldName = fieldType != null ? fieldType.name() : fieldName;
this.sourceFieldNames = sourceFieldNames == null
? new String[] { indexedFieldName }
@ -83,9 +91,9 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
this.includeExclude = includeExclude;
this.filter = filterBuilder == null
? null
: filterBuilder.toQuery(context.getQueryShardContext());
: filterBuilder.toQuery(queryShardContext);
this.filterDuplicateText = filterDuplicateText;
IndexSearcher searcher = context.searcher();
IndexSearcher searcher = queryShardContext.searcher();
// Important - need to use the doc count that includes deleted docs
// or we have this issue: https://github.com/elastic/elasticsearch/issues/7951
this.supersetNumDocs = filter == null
@ -106,9 +114,9 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
if (termsEnum != null) {
return termsEnum;
}
IndexReader reader = context.searcher().getIndexReader();
IndexReader reader = queryShardContext.getIndexReader();
if (numberOfAggregatorsCreated > 1) {
termsEnum = new FreqTermsEnum(reader, field, true, false, filter, context.bigArrays());
termsEnum = new FreqTermsEnum(reader, field, true, false, filter, queryShardContext.bigArrays());
} else {
termsEnum = new FilterableTermsEnum(reader, indexedFieldName, PostingsEnum.NONE, filter);
}
@ -116,7 +124,7 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
}
private long getBackgroundFrequency(String value) throws IOException {
Query query = fieldType.termQuery(value, context.getQueryShardContext());
Query query = fieldType.termQuery(value, queryShardContext);
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
@ -135,7 +143,7 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
.add(filter, Occur.FILTER)
.build();
}
return context.searcher().count(query);
return queryShardContext.searcher().count(query);
}
public long getBackgroundFrequency(BytesRef termBytes) throws IOException {
@ -156,11 +164,11 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
}
@Override
protected Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
protected Aggregator createInternal(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
numberOfAggregatorsCreated++;
@ -183,7 +191,7 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory
IncludeExclude.StringFilter incExcFilter = includeExclude == null ? null:
includeExclude.convertToStringFilter(DocValueFormat.RAW);
return new SignificantTextAggregator(name, factories, context, parent, pipelineAggregators, bucketCountThresholds,
return new SignificantTextAggregator(name, factories, searchContext, parent, pipelineAggregators, bucketCountThresholds,
incExcFilter, significanceHeuristic, this, indexedFieldName, sourceFieldNames, filterDuplicateText, metaData);
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.SignificantTermsHeuristicScoreScript;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.HashMap;
@ -101,9 +100,8 @@ public class ScriptHeuristic extends SignificanceHeuristic {
}
@Override
public SignificanceHeuristic rewrite(SearchContext context) {
QueryShardContext shardContext = context.getQueryShardContext();
SignificantTermsHeuristicScoreScript.Factory compiledScript = shardContext.getScriptService().compile(script,
public SignificanceHeuristic rewrite(QueryShardContext queryShardContext) {
SignificantTermsHeuristicScoreScript.Factory compiledScript = queryShardContext.getScriptService().compile(script,
SignificantTermsHeuristicScoreScript.CONTEXT);
return new ExecutableScriptHeuristic(script, compiledScript.newInstance());
}

View File

@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms;
import org.elasticsearch.search.internal.SearchContext;
/**
* Heuristic for that {@link SignificantTerms} uses to pick out significant terms.
@ -65,10 +65,10 @@ public abstract class SignificanceHeuristic implements NamedWriteable, ToXConten
/**
* Provides a hook for subclasses to provide a version of the heuristic
* prepared for execution on data on a shard.
* @param context the search context on the data node
* @param queryShardContext the shard context on the data node
* @return a version of this heuristic suitable for execution
*/
public SignificanceHeuristic rewrite(SearchContext context) {
public SignificanceHeuristic rewrite(QueryShardContext queryShardContext) {
return this;
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,7 +36,6 @@ 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.Map;
@ -162,12 +162,12 @@ public class RareTermsAggregationBuilder extends ValuesSourceAggregationBuilder<
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new RareTermsAggregatorFactory(name, config, includeExclude,
context, parent, subFactoriesBuilder, metaData, maxDocCount, precision);
queryShardContext, parent, subFactoriesBuilder, metaData, maxDocCount, precision);
}
@Override

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
@ -45,20 +46,22 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
RareTermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config,
IncludeExclude includeExclude,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData, int maxDocCount, double precision) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.includeExclude = includeExclude;
this.maxDocCount = maxDocCount;
this.precision = precision;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedRareTerms(name, pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, factories, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, factories, pipelineAggregators, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -67,10 +70,14 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = ExecutionMode.MAP; //TODO global ords not implemented yet, only supports "map"
@ -83,7 +90,7 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
}
return execution.create(name, factories, valuesSource, format,
includeExclude, context, parent, pipelineAggregators, metaData, maxDocCount, precision);
includeExclude, searchContext, parent, pipelineAggregators, metaData, maxDocCount, precision);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
@ -101,7 +108,7 @@ public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
longFilter = includeExclude.convertToLongFilter(config.format());
}
return new LongRareTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
context, parent, longFilter, maxDocCount, precision, pipelineAggregators, metaData);
searchContext, parent, longFilter, maxDocCount, precision, pipelineAggregators, metaData);
}
throw new AggregationExecutionException("RareTerms aggregation cannot be applied to field [" + config.fieldContext().field()

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -42,7 +43,6 @@ 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;
@ -334,10 +334,12 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<ValuesSource> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
return new TermsAggregatorFactory(name, config, order, includeExclude, executionHint, collectMode,
bucketCountThresholds, showTermDocCountError, context, parent, subFactoriesBuilder, metaData);
bucketCountThresholds, showTermDocCountError, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -23,6 +23,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
@ -66,11 +67,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
SubAggCollectionMode collectMode,
TermsAggregator.BucketCountThresholds bucketCountThresholds,
boolean showTermDocCountError,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.order = order;
this.includeExclude = includeExclude;
this.executionHint = executionHint;
@ -80,11 +81,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(),
bucketCountThresholds.getMinDocCount(), pipelineAggregators, metaData);
return new NonCollectingAggregator(name, context, parent, factories, pipelineAggregators, metaData) {
return new NonCollectingAggregator(name, searchContext, parent, factories, pipelineAggregators, metaData) {
{
// even in the case of an unmapped aggregator, validate the
// order
@ -110,10 +113,14 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
if (InternalOrder.isKeyOrder(order) == false
@ -133,7 +140,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals == false) {
execution = ExecutionMode.MAP;
}
final long maxOrd = execution == ExecutionMode.GLOBAL_ORDINALS ? getMaxOrd(valuesSource, context.searcher()) : -1;
final long maxOrd = execution == ExecutionMode.GLOBAL_ORDINALS ? getMaxOrd(valuesSource, searchContext.searcher()) : -1;
if (execution == null) {
execution = ExecutionMode.GLOBAL_ORDINALS;
}
@ -152,8 +159,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
+ "include/exclude clauses");
}
return execution.create(name, factories, valuesSource, order, format, bucketCountThresholds, includeExclude, context, parent,
cm, showTermDocCountError, pipelineAggregators, metaData);
return execution.create(name, factories, valuesSource, order, format,
bucketCountThresholds, includeExclude, searchContext, parent, cm, showTermDocCountError, pipelineAggregators, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
@ -177,14 +184,14 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
longFilter = includeExclude.convertToDoubleFilter();
}
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, context, parent, cm, showTermDocCountError, longFilter,
bucketCountThresholds, searchContext, parent, cm, showTermDocCountError, longFilter,
pipelineAggregators, metaData);
}
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter(config.format());
}
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order,
bucketCountThresholds, context, parent, cm, showTermDocCountError, longFilter, pipelineAggregators,
bucketCountThresholds, searchContext, parent, cm, showTermDocCountError, longFilter, pipelineAggregators,
metaData);
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -78,9 +78,9 @@ public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected AvgAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new AvgAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected AvgAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new AvgAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,20 +36,26 @@ import java.util.Map;
class AvgAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
AvgAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
AvgAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new AvgAggregator(name, null, config.format(), context, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, null, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,7 +36,6 @@ 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;
import java.util.Map;
@ -124,9 +124,9 @@ public final class CardinalityAggregationBuilder
}
@Override
protected CardinalityAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new CardinalityAggregatorFactory(name, config, precisionThreshold, context, parent, subFactoriesBuilder, metaData);
protected CardinalityAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new CardinalityAggregatorFactory(name, config, precisionThreshold, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -36,23 +37,32 @@ class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesS
private final Long precisionThreshold;
CardinalityAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, Long precisionThreshold,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
CardinalityAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config,
Long precisionThreshold,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.precisionThreshold = precisionThreshold;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new CardinalityAggregator(name, null, precision(), context, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, null, precision(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, valuesSource, precision(), context, parent, pipelineAggregators,
protected Aggregator doCreateInternal(ValuesSource valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, valuesSource, precision(), searchContext, parent, pipelineAggregators,
metaData);
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -98,9 +98,9 @@ public class ExtendedStatsAggregationBuilder
}
@Override
protected ExtendedStatsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new ExtendedStatsAggregatorFactory(name, config, sigma, context, parent, subFactoriesBuilder, metaData);
protected ExtendedStatsAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new ExtendedStatsAggregatorFactory(name, config, sigma, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -37,22 +38,34 @@ class ExtendedStatsAggregatorFactory extends ValuesSourceAggregatorFactory<Value
private final double sigma;
ExtendedStatsAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double sigma,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
ExtendedStatsAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
double sigma,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.sigma = sigma;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new ExtendedStatsAggregator(name, null, config.format(), context, parent, sigma, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new ExtendedStatsAggregator(name, null, config.format(), searchContext,
parent, sigma, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new ExtendedStatsAggregator(name, valuesSource, config.format(), context, parent, sigma, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new ExtendedStatsAggregator(name, valuesSource, config.format(), searchContext,
parent, sigma, pipelineAggregators, metaData);
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -33,7 +34,6 @@ 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;
import java.util.Map;
@ -98,9 +98,9 @@ public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<
}
@Override
protected GeoBoundsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new GeoBoundsAggregatorFactory(name, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
protected GeoBoundsAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new GeoBoundsAggregatorFactory(name, config, wrapLongitude, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -36,22 +37,32 @@ class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSou
private final boolean wrapLongitude;
GeoBoundsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
GeoBoundsAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.GeoPoint> config,
boolean wrapLongitude,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.wrapLongitude = wrapLongitude;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoBoundsAggregator(name, context, parent, null, wrapLongitude, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, searchContext, parent, null, wrapLongitude, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, context, parent, valuesSource, wrapLongitude, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, searchContext, parent, valuesSource, wrapLongitude, pipelineAggregators, metaData);
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -33,7 +34,6 @@ 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;
import java.util.Map;
@ -78,9 +78,9 @@ public class GeoCentroidAggregationBuilder
}
@Override
protected GeoCentroidAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new GeoCentroidAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected GeoCentroidAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new GeoCentroidAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,22 +35,30 @@ import java.util.Map;
class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
GeoCentroidAggregatorFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config,
SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
GeoCentroidAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.GeoPoint> config,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new GeoCentroidAggregator(name, context, parent, null, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new GeoCentroidAggregator(name, searchContext, parent, null, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new GeoCentroidAggregator(name, context, parent, valuesSource, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new GeoCentroidAggregator(name, searchContext, parent, valuesSource, pipelineAggregators, metaData);
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -41,25 +42,32 @@ class HDRPercentileRanksAggregatorFactory
private final boolean keyed;
HDRPercentileRanksAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double[] values,
int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
int numberOfSignificantValueDigits, boolean keyed, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.values = values;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new HDRPercentileRanksAggregator(name, null, context, parent, values, numberOfSignificantValueDigits, keyed,
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new HDRPercentileRanksAggregator(name, null, searchContext, parent, values, numberOfSignificantValueDigits, keyed,
config.format(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new HDRPercentileRanksAggregator(name, valuesSource, context, parent, values, numberOfSignificantValueDigits, keyed,
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new HDRPercentileRanksAggregator(name, valuesSource, searchContext, parent, values, numberOfSignificantValueDigits, keyed,
config.format(), pipelineAggregators, metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -39,26 +40,39 @@ class HDRPercentilesAggregatorFactory extends ValuesSourceAggregatorFactory<Valu
private final int numberOfSignificantValueDigits;
private final boolean keyed;
HDRPercentilesAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double[] percents,
int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
HDRPercentilesAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
double[] percents,
int numberOfSignificantValueDigits,
boolean keyed,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.percents = percents;
this.numberOfSignificantValueDigits = numberOfSignificantValueDigits;
this.keyed = keyed;
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
return new HDRPercentilesAggregator(name, null, context, parent, percents, numberOfSignificantValueDigits, keyed,
return new HDRPercentilesAggregator(name, null, searchContext, parent, percents, numberOfSignificantValueDigits, keyed,
config.format(), pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new HDRPercentilesAggregator(name, valuesSource, context, parent, percents, numberOfSignificantValueDigits, keyed,
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new HDRPercentilesAggregator(name, valuesSource, searchContext, parent, percents, numberOfSignificantValueDigits, keyed,
config.format(), pipelineAggregators, metaData);
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -78,9 +78,9 @@ public class MaxAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected MaxAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new MaxAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected MaxAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new MaxAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,21 +36,26 @@ import java.util.Map;
class MaxAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
MaxAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
MaxAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, config, null, context, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, config, null, searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent,
boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MaxAggregator(name, config, valuesSource, context, parent, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, config, valuesSource, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,7 +36,6 @@ 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.Map;
@ -107,13 +107,13 @@ public class MedianAbsoluteDeviationAggregationBuilder extends LeafOnly<ValuesSo
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.Numeric> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.Numeric> config,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder)
throws IOException {
return new MedianAbsoluteDeviationAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData, compression);
protected ValuesSourceAggregatorFactory<ValuesSource.Numeric> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<ValuesSource.Numeric> config,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder)
throws IOException {
return new MedianAbsoluteDeviationAggregatorFactory(name, config, queryShardContext,
parent, subFactoriesBuilder, metaData, compression);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -38,24 +39,25 @@ public class MedianAbsoluteDeviationAggregatorFactory extends ValuesSourceAggreg
MedianAbsoluteDeviationAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.Numeric> config,
SearchContext context,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData,
double compression) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
this.compression = compression;
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MedianAbsoluteDeviationAggregator(
name,
context,
searchContext,
parent,
pipelineAggregators,
metaData,
@ -67,14 +69,15 @@ public class MedianAbsoluteDeviationAggregatorFactory extends ValuesSourceAggreg
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MedianAbsoluteDeviationAggregator(
name,
context,
searchContext,
parent,
pipelineAggregators,
metaData,

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -78,9 +78,9 @@ public class MinAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected MinAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new MinAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected MinAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new MinAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,20 +36,26 @@ import java.util.Map;
class MinAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
MinAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
MinAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, QueryShardContext queryShardContext,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
return new MinAggregator(name, config, null, context, parent, pipelineAggregators, metaData);
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, config, null, searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, config, valuesSource, context, parent, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, config, valuesSource, searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -37,7 +38,6 @@ 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;
@ -240,14 +240,14 @@ public class PercentileRanksAggregationBuilder extends LeafOnly<ValuesSource.Num
}
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
switch (method) {
case TDIGEST:
return new TDigestPercentileRanksAggregatorFactory(name, config, values, compression, keyed, context, parent,
return new TDigestPercentileRanksAggregatorFactory(name, config, values, compression, keyed, queryShardContext, parent,
subFactoriesBuilder, metaData);
case HDR:
return new HDRPercentileRanksAggregatorFactory(name, config, values, numberOfSignificantValueDigits, keyed, context,
return new HDRPercentileRanksAggregatorFactory(name, config, values, numberOfSignificantValueDigits, keyed, queryShardContext,
parent, subFactoriesBuilder, metaData);
default:
throw new IllegalStateException("Illegal method [" + method + "]");

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -36,7 +37,6 @@ 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;
@ -263,15 +263,17 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
}
@Override
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
protected ValuesSourceAggregatorFactory<Numeric> innerBuild(QueryShardContext queryShardContext,
ValuesSourceConfig<Numeric> config,
AggregatorFactory parent,
Builder subFactoriesBuilder) throws IOException {
switch (method) {
case TDIGEST:
return new TDigestPercentilesAggregatorFactory(name, config, percents, compression, keyed, context, parent,
return new TDigestPercentilesAggregatorFactory(name, config, percents, compression, keyed, queryShardContext, parent,
subFactoriesBuilder, metaData);
case HDR:
return new HDRPercentilesAggregatorFactory(name, config, percents, numberOfSignificantValueDigits, keyed, context, parent,
subFactoriesBuilder, metaData);
return new HDRPercentilesAggregatorFactory(name, config, percents,
numberOfSignificantValueDigits, keyed, queryShardContext, parent, subFactoriesBuilder, metaData);
default:
throw new IllegalStateException("Illegal method [" + method + "]");
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Collections;
@ -193,8 +192,8 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
}
@Override
protected ScriptedMetricAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent,
Builder subfactoriesBuilder) throws IOException {
protected ScriptedMetricAggregatorFactory doBuild(QueryShardContext queryShardContext, AggregatorFactory parent,
Builder subfactoriesBuilder) throws IOException {
if (combineScript == null) {
throw new IllegalArgumentException("[combineScript] must not be null: [" + name + "]");
@ -204,8 +203,6 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
throw new IllegalArgumentException("[reduceScript] must not be null: [" + name + "]");
}
QueryShardContext queryShardContext = context.getQueryShardContext();
// Extract params from scripts and pass them along to ScriptedMetricAggregatorFactory, since it won't have
// access to them for the scripts it's given precompiled.
@ -233,7 +230,7 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
return new ScriptedMetricAggregatorFactory(name, compiledMapScript, mapScriptParams, compiledInitScript,
initScriptParams, compiledCombineScript, combineScriptParams, reduceScript,
params, queryShardContext.lookup(), context, parent, subfactoriesBuilder, metaData);
params, queryShardContext.lookup(), queryShardContext, parent, subfactoriesBuilder, metaData);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.ScriptedMetricAggContexts;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.script.Script;
@ -53,9 +54,9 @@ class ScriptedMetricAggregatorFactory extends AggregatorFactory {
ScriptedMetricAggContexts.InitScript.Factory initScript, Map<String, Object> initScriptParams,
ScriptedMetricAggContexts.CombineScript.Factory combineScript,
Map<String, Object> combineScriptParams, Script reduceScript, Map<String, Object> aggParams,
SearchLookup lookup, SearchContext context, AggregatorFactory parent,
SearchLookup lookup, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, context, parent, subFactories, metaData);
super(name, queryShardContext, parent, subFactories, metaData);
this.mapScript = mapScript;
this.mapScriptParams = mapScriptParams;
this.initScript = initScript;
@ -68,14 +69,17 @@ class ScriptedMetricAggregatorFactory extends AggregatorFactory {
}
@Override
public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
public Aggregator createInternal(SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
return asMultiBucketAggregator(this, searchContext, parent);
}
Map<String, Object> aggParams = this.aggParams;
if (aggParams != null) {
aggParams = deepCopyParams(aggParams, context);
aggParams = deepCopyParams(aggParams, searchContext);
} else {
aggParams = new HashMap<>();
}
@ -89,13 +93,13 @@ class ScriptedMetricAggregatorFactory extends AggregatorFactory {
final ScriptedMetricAggContexts.CombineScript combineScript = this.combineScript.newInstance(
mergeParams(aggParams, combineScriptParams), aggState);
final Script reduceScript = deepCopyScript(this.reduceScript, context, aggParams);
final Script reduceScript = deepCopyScript(this.reduceScript, searchContext, aggParams);
if (initScript != null) {
initScript.execute();
CollectionUtils.ensureNoSelfReferences(aggState, "Scripted metric aggs init script");
}
return new ScriptedMetricAggregator(name, mapScript,
combineScript, reduceScript, aggState, context, parent,
combineScript, reduceScript, aggState, searchContext, parent,
pipelineAggregators, metaData);
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -79,9 +79,9 @@ public class StatsAggregationBuilder extends ValuesSourceAggregationBuilder.Leaf
}
@Override
protected StatsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new StatsAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected StatsAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new StatsAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,20 +36,31 @@ import java.util.Map;
class StatsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
StatsAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
StatsAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
return new StatsAggregator(name, null, config.format(), context, parent, pipelineAggregators, metaData);
return new StatsAggregator(name, null, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new StatsAggregator(name, valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new StatsAggregator(name, valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -34,7 +35,6 @@ 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;
import java.util.Map;
@ -78,9 +78,9 @@ public class SumAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
protected SumAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new SumAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
protected SumAggregatorFactory innerBuild(QueryShardContext queryShardContext, ValuesSourceConfig<Numeric> config,
AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException {
return new SumAggregatorFactory(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -35,20 +36,31 @@ import java.util.Map;
class SumAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
SumAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
SumAggregatorFactory(String name,
ValuesSourceConfig<Numeric> config,
QueryShardContext queryShardContext,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData);
}
@Override
protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData)
throws IOException {
return new SumAggregator(name, null, config.format(), context, parent, pipelineAggregators, metaData);
return new SumAggregator(name, null, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new SumAggregator(name, valuesSource, config.format(), context, parent, pipelineAggregators, metaData);
protected Aggregator doCreateInternal(Numeric valuesSource,
SearchContext searchContext,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new SumAggregator(name, valuesSource, config.format(), searchContext, parent, pipelineAggregators, metaData);
}
}

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