Aggregations Refactor: Refactor Top Hits Aggregation

This commit is contained in:
Colin Goodheart-Smithe 2015-12-16 10:22:54 +00:00
parent 1aea0faa86
commit cac2ca845f
42 changed files with 1248 additions and 444 deletions

View File

@ -48,6 +48,7 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.internal.DefaultSearchContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchLocalRequest;
@ -75,17 +76,20 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
private final BigArrays bigArrays;
private final FetchPhase fetchPhase;
@Inject
public TransportValidateQueryAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
TransportService transportService, IndicesService indicesService,
ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
TransportService transportService, IndicesService indicesService, ScriptService scriptService,
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, FetchPhase fetchPhase) {
super(settings, ValidateQueryAction.NAME, threadPool, clusterService, transportService, actionFilters,
indexNameExpressionResolver, ValidateQueryRequest::new, ShardValidateQueryRequest::new, ThreadPool.Names.SEARCH);
this.indicesService = indicesService;
this.scriptService = scriptService;
this.pageCacheRecycler = pageCacheRecycler;
this.bigArrays = bigArrays;
this.fetchPhase = fetchPhase;
}
@Override
@ -171,11 +175,9 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
Engine.Searcher searcher = indexShard.acquireSearcher("validate_query");
DefaultSearchContext searchContext = new DefaultSearchContext(0,
new ShardSearchLocalRequest(request.types(), request.nowInMillis(), request.filteringAliases()),
null, searcher, indexService, indexShard,
scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
SearchService.NO_TIMEOUT
);
new ShardSearchLocalRequest(request.types(), request.nowInMillis(), request.filteringAliases()), null, searcher,
indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(),
parseFieldMatcher, SearchService.NO_TIMEOUT, fetchPhase);
SearchContext.setCurrent(searchContext);
try {
searchContext.parsedQuery(queryShardContext.toQuery(request.query()));

View File

@ -44,6 +44,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.internal.DefaultSearchContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchLocalRequest;
@ -68,17 +69,20 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
private final BigArrays bigArrays;
private final FetchPhase fetchPhase;
@Inject
public TransportExplainAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
TransportService transportService, IndicesService indicesService,
ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
TransportService transportService, IndicesService indicesService, ScriptService scriptService,
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, FetchPhase fetchPhase) {
super(settings, ExplainAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
ExplainRequest::new, ThreadPool.Names.GET);
this.indicesService = indicesService;
this.scriptService = scriptService;
this.pageCacheRecycler = pageCacheRecycler;
this.bigArrays = bigArrays;
this.fetchPhase = fetchPhase;
}
@Override
@ -111,13 +115,10 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
return new ExplainResponse(shardId.getIndex(), request.type(), request.id(), false);
}
SearchContext context = new DefaultSearchContext(
0, new ShardSearchLocalRequest(new String[]{request.type()}, request.nowInMillis, request.filteringAlias()),
null, result.searcher(), indexService, indexShard,
scriptService, pageCacheRecycler,
bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
SearchService.NO_TIMEOUT
);
SearchContext context = new DefaultSearchContext(0,
new ShardSearchLocalRequest(new String[] { request.type() }, request.nowInMillis, request.filteringAlias()), null,
result.searcher(), indexService, indexShard, scriptService, pageCacheRecycler, bigArrays,
threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, SearchService.NO_TIMEOUT, fetchPhase);
SearchContext.setCurrent(context);
try {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.percolator;
import com.carrotsearch.hppc.ObjectObjectAssociativeContainer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReaderContext;
@ -57,6 +58,7 @@ import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
@ -123,13 +125,15 @@ public class PercolateContext extends SearchContext {
private Sort sort;
private final Map<String, FetchSubPhaseContext> subPhaseContexts = new HashMap<>();
private final Map<Class<?>, Collector> queryCollectors = new HashMap<>();
private final FetchPhase fetchPhase;
public PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, IndexShard indexShard,
IndexService indexService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, ScriptService scriptService, Query aliasFilter, ParseFieldMatcher parseFieldMatcher) {
IndexService indexService, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ScriptService scriptService,
Query aliasFilter, ParseFieldMatcher parseFieldMatcher, FetchPhase fetchPhase) {
super(parseFieldMatcher, request);
this.indexShard = indexShard;
this.indexService = indexService;
this.fetchPhase = fetchPhase;
this.fieldDataService = indexService.fieldData();
this.searchShardTarget = searchShardTarget;
this.percolateQueryRegistry = indexShard.percolateRegistry();
@ -635,6 +639,11 @@ public class PercolateContext extends SearchContext {
throw new UnsupportedOperationException();
}
@Override
public FetchPhase fetchPhase() {
return fetchPhase;
}
@Override
public MappedFieldType smartNameFieldType(String name) {
return mapperService().smartNameFieldType(name, types);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.percolator;
import com.carrotsearch.hppc.IntObjectHashMap;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.memory.ExtendedMemoryIndex;
@ -86,6 +87,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.highlight.HighlightPhase;
import org.elasticsearch.search.internal.SearchContext;
@ -127,15 +129,16 @@ public class PercolatorService extends AbstractComponent {
private final CloseableThreadLocal<MemoryIndex> cache;
private final ParseFieldMatcher parseFieldMatcher;
private final FetchPhase fetchPhase;
@Inject
public PercolatorService(Settings settings, IndexNameExpressionResolver indexNameExpressionResolver, IndicesService indicesService,
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays,
HighlightPhase highlightPhase, ClusterService clusterService,
AggregationPhase aggregationPhase, ScriptService scriptService,
MappingUpdatedAction mappingUpdatedAction) {
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, HighlightPhase highlightPhase, ClusterService clusterService,
AggregationPhase aggregationPhase, ScriptService scriptService, MappingUpdatedAction mappingUpdatedAction,
FetchPhase fetchPhase) {
super(settings);
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.fetchPhase = fetchPhase;
this.parseFieldMatcher = new ParseFieldMatcher(settings);
this.indicesService = indicesService;
this.pageCacheRecycler = pageCacheRecycler;
@ -190,10 +193,10 @@ public class PercolatorService extends AbstractComponent {
);
Query aliasFilter = percolateIndexService.aliasFilter(indexShard.getQueryShardContext(), filteringAliases);
SearchShardTarget searchShardTarget = new SearchShardTarget(clusterService.localNode().id(), request.shardId().getIndex(), request.shardId().id());
final PercolateContext context = new PercolateContext(
request, searchShardTarget, indexShard, percolateIndexService, pageCacheRecycler, bigArrays, scriptService, aliasFilter, parseFieldMatcher
);
SearchShardTarget searchShardTarget = new SearchShardTarget(clusterService.localNode().id(), request.shardId().getIndex(),
request.shardId().id());
final PercolateContext context = new PercolateContext(request, searchShardTarget, indexShard, percolateIndexService,
pageCacheRecycler, bigArrays, scriptService, aliasFilter, parseFieldMatcher, fetchPhase);
SearchContext.setCurrent(context);
try {
ParsedDocument parsedDocument = parseRequest(indexShard, request, context, request.shardId().getIndex());

View File

@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectFloatHashMap;
import com.carrotsearch.hppc.ObjectHashSet;
import com.carrotsearch.hppc.ObjectSet;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
@ -561,7 +562,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher;
DefaultSearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, defaultSearchTimeout);
DefaultSearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher,
indexService,
indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
defaultSearchTimeout, fetchPhase);
SearchContext.setCurrent(context);
try {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.internal.SearchContext;
/**
@ -29,8 +30,8 @@ import org.elasticsearch.search.internal.SearchContext;
public class AggregationBinaryParseElement extends AggregationParseElement {
@Inject
public AggregationBinaryParseElement(AggregatorParsers aggregatorParsers) {
super(aggregatorParsers);
public AggregationBinaryParseElement(AggregatorParsers aggregatorParsers, IndicesQueriesRegistry queriesRegistry) {
super(aggregatorParsers, queriesRegistry);
}
@Override

View File

@ -20,6 +20,8 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.internal.SearchContext;
@ -49,15 +51,20 @@ import org.elasticsearch.search.internal.SearchContext;
public class AggregationParseElement implements SearchParseElement {
private final AggregatorParsers aggregatorParsers;
private IndicesQueriesRegistry queriesRegistry;
@Inject
public AggregationParseElement(AggregatorParsers aggregatorParsers) {
public AggregationParseElement(AggregatorParsers aggregatorParsers, IndicesQueriesRegistry queriesRegistry) {
this.aggregatorParsers = aggregatorParsers;
this.queriesRegistry = queriesRegistry;
}
@Override
public void parse(XContentParser parser, SearchContext context) throws Exception {
AggregatorFactories factories = aggregatorParsers.parseAggregators(parser, context);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry);
parseContext.reset(parser);
parseContext.parseFieldMatcher(context.parseFieldMatcher());
AggregatorFactories factories = aggregatorParsers.parseAggregators(parser, parseContext);
context.aggregations(new SearchContextAggregations(factories));
}
}

View File

@ -27,9 +27,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -62,7 +62,7 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
* @return The resolved aggregator factory or {@code null} in case the aggregation should be skipped
* @throws java.io.IOException When parsing fails
*/
AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException;
AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException;
/**
* @return an empty {@link AggregatorFactory} instance for this parser

View File

@ -18,14 +18,14 @@
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.HashMap;
@ -107,37 +107,37 @@ public class AggregatorParsers {
* Parses the aggregation request recursively generating aggregator factories in turn.
*
* @param parser The input xcontent that will be parsed.
* @param context The search context.
* @param parseContext The parse context.
*
* @return The parsed aggregator factories.
*
* @throws IOException When parsing fails for unknown reasons.
*/
public AggregatorFactories parseAggregators(XContentParser parser, SearchContext context) throws IOException {
return parseAggregators(parser, context, 0);
public AggregatorFactories parseAggregators(XContentParser parser, QueryParseContext parseContext) throws IOException {
return parseAggregators(parser, parseContext, 0);
}
private AggregatorFactories parseAggregators(XContentParser parser, SearchContext context, int level) throws IOException {
private AggregatorFactories parseAggregators(XContentParser parser, QueryParseContext parseContext, int level) throws IOException {
Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
XContentParser.Token token = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new SearchParseException(context, "Unexpected token " + token
+ " in [aggs]: aggregations definitions must start with the name of the aggregation.", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [aggs]: aggregations definitions must start with the name of the aggregation.");
}
final String aggregationName = parser.currentName();
if (!validAggMatcher.reset(aggregationName).matches()) {
throw new SearchParseException(context, "Invalid aggregation name [" + aggregationName
+ "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Invalid aggregation name [" + aggregationName
+ "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'");
}
token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new SearchParseException(context, "Aggregation definition for [" + aggregationName + " starts with a [" + token
+ "], expected a [" + XContentParser.Token.START_OBJECT + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Aggregation definition for [" + aggregationName + " starts with a ["
+ token + "], expected a [" + XContentParser.Token.START_OBJECT + "].");
}
AggregatorFactory aggFactory = null;
@ -148,7 +148,8 @@ public class AggregatorParsers {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new SearchParseException(context, "Expected [" + XContentParser.Token.FIELD_NAME + "] under a ["
throw new ParsingException(
parser.getTokenLocation(), "Expected [" + XContentParser.Token.FIELD_NAME + "] under a ["
+ XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]",
parser.getTokenLocation());
}
@ -157,7 +158,8 @@ public class AggregatorParsers {
token = parser.nextToken();
if ("aggregations_binary".equals(fieldName)) {
if (subFactories != null) {
throw new SearchParseException(context, "Found two sub aggregation definitions under [" + aggregationName + "]",
throw new ParsingException(parser.getTokenLocation(),
"Found two sub aggregation definitions under [" + aggregationName + "]",
parser.getTokenLocation());
}
XContentParser binaryParser = null;
@ -165,17 +167,17 @@ public class AggregatorParsers {
byte[] source = parser.binaryValue();
binaryParser = XContentFactory.xContent(source).createParser(source);
} else {
throw new SearchParseException(context, "Expected [" + XContentParser.Token.VALUE_STRING + " or "
+ XContentParser.Token.VALUE_EMBEDDED_OBJECT + "] for [" + fieldName + "], but got a [" + token + "] in ["
+ aggregationName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Expected [" + XContentParser.Token.VALUE_STRING + " or " + XContentParser.Token.VALUE_EMBEDDED_OBJECT
+ "] for [" + fieldName + "], but got a [" + token + "] in [" + aggregationName + "]");
}
XContentParser.Token binaryToken = binaryParser.nextToken();
if (binaryToken != XContentParser.Token.START_OBJECT) {
throw new SearchParseException(context, "Expected [" + XContentParser.Token.START_OBJECT
+ "] as first token when parsing [" + fieldName + "], but got a [" + binaryToken + "] in ["
+ aggregationName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Expected [" + XContentParser.Token.START_OBJECT + "] as first token when parsing [" + fieldName
+ "], but got a [" + binaryToken + "] in [" + aggregationName + "]");
}
subFactories = parseAggregators(binaryParser, context, level + 1);
subFactories = parseAggregators(binaryParser, parseContext, level + 1);
} else if (token == XContentParser.Token.START_OBJECT) {
switch (fieldName) {
case "meta":
@ -184,42 +186,42 @@ public class AggregatorParsers {
case "aggregations":
case "aggs":
if (subFactories != null) {
throw new SearchParseException(context,
"Found two sub aggregation definitions under [" + aggregationName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Found two sub aggregation definitions under [" + aggregationName + "]");
}
subFactories = parseAggregators(parser, context, level + 1);
subFactories = parseAggregators(parser, parseContext, level + 1);
break;
default:
if (aggFactory != null) {
throw new SearchParseException(context, "Found two aggregation type definitions in [" + aggregationName
+ "]: [" + aggFactory.type + "] and [" + fieldName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Found two aggregation type definitions in ["
+ aggregationName + "]: [" + aggFactory.type + "] and [" + fieldName + "]");
}
if (pipelineAggregatorFactory != null) {
throw new SearchParseException(context, "Found two aggregation type definitions in [" + aggregationName
+ "]: [" + pipelineAggregatorFactory + "] and [" + fieldName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Found two aggregation type definitions in ["
+ aggregationName + "]: [" + pipelineAggregatorFactory + "] and [" + fieldName + "]");
}
Aggregator.Parser aggregatorParser = parser(fieldName);
if (aggregatorParser == null) {
PipelineAggregator.Parser pipelineAggregatorParser = pipelineAggregator(fieldName);
if (pipelineAggregatorParser == null) {
throw new SearchParseException(context, "Could not find aggregator type [" + fieldName + "] in ["
+ aggregationName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Could not find aggregator type [" + fieldName + "] in [" + aggregationName + "]");
} else {
pipelineAggregatorFactory = pipelineAggregatorParser.parse(aggregationName, parser, context);
pipelineAggregatorFactory = pipelineAggregatorParser.parse(aggregationName, parser, parseContext);
}
} else {
aggFactory = aggregatorParser.parse(aggregationName, parser, context);
aggFactory = aggregatorParser.parse(aggregationName, parser, parseContext);
}
}
} else {
throw new SearchParseException(context, "Expected [" + XContentParser.Token.START_OBJECT + "] under [" + fieldName
+ "], but got a [" + token + "] in [" + aggregationName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.START_OBJECT + "] under ["
+ fieldName + "], but got a [" + token + "] in [" + aggregationName + "]");
}
}
if (aggFactory == null && pipelineAggregatorFactory == null) {
throw new SearchParseException(context, "Missing definition for aggregation [" + aggregationName + "]",
throw new ParsingException(parser.getTokenLocation(), "Missing definition for aggregation [" + aggregationName + "]",
parser.getTokenLocation());
} else if (aggFactory != null) {
assert pipelineAggregatorFactory == null;
@ -239,7 +241,8 @@ public class AggregatorParsers {
} else {
assert pipelineAggregatorFactory != null;
if (subFactories != null) {
throw new SearchParseException(context, "Aggregation [" + aggregationName + "] cannot define sub-aggregations",
throw new ParsingException(parser.getTokenLocation(),
"Aggregation [" + aggregationName + "] cannot define sub-aggregations",
parser.getTokenLocation());
}
if (level == 0) {

View File

@ -18,11 +18,11 @@
*/
package org.elasticsearch.search.aggregations.bucket.children;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -37,7 +37,7 @@ public class ChildrenParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String childType = null;
XContentParser.Token token;
@ -49,18 +49,17 @@ public class ChildrenParser implements Aggregator.Parser {
if ("type".equals(currentFieldName)) {
childType = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
}
}
if (childType == null) {
throw new SearchParseException(context, "Missing [child_type] field for children aggregation [" + aggregationName + "]",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Missing [child_type] field for children aggregation [" + aggregationName + "]");
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -48,11 +47,8 @@ public class FilterParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
QueryParseContext queryParseContext = new QueryParseContext(queriesRegistry);
queryParseContext.reset(parser);
queryParseContext.parseFieldMatcher(context.parseFieldMatcher());
QueryBuilder<?> filter = queryParseContext.parseInnerQueryBuilder();
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
QueryBuilder<?> filter = context.parseInnerQueryBuilder();
FilterAggregator.Factory factory = new FilterAggregator.Factory(aggregationName);
factory.filter(filter == null ? new MatchAllQueryBuilder() : filter);

View File

@ -20,16 +20,15 @@
package org.elasticsearch.search.aggregations.bucket.filters;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -57,7 +56,7 @@ public class FiltersParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
List<FiltersAggregator.KeyedFilter> keyedFilters = null;
List<QueryBuilder<?>> nonKeyedFilters = null;
@ -73,15 +72,15 @@ public class FiltersParser implements Aggregator.Parser {
if (context.parseFieldMatcher().match(currentFieldName, OTHER_BUCKET_FIELD)) {
otherBucket = parser.booleanValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (context.parseFieldMatcher().match(currentFieldName, OTHER_BUCKET_KEY_FIELD)) {
otherBucketKey = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, FILTERS_FIELD)) {
@ -100,8 +99,8 @@ public class FiltersParser implements Aggregator.Parser {
}
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, FILTERS_FIELD)) {
@ -114,12 +113,12 @@ public class FiltersParser implements Aggregator.Parser {
nonKeyedFilters.add(filter == null ? QueryBuilders.matchAllQuery() : filter);
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -36,7 +36,7 @@ public class GlobalParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
parser.nextToken();
return new GlobalAggregator.Factory(aggregationName);
}

View File

@ -18,11 +18,11 @@
*/
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -37,7 +37,7 @@ public class NestedParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String path = null;
XContentParser.Token token;
@ -49,19 +49,17 @@ public class NestedParser implements Aggregator.Parser {
if (context.parseFieldMatcher().match(currentFieldName, NestedAggregator.PATH_FIELD)) {
path = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
}
}
if (path == null) {
// "field" doesn't exist, so we fall back to the context of the ancestors
throw new SearchParseException(context, "Missing [path] field for nested aggregation [" + aggregationName + "]",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing [path] field for nested aggregation [" + aggregationName + "]");
}
return new NestedAggregator.Factory(aggregationName, path);

View File

@ -18,11 +18,11 @@
*/
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -37,7 +37,7 @@ public class ReverseNestedParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String path = null;
XContentParser.Token token;
@ -49,12 +49,11 @@ public class ReverseNestedParser implements Aggregator.Parser {
if ("path".equals(currentFieldName)) {
path = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
}
}

View File

@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -38,7 +38,7 @@ public class SamplerParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
XContentParser.Token token;
String currentFieldName = null;

View File

@ -1,69 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
/**
*
*/
public abstract class NumericValuesSourceMetricsAggregatorParser<S extends InternalNumericMetricsAggregation> implements Aggregator.Parser {
protected final InternalAggregation.Type aggType;
protected NumericValuesSourceMetricsAggregatorParser(InternalAggregation.Type aggType) {
this.aggType = aggType;
}
@Override
public String type() {
return aggType.name();
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
ValuesSourceParser<ValuesSource.Numeric> vsParser = ValuesSourceParser.numeric(aggregationName, aggType, context).formattable(true)
.build();
XContentParser.Token token;
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (!vsParser.token(currentFieldName, token, parser)) {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
}
}
return createFactory(aggregationName, vsParser.input());
}
protected abstract AggregatorFactory createFactory(String aggregationName, ValuesSourceParser.Input<ValuesSource.Numeric> config);
}

View File

@ -20,14 +20,14 @@
package org.elasticsearch.search.aggregations.metrics.scripted;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptParameterParser;
import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.HashSet;
@ -54,7 +54,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
Script initScript = null;
Script mapScript = null;
Script combineScript = null;
@ -87,17 +87,16 @@ public class ScriptedMetricParser implements Aggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, REDUCE_PARAMS_FIELD)) {
reduceParams = parser.map();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token.isValue()) {
if (!scriptParameterParser.token(currentFieldName, token, parser, context.parseFieldMatcher())) {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
}
}
@ -107,10 +106,8 @@ public class ScriptedMetricParser implements Aggregator.Parser {
initScript = new Script(scriptValue.script(), scriptValue.scriptType(), scriptParameterParser.lang(), params);
}
} else if (initScript.getParams() != null) {
throw new SearchParseException(
context,
"init_script params are not supported. Parameters for the init_script must be specified in the params field on the scripted_metric aggregator not inside the init_script object",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"init_script params are not supported. Parameters for the init_script must be specified in the params field on the scripted_metric aggregator not inside the init_script object");
}
if (mapScript == null) { // Didn't find anything using the new API so try using the old one instead
@ -119,10 +116,8 @@ public class ScriptedMetricParser implements Aggregator.Parser {
mapScript = new Script(scriptValue.script(), scriptValue.scriptType(), scriptParameterParser.lang(), params);
}
} else if (mapScript.getParams() != null) {
throw new SearchParseException(
context,
"map_script params are not supported. Parameters for the map_script must be specified in the params field on the scripted_metric aggregator not inside the map_script object",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"map_script params are not supported. Parameters for the map_script must be specified in the params field on the scripted_metric aggregator not inside the map_script object");
}
if (combineScript == null) { // Didn't find anything using the new API so try using the old one instead
@ -131,10 +126,8 @@ public class ScriptedMetricParser implements Aggregator.Parser {
combineScript = new Script(scriptValue.script(), scriptValue.scriptType(), scriptParameterParser.lang(), params);
}
} else if (combineScript.getParams() != null) {
throw new SearchParseException(
context,
"combine_script params are not supported. Parameters for the combine_script must be specified in the params field on the scripted_metric aggregator not inside the combine_script object",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"combine_script params are not supported. Parameters for the combine_script must be specified in the params field on the scripted_metric aggregator not inside the combine_script object");
}
if (reduceScript == null) { // Didn't find anything using the new API so try using the old one instead
@ -145,7 +138,7 @@ public class ScriptedMetricParser implements Aggregator.Parser {
}
if (mapScript == null) {
throw new SearchParseException(context, "map_script field is required in [" + aggregationName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "map_script field is required in [" + aggregationName + "].");
}
ScriptedMetricAggregator.Factory factory = new ScriptedMetricAggregator.Factory(aggregationName);

View File

@ -30,9 +30,23 @@ import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.TopScoreDocCollector;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.util.LongObjectPagedHashMap;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -43,15 +57,29 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsContext;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsContext.FieldDataField;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHits;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.sort.SortParseElement;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
*/
@ -183,24 +211,574 @@ public class TopHitsAggregator extends MetricsAggregator {
public static class Factory extends AggregatorFactory {
private final FetchPhase fetchPhase;
private final SubSearchContext subSearchContext;
private static final SortParseElement sortParseElement = new SortParseElement();
private int from = 0;
private int size = 3;
private boolean explain = false;
private boolean version = false;
private boolean trackScores = false;
private List<BytesReference> sorts = null;
private HighlightBuilder highlightBuilder;
private List<String> fieldNames;
private List<String> fieldDataFields;
private List<ScriptField> scriptFields;
private FetchSourceContext fetchSourceContext;
public Factory(String name, FetchPhase fetchPhase, SubSearchContext subSearchContext) {
public Factory(String name) {
super(name, InternalTopHits.TYPE);
this.fetchPhase = fetchPhase;
this.subSearchContext = subSearchContext;
}
/**
* From index to start the search from. Defaults to <tt>0</tt>.
*/
public void from(int from) {
this.from = from;
}
/**
* Gets the from index to start the search from.
**/
public int from() {
return from;
}
/**
* The number of search hits to return. Defaults to <tt>10</tt>.
*/
public void size(int size) {
this.size = size;
}
/**
* Gets the number of search hits to return.
*/
public int size() {
return size;
}
/**
* Adds a sort against the given field name and the sort ordering.
*
* @param name
* The name of the field
* @param order
* The sort ordering
*/
public void sort(String name, SortOrder order) {
sort(SortBuilders.fieldSort(name).order(order));
}
/**
* Add a sort against the given field name.
*
* @param name
* The name of the field to sort by
*/
public void sort(String name) {
sort(SortBuilders.fieldSort(name));
}
/**
* Adds a sort builder.
*/
public void sort(SortBuilder sort) {
try {
if (sorts == null) {
sorts = new ArrayList<>();
}
// NORELEASE when sort has been refactored and made writeable
// add the sortBuilcer to the List directly instead of
// serialising to XContent
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
sort.toXContent(builder, EMPTY_PARAMS);
builder.endObject();
sorts.add(builder.bytes());
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* Adds a sort builder.
*/
public void sorts(List<BytesReference> sorts) {
if (this.sorts == null) {
this.sorts = new ArrayList<>();
}
for (BytesReference sort : sorts) {
this.sorts.add(sort);
}
}
/**
* Gets the bytes representing the sort builders for this request.
*/
public List<BytesReference> sorts() {
return sorts;
}
/**
* Adds highlight to perform as part of the search.
*/
public void highlighter(HighlightBuilder highlightBuilder) {
this.highlightBuilder = highlightBuilder;
}
/**
* Gets the hightlighter builder for this request.
*/
public HighlightBuilder highlighter() {
return highlightBuilder;
}
/**
* Indicates whether the response should contain the stored _source for
* every hit
*/
public void fetchSource(boolean fetch) {
if (this.fetchSourceContext == null) {
this.fetchSourceContext = new FetchSourceContext(fetch);
} else {
this.fetchSourceContext.fetchSource(fetch);
}
}
/**
* Indicate that _source should be returned with every hit, with an
* "include" and/or "exclude" set which can include simple wildcard
* elements.
*
* @param include
* An optional include (optionally wildcarded) pattern to
* filter the returned _source
* @param exclude
* An optional exclude (optionally wildcarded) pattern to
* filter the returned _source
*/
public void fetchSource(@Nullable String include, @Nullable String exclude) {
fetchSource(include == null ? Strings.EMPTY_ARRAY : new String[] { include },
exclude == null ? Strings.EMPTY_ARRAY : new String[] { exclude });
}
/**
* Indicate that _source should be returned with every hit, with an
* "include" and/or "exclude" set which can include simple wildcard
* elements.
*
* @param includes
* An optional list of include (optionally wildcarded)
* pattern to filter the returned _source
* @param excludes
* An optional list of exclude (optionally wildcarded)
* pattern to filter the returned _source
*/
public void fetchSource(@Nullable String[] includes, @Nullable String[] excludes) {
fetchSourceContext = new FetchSourceContext(includes, excludes);
}
/**
* Indicate how the _source should be fetched.
*/
public void fetchSource(@Nullable FetchSourceContext fetchSourceContext) {
this.fetchSourceContext = fetchSourceContext;
}
/**
* Gets the {@link FetchSourceContext} which defines how the _source
* should be fetched.
*/
public FetchSourceContext fetchSource() {
return fetchSourceContext;
}
/**
* Adds a field to load and return (note, it must be stored) as part of
* the search request. If none are specified, the source of the document
* will be return.
*/
public void field(String name) {
if (fieldNames == null) {
fieldNames = new ArrayList<>();
}
fieldNames.add(name);
}
/**
* Sets the fields to load and return as part of the search request. If
* none are specified, the source of the document will be returned.
*/
public void fields(List<String> fields) {
this.fieldNames = fields;
}
/**
* Sets no fields to be loaded, resulting in only id and type to be
* returned per field.
*/
public void noFields() {
this.fieldNames = Collections.emptyList();
}
/**
* Gets the fields to load and return as part of the search request.
*/
public List<String> fields() {
return fieldNames;
}
/**
* Adds a field to load from the field data cache and return as part of
* the search request.
*/
public void fieldDataField(String name) {
if (fieldDataFields == null) {
fieldDataFields = new ArrayList<>();
}
fieldDataFields.add(name);
}
/**
* Adds fields to load from the field data cache and return as part of
* the search request.
*/
public void fieldDataFields(List<String> names) {
if (fieldDataFields == null) {
fieldDataFields = new ArrayList<>();
}
fieldDataFields.addAll(names);
}
/**
* Gets the field-data fields.
*/
public List<String> fieldDataFields() {
return fieldDataFields;
}
/**
* Adds a script field under the given name with the provided script.
*
* @param name
* The name of the field
* @param script
* The script
*/
public void scriptField(String name, Script script) {
scriptField(name, script, false);
}
/**
* Adds a script field under the given name with the provided script.
*
* @param name
* The name of the field
* @param script
* The script
*/
public void scriptField(String name, Script script, boolean ignoreFailure) {
if (scriptFields == null) {
scriptFields = new ArrayList<>();
}
scriptFields.add(new ScriptField(name, script, ignoreFailure));
}
public void scriptFields(List<ScriptField> scriptFields) {
if (this.scriptFields == null) {
this.scriptFields = new ArrayList<>();
}
this.scriptFields.addAll(scriptFields);
}
/**
* Gets the script fields.
*/
public List<ScriptField> scriptFields() {
return scriptFields;
}
/**
* Should each {@link org.elasticsearch.search.SearchHit} be returned
* with an explanation of the hit (ranking).
*/
public void explain(boolean explain) {
this.explain = explain;
}
/**
* Indicates whether each search hit will be returned with an
* explanation of the hit (ranking)
*/
public boolean explain() {
return explain;
}
/**
* Should each {@link org.elasticsearch.search.SearchHit} be returned
* with a version associated with it.
*/
public void version(boolean version) {
this.version = version;
}
/**
* Indicates whether the document's version will be included in the
* search hits.
*/
public boolean version() {
return version;
}
/**
* Applies when sorting, and controls if scores will be tracked as well.
* Defaults to <tt>false</tt>.
*/
public void trackScores(boolean trackScores) {
this.trackScores = trackScores;
}
/**
* Indicates whether scores will be tracked for this request.
*/
public boolean trackScores() {
return trackScores;
}
@Override
public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
return new TopHitsAggregator(fetchPhase, subSearchContext, name, aggregationContext, parent, pipelineAggregators, metaData);
SubSearchContext subSearchContext = new SubSearchContext(aggregationContext.searchContext());
subSearchContext.explain(explain);
subSearchContext.version(version);
subSearchContext.trackScores(trackScores);
subSearchContext.from(from);
subSearchContext.size(size);
if (sorts != null) {
XContentParser completeSortParser = null;
try {
XContentBuilder completeSortBuilder = XContentFactory.jsonBuilder();
completeSortBuilder.startObject();
completeSortBuilder.startArray("sort");
for (BytesReference sort : sorts) {
XContentParser parser = XContentFactory.xContent(sort).createParser(sort);
parser.nextToken();
completeSortBuilder.copyCurrentStructure(parser);
}
completeSortBuilder.endArray();
completeSortBuilder.endObject();
BytesReference completeSortBytes = completeSortBuilder.bytes();
completeSortParser = XContentFactory.xContent(completeSortBytes).createParser(completeSortBytes);
completeSortParser.nextToken();
completeSortParser.nextToken();
completeSortParser.nextToken();
sortParseElement.parse(completeSortParser, subSearchContext);
} catch (Exception e) {
XContentLocation location = completeSortParser != null ? completeSortParser.getTokenLocation() : null;
throw new ParsingException(location, "failed to parse sort source in aggregation [" + name + "]", e);
}
}
if (fieldNames != null) {
subSearchContext.fieldNames().addAll(fieldNames);
}
if (fieldDataFields != null) {
FieldDataFieldsContext fieldDataFieldsContext = subSearchContext
.getFetchSubPhaseContext(FieldDataFieldsFetchSubPhase.CONTEXT_FACTORY);
for (String field : fieldDataFields) {
fieldDataFieldsContext.add(new FieldDataField(field));
}
fieldDataFieldsContext.setHitExecutionNeeded(true);
}
if (scriptFields != null) {
for (ScriptField field : scriptFields) {
SearchScript searchScript = subSearchContext.scriptService().search(subSearchContext.lookup(), field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
subSearchContext.scriptFields().add(new org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
}
}
if (fetchSourceContext != null) {
subSearchContext.fetchSourceContext(fetchSourceContext);
}
if (highlightBuilder != null) {
subSearchContext.highlight(highlightBuilder.build(aggregationContext.searchContext().indexShard().getQueryShardContext()));
}
return new TopHitsAggregator(aggregationContext.searchContext().fetchPhase(), subSearchContext, name, aggregationContext,
parent, pipelineAggregators, metaData);
}
@Override
public AggregatorFactory subFactories(AggregatorFactories subFactories) {
throw new AggregationInitializationException("Aggregator [" + name + "] of type [" + type + "] cannot accept sub-aggregations");
}
@Override
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(SearchSourceBuilder.FROM_FIELD.getPreferredName(), from);
builder.field(SearchSourceBuilder.SIZE_FIELD.getPreferredName(), size);
builder.field(SearchSourceBuilder.VERSION_FIELD.getPreferredName(), version);
builder.field(SearchSourceBuilder.EXPLAIN_FIELD.getPreferredName(), explain);
if (fetchSourceContext != null) {
builder.field(SearchSourceBuilder._SOURCE_FIELD.getPreferredName(), fetchSourceContext);
}
if (fieldNames != null) {
if (fieldNames.size() == 1) {
builder.field(SearchSourceBuilder.FIELDS_FIELD.getPreferredName(), fieldNames.get(0));
} else {
builder.startArray(SearchSourceBuilder.FIELDS_FIELD.getPreferredName());
for (String fieldName : fieldNames) {
builder.value(fieldName);
}
builder.endArray();
}
}
if (fieldDataFields != null) {
builder.startArray(SearchSourceBuilder.FIELDDATA_FIELDS_FIELD.getPreferredName());
for (String fieldDataField : fieldDataFields) {
builder.value(fieldDataField);
}
builder.endArray();
}
if (scriptFields != null) {
builder.startObject(SearchSourceBuilder.SCRIPT_FIELDS_FIELD.getPreferredName());
for (ScriptField scriptField : scriptFields) {
scriptField.toXContent(builder, params);
}
builder.endObject();
}
if (sorts != null) {
builder.startArray(SearchSourceBuilder.SORT_FIELD.getPreferredName());
for (BytesReference sort : sorts) {
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(sort);
parser.nextToken();
builder.copyCurrentStructure(parser);
}
builder.endArray();
}
if (trackScores) {
builder.field(SearchSourceBuilder.TRACK_SCORES_FIELD.getPreferredName(), true);
}
if (highlightBuilder != null) {
this.highlightBuilder.toXContent(builder, params);
}
builder.endObject();
return builder;
}
@Override
protected AggregatorFactory doReadFrom(String name, StreamInput in) throws IOException {
Factory factory = new Factory(name);
factory.explain = in.readBoolean();
factory.fetchSourceContext = FetchSourceContext.optionalReadFromStream(in);
if (in.readBoolean()) {
int size = in.readVInt();
List<String> fieldDataFields = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
fieldDataFields.add(in.readString());
}
factory.fieldDataFields = fieldDataFields;
}
if (in.readBoolean()) {
int size = in.readVInt();
List<String> fieldNames = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
fieldNames.add(in.readString());
}
factory.fieldNames = fieldNames;
}
factory.from = in.readVInt();
if (in.readBoolean()) {
factory.highlightBuilder = HighlightBuilder.PROTOTYPE.readFrom(in);
}
if (in.readBoolean()) {
int size = in.readVInt();
List<ScriptField> scriptFields = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
scriptFields.add(ScriptField.PROTOTYPE.readFrom(in));
}
factory.scriptFields = scriptFields;
}
factory.size = in.readVInt();
if (in.readBoolean()) {
int size = in.readVInt();
List<BytesReference> sorts = new ArrayList<>();
for (int i = 0; i < size; i++) {
sorts.add(in.readBytesReference());
}
factory.sorts = sorts;
}
factory.trackScores = in.readBoolean();
factory.version = in.readBoolean();
return factory;
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeBoolean(explain);
FetchSourceContext.optionalWriteToStream(fetchSourceContext, out);
boolean hasFieldDataFields = fieldDataFields != null;
out.writeBoolean(hasFieldDataFields);
if (hasFieldDataFields) {
out.writeVInt(fieldDataFields.size());
for (String fieldName : fieldDataFields) {
out.writeString(fieldName);
}
}
boolean hasFieldNames = fieldNames != null;
out.writeBoolean(hasFieldNames);
if (hasFieldNames) {
out.writeVInt(fieldNames.size());
for (String fieldName : fieldNames) {
out.writeString(fieldName);
}
}
out.writeVInt(from);
boolean hasHighlighter = highlightBuilder != null;
out.writeBoolean(hasHighlighter);
if (hasHighlighter) {
highlightBuilder.writeTo(out);
}
boolean hasScriptFields = scriptFields != null;
out.writeBoolean(hasScriptFields);
if (hasScriptFields) {
out.writeVInt(scriptFields.size());
for (ScriptField scriptField : scriptFields) {
scriptField.writeTo(out);
}
}
out.writeVInt(size);
boolean hasSorts = sorts != null;
out.writeBoolean(hasSorts);
if (hasSorts) {
out.writeVInt(sorts.size());
for (BytesReference sort : sorts) {
out.writeBytesReference(sort);
}
}
out.writeBoolean(trackScores);
out.writeBoolean(version);
}
@Override
protected int doHashCode() {
return Objects.hash(explain, fetchSourceContext, fieldDataFields, fieldNames, from, highlightBuilder, scriptFields, size, sorts,
trackScores, version);
}
@Override
protected boolean doEquals(Object obj) {
Factory other = (Factory) obj;
return Objects.equals(explain, other.explain)
&& Objects.equals(fetchSourceContext, other.fetchSourceContext)
&& Objects.equals(fieldDataFields, other.fieldDataFields)
&& Objects.equals(fieldNames, other.fieldNames)
&& Objects.equals(from, other.from)
&& Objects.equals(highlightBuilder, other.highlightBuilder)
&& Objects.equals(scriptFields, other.scriptFields)
&& Objects.equals(size, other.size)
&& Objects.equals(sorts, other.sorts)
&& Objects.equals(trackScores, other.trackScores)
&& Objects.equals(version, other.version);
}
}
}

View File

@ -18,30 +18,36 @@
*/
package org.elasticsearch.search.aggregations.metrics.tophits;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.FieldsParseElement;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsParseElement;
import org.elasticsearch.search.fetch.script.ScriptFieldsParseElement;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.source.FetchSourceParseElement;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.highlight.HighlighterParseElement;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.sort.SortParseElement;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
*
*/
public class TopHitsParser implements Aggregator.Parser {
private final FetchPhase fetchPhase;
private final SortParseElement sortParseElement;
private final FetchSourceParseElement sourceParseElement;
private final HighlighterParseElement highlighterParseElement;
@ -50,10 +56,9 @@ public class TopHitsParser implements Aggregator.Parser {
private final FieldsParseElement fieldsParseElement;
@Inject
public TopHitsParser(FetchPhase fetchPhase, SortParseElement sortParseElement, FetchSourceParseElement sourceParseElement,
public TopHitsParser(SortParseElement sortParseElement, FetchSourceParseElement sourceParseElement,
HighlighterParseElement highlighterParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement,
ScriptFieldsParseElement scriptFieldsParseElement, FieldsParseElement fieldsParseElement) {
this.fetchPhase = fetchPhase;
this.sortParseElement = sortParseElement;
this.sourceParseElement = sourceParseElement;
this.highlighterParseElement = highlighterParseElement;
@ -68,80 +73,140 @@ public class TopHitsParser implements Aggregator.Parser {
}
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
SubSearchContext subSearchContext = new SubSearchContext(context);
public AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
TopHitsAggregator.Factory factory = new TopHitsAggregator.Factory(aggregationName);
XContentParser.Token token;
String currentFieldName = null;
try {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if ("sort".equals(currentFieldName)) {
sortParseElement.parse(parser, subSearchContext);
} else if ("_source".equals(currentFieldName)) {
sourceParseElement.parse(parser, subSearchContext);
} else if ("fields".equals(currentFieldName)) {
fieldsParseElement.parse(parser, subSearchContext);
} else if (token.isValue()) {
switch (currentFieldName) {
case "from":
subSearchContext.from(parser.intValue());
break;
case "size":
subSearchContext.size(parser.intValue());
break;
case "track_scores":
case "trackScores":
subSearchContext.trackScores(parser.booleanValue());
break;
case "version":
subSearchContext.version(parser.booleanValue());
break;
case "explain":
subSearchContext.explain(parser.booleanValue());
break;
default:
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_OBJECT) {
switch (currentFieldName) {
case "highlight":
highlighterParseElement.parse(parser, subSearchContext);
break;
case "scriptFields":
case "script_fields":
scriptFieldsParseElement.parse(parser, subSearchContext);
break;
default:
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_ARRAY) {
switch (currentFieldName) {
case "fielddataFields":
case "fielddata_fields":
fieldDataFieldsParseElement.parse(parser, subSearchContext);
break;
default:
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
}
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.FROM_FIELD)) {
factory.from(parser.intValue());
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SIZE_FIELD)) {
factory.size(parser.intValue());
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.VERSION_FIELD)) {
factory.version(parser.booleanValue());
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.EXPLAIN_FIELD)) {
factory.explain(parser.booleanValue());
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.TRACK_SCORES_FIELD)) {
factory.trackScores(parser.booleanValue());
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder._SOURCE_FIELD)) {
factory.fetchSource(FetchSourceContext.parse(parser, context));
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.FIELDS_FIELD)) {
List<String> fieldNames = new ArrayList<>();
fieldNames.add(parser.text());
factory.fields(fieldNames);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SORT_FIELD)) {
factory.sort(parser.text());
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].",
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder._SOURCE_FIELD)) {
factory.fetchSource(FetchSourceContext.parse(parser, context));
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SCRIPT_FIELDS_FIELD)) {
List<ScriptField> scriptFields = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
String scriptFieldName = parser.currentName();
token = parser.nextToken();
if (token == XContentParser.Token.START_OBJECT) {
Script script = null;
boolean ignoreFailure = false;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SCRIPT_FIELD)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else if (context.parseFieldMatcher().match(currentFieldName,
SearchSourceBuilder.IGNORE_FAILURE_FIELD)) {
ignoreFailure = parser.booleanValue();
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SCRIPT_FIELD)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + currentFieldName + "].", parser.getTokenLocation());
}
}
scriptFields.add(new ScriptField(scriptFieldName, script, ignoreFailure));
} else {
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.START_OBJECT
+ "] in [" + currentFieldName + "] but found [" + token + "]", parser.getTokenLocation());
}
}
factory.scriptFields(scriptFields);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.HIGHLIGHT_FIELD)) {
factory.highlighter(HighlightBuilder.PROTOTYPE.fromXContent(context));
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SORT_FIELD)) {
List<BytesReference> sorts = new ArrayList<>();
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
sorts.add(xContentBuilder.bytes());
factory.sorts(sorts);
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.FIELDS_FIELD)) {
List<String> fieldNames = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.VALUE_STRING) {
fieldNames.add(parser.text());
} else {
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.VALUE_STRING
+ "] in [" + currentFieldName + "] but found [" + token + "]", parser.getTokenLocation());
}
}
factory.fields(fieldNames);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.FIELDDATA_FIELDS_FIELD)) {
List<String> fieldDataFields = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.VALUE_STRING) {
fieldDataFields.add(parser.text());
} else {
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.VALUE_STRING
+ "] in [" + currentFieldName + "] but found [" + token + "]", parser.getTokenLocation());
}
}
factory.fieldDataFields(fieldDataFields);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SORT_FIELD)) {
List<BytesReference> sorts = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
sorts.add(xContentBuilder.bytes());
}
factory.sorts(sorts);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder._SOURCE_FIELD)) {
factory.fetchSource(FetchSourceContext.parse(parser, context));
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
}
} catch (Exception e) {
throw ExceptionsHelper.convertToElastic(e);
}
return new TopHitsAggregator.Factory(aggregationName, fetchPhase, subSearchContext);
return factory;
}
// NORELEASE implement this method when refactoring this aggregation
@Override
public AggregatorFactory[] getFactoryPrototypes() {
return null;
return new AggregatorFactory[] { new TopHitsAggregator.Factory(null) };
}
}

View File

@ -20,17 +20,17 @@
package org.elasticsearch.search.aggregations.pipeline;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
import org.elasticsearch.search.aggregations.InvalidAggregationPathException;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativeParser;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -62,7 +62,7 @@ public class BucketHelpers {
* @param text GapPolicy in string format (e.g. "ignore")
* @return GapPolicy enum
*/
public static GapPolicy parse(SearchContext context, String text, XContentLocation tokenLocation) {
public static GapPolicy parse(QueryParseContext context, String text, XContentLocation tokenLocation) {
GapPolicy result = null;
for (GapPolicy policy : values()) {
if (context.parseFieldMatcher().match(text, policy.parseField)) {
@ -79,7 +79,7 @@ public class BucketHelpers {
for (GapPolicy policy : values()) {
validNames.add(policy.getName());
}
throw new SearchParseException(context, "Invalid gap policy: [" + text + "], accepted values: " + validNames, tokenLocation);
throw new ParsingException(tokenLocation, "Invalid gap policy: [" + text + "], accepted values: " + validNames);
}
return result;
}

View File

@ -25,10 +25,10 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
@ -38,7 +38,7 @@ public abstract class PipelineAggregator implements Streamable {
/**
* Parses the pipeline aggregation request and creates the appropriate
* pipeline aggregator factory for it.
*
*
* @see PipelineAggregatorFactory
*/
public static interface Parser {
@ -56,7 +56,7 @@ public abstract class PipelineAggregator implements Streamable {
/**
* Returns the pipeline aggregator factory with which this parser is
* associated.
*
*
* @param pipelineAggregatorName
* The name of the pipeline aggregation
* @param parser
@ -67,7 +67,7 @@ public abstract class PipelineAggregator implements Streamable {
* @throws java.io.IOException
* When parsing fails
*/
PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, SearchContext context) throws IOException;
PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, QueryParseContext context) throws IOException;
/**
* @return an empty {@link PipelineAggregatorFactory} instance for this

View File

@ -20,12 +20,12 @@
package org.elasticsearch.search.aggregations.pipeline.bucketmetrics;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.text.ParseException;
@ -46,7 +46,8 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
}
@Override
public final PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, SearchContext context) throws IOException {
public final PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, QueryParseContext context)
throws IOException {
XContentParser.Token token;
String currentFieldName = null;
String[] bucketsPaths = null;
@ -84,8 +85,8 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
}
if (bucketsPaths == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for aggregation [" + pipelineAggregatorName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Missing required field [" + BUCKETS_PATH.getPreferredName() + "] for aggregation [" + pipelineAggregatorName + "]");
}
BucketMetricsFactory factory = null;
@ -98,12 +99,13 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
factory.gapPolicy(gapPolicy);
}
} catch (ParseException exception) {
throw new SearchParseException(context, "Could not parse settings for aggregation ["
+ pipelineAggregatorName + "].", null, exception);
throw new ParsingException(parser.getTokenLocation(),
"Could not parse settings for aggregation [" + pipelineAggregatorName + "].", exception);
}
if (leftover.size() > 0) {
throw new SearchParseException(context, "Unexpected tokens " + leftover.keySet() + " in [" + pipelineAggregatorName + "].", null);
throw new ParsingException(parser.getTokenLocation(),
"Unexpected tokens " + leftover.keySet() + " in [" + pipelineAggregatorName + "].");
}
assert(factory != null);

View File

@ -20,14 +20,14 @@
package org.elasticsearch.search.aggregations.pipeline.bucketscript;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -48,7 +48,7 @@ public class BucketScriptParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, QueryParseContext context) throws IOException {
XContentParser.Token token;
Script script = null;
String currentFieldName = null;
@ -70,8 +70,8 @@ public class BucketScriptParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -85,8 +85,8 @@ public class BucketScriptParser implements PipelineAggregator.Parser {
bucketsPathsMap.put("_value" + i, paths.get(i));
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
@ -98,23 +98,22 @@ public class BucketScriptParser implements PipelineAggregator.Parser {
bucketsPathsMap.put(entry.getKey(), String.valueOf(entry.getValue()));
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + reducerName + "].");
}
}
if (bucketsPathsMap == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for series_arithmetic aggregation [" + reducerName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for series_arithmetic aggregation [" + reducerName + "]");
}
if (script == null) {
throw new SearchParseException(context, "Missing required field [" + ScriptField.SCRIPT.getPreferredName()
+ "] for series_arithmetic aggregation [" + reducerName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + ScriptField.SCRIPT.getPreferredName()
+ "] for series_arithmetic aggregation [" + reducerName + "]");
}
BucketScriptPipelineAggregator.Factory factory = new BucketScriptPipelineAggregator.Factory(reducerName, bucketsPathsMap, script);

View File

@ -20,11 +20,11 @@
package org.elasticsearch.search.aggregations.pipeline.cumulativesum;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -42,7 +42,8 @@ public class CumulativeSumParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, QueryParseContext context)
throws IOException {
XContentParser.Token token;
String currentFieldName = null;
String[] bucketsPaths = null;
@ -57,8 +58,8 @@ public class CumulativeSumParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
bucketsPaths = new String[] { parser.text() };
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -69,18 +70,18 @@ public class CumulativeSumParser implements PipelineAggregator.Parser {
}
bucketsPaths = paths.toArray(new String[paths.size()]);
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + pipelineAggregatorName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [" + pipelineAggregatorName + "].");
}
}
if (bucketsPaths == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for derivative aggregation [" + pipelineAggregatorName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for derivative aggregation [" + pipelineAggregatorName + "]");
}
CumulativeSumPipelineAggregator.Factory factory = new CumulativeSumPipelineAggregator.Factory(pipelineAggregatorName, bucketsPaths);

View File

@ -20,12 +20,12 @@
package org.elasticsearch.search.aggregations.pipeline.derivative;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -43,7 +43,8 @@ public class DerivativeParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, QueryParseContext context)
throws IOException {
XContentParser.Token token;
String currentFieldName = null;
String[] bucketsPaths = null;
@ -64,8 +65,8 @@ public class DerivativeParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, UNIT)) {
units = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -76,18 +77,18 @@ public class DerivativeParser implements PipelineAggregator.Parser {
}
bucketsPaths = paths.toArray(new String[paths.size()]);
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + pipelineAggregatorName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [" + pipelineAggregatorName + "].");
}
}
if (bucketsPaths == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for derivative aggregation [" + pipelineAggregatorName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for derivative aggregation [" + pipelineAggregatorName + "]");
}
DerivativePipelineAggregator.Factory factory = new DerivativePipelineAggregator.Factory(pipelineAggregatorName, bucketsPaths);

View File

@ -20,14 +20,14 @@
package org.elasticsearch.search.aggregations.pipeline.having;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -48,7 +48,7 @@ public class BucketSelectorParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, QueryParseContext context) throws IOException {
XContentParser.Token token;
Script script = null;
String currentFieldName = null;
@ -67,8 +67,8 @@ public class BucketSelectorParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -82,8 +82,8 @@ public class BucketSelectorParser implements PipelineAggregator.Parser {
bucketsPathsMap.put("_value" + i, paths.get(i));
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
@ -95,23 +95,22 @@ public class BucketSelectorParser implements PipelineAggregator.Parser {
bucketsPathsMap.put(entry.getKey(), String.valueOf(entry.getValue()));
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + reducerName + "].");
}
}
if (bucketsPathsMap == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for bucket_selector aggregation [" + reducerName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for bucket_selector aggregation [" + reducerName + "]");
}
if (script == null) {
throw new SearchParseException(context, "Missing required field [" + ScriptField.SCRIPT.getPreferredName()
+ "] for bucket_selector aggregation [" + reducerName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + ScriptField.SCRIPT.getPreferredName()
+ "] for bucket_selector aggregation [" + reducerName + "]");
}
BucketSelectorPipelineAggregator.Factory factory = new BucketSelectorPipelineAggregator.Factory(reducerName, bucketsPathsMap,

View File

@ -20,15 +20,15 @@
package org.elasticsearch.search.aggregations.pipeline.movavg;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel;
import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModelParserMapper;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.text.ParseException;
@ -57,7 +57,8 @@ public class MovAvgParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String pipelineAggregatorName, XContentParser parser, QueryParseContext context)
throws IOException {
XContentParser.Token token;
String currentFieldName = null;
String[] bucketsPaths = null;
@ -77,20 +78,18 @@ public class MovAvgParser implements PipelineAggregator.Parser {
if (context.parseFieldMatcher().match(currentFieldName, WINDOW)) {
window = parser.intValue();
if (window <= 0) {
throw new SearchParseException(context, "[" + currentFieldName + "] value must be a positive, "
+ "non-zero integer. Value supplied was [" + predict + "] in [" + pipelineAggregatorName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "[" + currentFieldName + "] value must be a positive, "
+ "non-zero integer. Value supplied was [" + predict + "] in [" + pipelineAggregatorName + "].");
}
} else if (context.parseFieldMatcher().match(currentFieldName, PREDICT)) {
predict = parser.intValue();
if (predict <= 0) {
throw new SearchParseException(context, "[" + currentFieldName + "] value must be a positive integer."
+ " Value supplied was [" + predict + "] in [" + pipelineAggregatorName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "[" + currentFieldName + "] value must be a positive integer."
+ " Value supplied was [" + predict + "] in [" + pipelineAggregatorName + "].");
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (context.parseFieldMatcher().match(currentFieldName, FORMAT)) {
@ -102,8 +101,8 @@ public class MovAvgParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, MODEL)) {
model = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -114,32 +113,32 @@ public class MovAvgParser implements PipelineAggregator.Parser {
}
bucketsPaths = paths.toArray(new String[paths.size()]);
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, SETTINGS)) {
settings = parser.map();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (context.parseFieldMatcher().match(currentFieldName, MINIMIZE)) {
minimize = parser.booleanValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + pipelineAggregatorName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + pipelineAggregatorName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [" + pipelineAggregatorName + "].");
}
}
if (bucketsPaths == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for movingAvg aggregation [" + pipelineAggregatorName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for movingAvg aggregation [" + pipelineAggregatorName + "]");
}
MovAvgPipelineAggregator.Factory factory = new MovAvgPipelineAggregator.Factory(pipelineAggregatorName, bucketsPaths);
@ -158,16 +157,15 @@ public class MovAvgParser implements PipelineAggregator.Parser {
if (model != null) {
MovAvgModel.AbstractModelParser modelParser = movAvgModelParserMapper.get(model);
if (modelParser == null) {
throw new SearchParseException(context,
"Unknown model [" + model + "] specified. Valid options are:" + movAvgModelParserMapper.getAllNames().toString(),
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown model [" + model + "] specified. Valid options are:" + movAvgModelParserMapper.getAllNames().toString());
}
MovAvgModel movAvgModel;
try {
movAvgModel = modelParser.parse(settings, pipelineAggregatorName, window, context.parseFieldMatcher());
} catch (ParseException exception) {
throw new SearchParseException(context, "Could not parse settings for model [" + model + "].", null, exception);
throw new ParsingException(parser.getTokenLocation(), "Could not parse settings for model [" + model + "].", exception);
}
factory.model(movAvgModel);
}

View File

@ -20,12 +20,12 @@
package org.elasticsearch.search.aggregations.pipeline.serialdiff;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -43,7 +43,7 @@ public class SerialDiffParser implements PipelineAggregator.Parser {
}
@Override
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException {
public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, QueryParseContext context) throws IOException {
XContentParser.Token token;
String currentFieldName = null;
String[] bucketsPaths = null;
@ -62,20 +62,21 @@ public class SerialDiffParser implements PipelineAggregator.Parser {
} else if (context.parseFieldMatcher().match(currentFieldName, GAP_POLICY)) {
gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (context.parseFieldMatcher().match(currentFieldName, LAG)) {
lag = parser.intValue(true);
if (lag <= 0) {
throw new SearchParseException(context, "Lag must be a positive, non-zero integer. Value supplied was" +
throw new ParsingException(parser.getTokenLocation(),
"Lag must be a positive, non-zero integer. Value supplied was" +
lag + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
+ currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
@ -86,18 +87,18 @@ public class SerialDiffParser implements PipelineAggregator.Parser {
}
bucketsPaths = paths.toArray(new String[paths.size()]);
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].",
throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + reducerName + "].",
parser.getTokenLocation());
}
}
if (bucketsPaths == null) {
throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName()
+ "] for derivative aggregation [" + reducerName + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Missing required field [" + BUCKETS_PATH.getPreferredName() + "] for derivative aggregation [" + reducerName + "]");
}
SerialDiffPipelineAggregator.Factory factory = new SerialDiffPipelineAggregator.Factory(reducerName, bucketsPaths);

View File

@ -21,13 +21,13 @@ package org.elasticsearch.search.aggregations.support;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.internal.SearchContext;
import org.joda.time.DateTimeZone;
import java.io.IOException;
@ -84,7 +84,7 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource> implem
}
@Override
public final AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
public final AggregatorFactory parse(String aggregationName, XContentParser parser, QueryParseContext context) throws IOException {
String field = null;
Script script = null;
@ -107,8 +107,8 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource> implem
} else if (token == XContentParser.Token.VALUE_NUMBER) {
timezone = DateTimeZone.forOffsetHours(parser.intValue());
} else {
throw new SearchParseException(context, "Unexpected token " + token + " [" + currentFieldName + "] in ["
+ aggregationName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if ("field".equals(currentFieldName)) {
@ -119,28 +119,29 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource> implem
if ("value_type".equals(currentFieldName) || "valueType".equals(currentFieldName)) {
valueType = ValueType.resolveForScript(parser.text());
if (targetValueType != null && valueType.isNotA(targetValueType)) {
throw new SearchParseException(context, type() + " aggregation [" + aggregationName
+ "] was configured with an incompatible value type [" + valueType + "]. [" + type()
+ "] aggregation can only work on value of type [" + targetValueType + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
type() + " aggregation [" + aggregationName + "] was configured with an incompatible value type ["
+ valueType + "]. [" + type() + "] aggregation can only work on value of type ["
+ targetValueType + "]");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.parseFieldMatcher(), otherOptions)) {
throw new SearchParseException(context, "Unexpected token " + token + " [" + currentFieldName + "] in ["
+ aggregationName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.parseFieldMatcher(), otherOptions)) {
throw new SearchParseException(context, "Unexpected token " + token + " [" + currentFieldName + "] in ["
+ aggregationName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (scriptable && token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else if (!token(aggregationName, currentFieldName, token, parser, context.parseFieldMatcher(), otherOptions)) {
throw new SearchParseException(context, "Unexpected token " + token + " [" + currentFieldName + "] in ["
+ aggregationName + "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.parseFieldMatcher(), otherOptions)) {
throw new SearchParseException(context, "Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName
+ "].", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
}

View File

@ -1080,7 +1080,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
this(fieldName, script, false);
}
private ScriptField(String fieldName, Script script, boolean ignoreFailure) {
public ScriptField(String fieldName, Script script, boolean ignoreFailure) {
this.fieldName = fieldName;
this.script = script;
this.ignoreFailure = ignoreFailure;

View File

@ -53,6 +53,7 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
@ -145,16 +146,16 @@ public class DefaultSearchContext extends SearchContext {
private final Map<String, FetchSubPhaseContext> subPhaseContexts = new HashMap<>();
private final Map<Class<?>, Collector> queryCollectors = new HashMap<>();
private FetchPhase fetchPhase;
public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget,
Engine.Searcher engineSearcher, IndexService indexService, IndexShard indexShard,
ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher,
TimeValue timeout
) {
public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher,
IndexService indexService, IndexShard indexShard, ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout,
FetchPhase fetchPhase) {
super(parseFieldMatcher, request);
this.id = id;
this.request = request;
this.fetchPhase = fetchPhase;
this.searchType = request.searchType();
this.shardTarget = shardTarget;
this.engineSearcher = engineSearcher;
@ -692,6 +693,11 @@ public class DefaultSearchContext extends SearchContext {
return queryResult;
}
@Override
public FetchPhase fetchPhase() {
return fetchPhase;
}
@Override
public FetchSearchResult fetchResult() {
return fetchResult;

View File

@ -40,6 +40,7 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
@ -491,6 +492,11 @@ public abstract class FilteredSearchContext extends SearchContext {
return in.fetchResult();
}
@Override
public FetchPhase fetchPhase() {
return in.fetchPhase();
}
@Override
public MappedFieldType smartNameFieldType(String name) {
return in.smartNameFieldType(name);

View File

@ -47,6 +47,7 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
@ -304,6 +305,8 @@ public abstract class SearchContext extends DelegatingHasContextAndHeaders imple
public abstract QuerySearchResult queryResult();
public abstract FetchPhase fetchPhase();
public abstract FetchSearchResult fetchResult();
/**

View File

@ -19,9 +19,12 @@
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -58,10 +61,14 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase {
private boolean needsScores(IndexService index, String agg) throws IOException {
AggregatorParsers parser = getInstanceFromNode(AggregatorParsers.class);
IndicesQueriesRegistry queriesRegistry = getInstanceFromNode(IndicesQueriesRegistry.class);
XContentParser aggParser = JsonXContent.jsonXContent.createParser(agg);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry);
parseContext.reset(aggParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
aggParser.nextToken();
SearchContext searchContext = createSearchContext(index);
final AggregatorFactories factories = parser.parseAggregators(aggParser, searchContext);
final AggregatorFactories factories = parser.parseAggregators(aggParser, parseContext);
AggregationContext aggregationContext = new AggregationContext(searchContext);
factories.init(aggregationContext);
final Aggregator[] aggregators = factories.createTopLevelAggregators();

View File

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Injector;
import org.elasticsearch.common.inject.ModulesBuilder;
@ -39,9 +40,11 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.EnvironmentModule;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.internal.SearchContext;
@ -83,6 +86,8 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
private static NamedWriteableRegistry namedWriteableRegistry;
private static AggregatorParsers aggParsers;
private static IndicesQueriesRegistry queriesRegistry;
private static ParseFieldMatcher parseFieldMatcher;
protected abstract AF createTestAggregatorFactory();
@ -138,6 +143,8 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
currentTypes[i] = type;
}
namedWriteableRegistry = injector.getInstance(NamedWriteableRegistry.class);
queriesRegistry = injector.getInstance(IndicesQueriesRegistry.class);
parseFieldMatcher = ParseFieldMatcher.STRICT;
}
@AfterClass
@ -174,6 +181,9 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
AggregatorFactories factories = AggregatorFactories.builder().addAggregator(testAgg).build();
String contentString = factories.toString();
XContentParser parser = XContentFactory.xContent(contentString).createParser(contentString);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry);
parseContext.reset(parser);
parseContext.parseFieldMatcher(parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.name, parser.currentName());
@ -181,7 +191,7 @@ public abstract class BaseAggregationTestCase<AF extends AggregatorFactory> exte
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.type.name(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
AggregatorFactory newAgg = aggParsers.parser(testAgg.getType()).parse(testAgg.name, parser, SearchContext.current());
AggregatorFactory newAgg = aggParsers.parser(testAgg.getType()).parse(testAgg.name, parser, parseContext);
assertSame(XContentParser.Token.END_OBJECT, parser.currentToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());

View File

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Injector;
import org.elasticsearch.common.inject.ModulesBuilder;
@ -39,9 +40,11 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.EnvironmentModule;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory;
@ -84,6 +87,8 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
private static NamedWriteableRegistry namedWriteableRegistry;
private static AggregatorParsers aggParsers;
private static ParseFieldMatcher parseFieldMatcher;
private static IndicesQueriesRegistry queriesRegistry;
protected abstract AF createTestAggregatorFactory();
@ -138,6 +143,8 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
currentTypes[i] = type;
}
namedWriteableRegistry = injector.getInstance(NamedWriteableRegistry.class);
queriesRegistry = injector.getInstance(IndicesQueriesRegistry.class);
parseFieldMatcher = ParseFieldMatcher.STRICT;
}
@AfterClass
@ -176,6 +183,9 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
String contentString = factories.toString();
System.out.println(contentString);
XContentParser parser = XContentFactory.xContent(contentString).createParser(contentString);
QueryParseContext parseContext = new QueryParseContext(queriesRegistry);
parseContext.reset(parser);
parseContext.parseFieldMatcher(parseFieldMatcher);
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.name(), parser.currentName());
@ -184,7 +194,7 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
assertEquals(testAgg.type(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
PipelineAggregatorFactory newAgg = aggParsers.pipelineAggregator(testAgg.getWriteableName()).parse(testAgg.name(), parser,
SearchContext.current());
parseContext);
assertSame(XContentParser.Token.END_OBJECT, parser.currentToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());

View File

@ -18,46 +18,52 @@
*/
package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TestSearchContext;
public class GeoHashGridParserTests extends ESTestCase {
public void testParseValidFromInts() throws Exception {
SearchContext searchContext = new TestSearchContext();
int precision = randomIntBetween(1, 12);
XContentParser stParser = JsonXContent.jsonXContent.createParser(
"{\"field\":\"my_loc\", \"precision\":" + precision + ", \"size\": 500, \"shard_size\": 550}");
QueryParseContext parseContext = new QueryParseContext(null);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
GeoHashGridParser parser = new GeoHashGridParser();
// can create a factory
assertNotNull(parser.parse("geohash_grid", stParser, searchContext));
assertNotNull(parser.parse("geohash_grid", stParser, parseContext));
}
public void testParseValidFromStrings() throws Exception {
SearchContext searchContext = new TestSearchContext();
int precision = randomIntBetween(1, 12);
XContentParser stParser = JsonXContent.jsonXContent.createParser(
"{\"field\":\"my_loc\", \"precision\":\"" + precision + "\", \"size\": \"500\", \"shard_size\": \"550\"}");
QueryParseContext parseContext = new QueryParseContext(null);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
GeoHashGridParser parser = new GeoHashGridParser();
// can create a factory
assertNotNull(parser.parse("geohash_grid", stParser, searchContext));
assertNotNull(parser.parse("geohash_grid", stParser, parseContext));
}
public void testParseErrorOnNonIntPrecision() throws Exception {
SearchContext searchContext = new TestSearchContext();
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":\"2.0\"}");
QueryParseContext parseContext = new QueryParseContext(null);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
GeoHashGridParser parser = new GeoHashGridParser();
try {
parser.parse("geohash_grid", stParser, searchContext);
parser.parse("geohash_grid", stParser, parseContext);
fail();
} catch (NumberFormatException ex) {
assertEquals("For input string: \"2.0\"", ex.getMessage());
@ -65,27 +71,31 @@ public class GeoHashGridParserTests extends ESTestCase {
}
public void testParseErrorOnBooleanPrecision() throws Exception {
SearchContext searchContext = new TestSearchContext();
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":false}");
QueryParseContext parseContext = new QueryParseContext(null);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
GeoHashGridParser parser = new GeoHashGridParser();
try {
parser.parse("geohash_grid", stParser, searchContext);
parser.parse("geohash_grid", stParser, parseContext);
fail();
} catch (SearchParseException ex) {
} catch (ParsingException ex) {
assertEquals("Unexpected token VALUE_BOOLEAN [precision] in [geohash_grid].", ex.getMessage());
}
}
public void testParseErrorOnPrecisionOutOfRange() throws Exception {
SearchContext searchContext = new TestSearchContext();
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":\"13\"}");
QueryParseContext parseContext = new QueryParseContext(null);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
XContentParser.Token token = stParser.nextToken();
assertSame(XContentParser.Token.START_OBJECT, token);
GeoHashGridParser parser = new GeoHashGridParser();
try {
parser.parse("geohash_grid", stParser, searchContext);
parser.parse("geohash_grid", stParser, parseContext);
fail();
} catch (IllegalArgumentException ex) {
assertEquals("Invalid geohash aggregation precision of 13. Must be between 1 and 12.", ex.getMessage());

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
@ -29,6 +30,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.InternalAggregation;
@ -236,8 +238,11 @@ public class SignificanceHeuristicTests extends ESTestCase {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, new HashSet<>(), new NamedWriteableRegistry());
try {
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"text\", " + faultyHeuristicDefinition + ",\"min_doc_count\":200}");
QueryParseContext parseContext = new QueryParseContext(registry);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
stParser.nextToken();
new SignificantTermsParser(heuristicParserMapper, registry).parse("testagg", stParser, searchContext);
new SignificantTermsParser(heuristicParserMapper, registry).parse("testagg", stParser, parseContext);
fail();
} catch (ElasticsearchParseException e) {
assertTrue(e.getMessage().contains(expectedError));
@ -256,9 +261,12 @@ public class SignificanceHeuristicTests extends ESTestCase {
private SignificanceHeuristic parseSignificanceHeuristic(SignificanceHeuristicParserMapper heuristicParserMapper,
SearchContext searchContext, XContentParser stParser) throws IOException {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, new HashSet<>(), new NamedWriteableRegistry());
QueryParseContext parseContext = new QueryParseContext(registry);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);
stParser.nextToken();
SignificantTermsAggregatorFactory aggregatorFactory = (SignificantTermsAggregatorFactory) new SignificantTermsParser(
heuristicParserMapper, registry).parse("testagg", stParser, searchContext);
heuristicParserMapper, registry).parse("testagg", stParser, parseContext);
stParser.nextToken();
assertThat(aggregatorFactory.getBucketCountThresholds().getMinDocCount(), equalTo(200l));
assertThat(stParser.currentToken(), equalTo(null));

View File

@ -0,0 +1,146 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.index.query.AbstractQueryTestCase;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregator;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import java.util.ArrayList;
import java.util.List;;
public class TopHitsTests extends BaseAggregationTestCase<TopHitsAggregator.Factory> {
@Override
protected final TopHitsAggregator.Factory createTestAggregatorFactory() {
TopHitsAggregator.Factory factory = new TopHitsAggregator.Factory("foo");
if (randomBoolean()) {
factory.from(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
factory.size(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
factory.explain(randomBoolean());
}
if (randomBoolean()) {
factory.version(randomBoolean());
}
if (randomBoolean()) {
factory.trackScores(randomBoolean());
}
if (randomBoolean()) {
int fieldsSize = randomInt(25);
List<String> fields = new ArrayList<>(fieldsSize);
for (int i = 0; i < fieldsSize; i++) {
fields.add(randomAsciiOfLengthBetween(5, 50));
}
factory.fields(fields);
}
if (randomBoolean()) {
int fieldDataFieldsSize = randomInt(25);
for (int i = 0; i < fieldDataFieldsSize; i++) {
factory.fieldDataField(randomAsciiOfLengthBetween(5, 50));
}
}
if (randomBoolean()) {
int scriptFieldsSize = randomInt(25);
for (int i = 0; i < scriptFieldsSize; i++) {
if (randomBoolean()) {
factory.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"), randomBoolean());
} else {
factory.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"));
}
}
}
if (randomBoolean()) {
FetchSourceContext fetchSourceContext;
int branch = randomInt(5);
String[] includes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < includes.length; i++) {
includes[i] = randomAsciiOfLengthBetween(5, 20);
}
String[] excludes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < excludes.length; i++) {
excludes[i] = randomAsciiOfLengthBetween(5, 20);
}
switch (branch) {
case 0:
fetchSourceContext = new FetchSourceContext(randomBoolean());
break;
case 1:
fetchSourceContext = new FetchSourceContext(includes, excludes);
break;
case 2:
fetchSourceContext = new FetchSourceContext(randomAsciiOfLengthBetween(5, 20), randomAsciiOfLengthBetween(5, 20));
break;
case 3:
fetchSourceContext = new FetchSourceContext(true, includes, excludes);
break;
case 4:
fetchSourceContext = new FetchSourceContext(includes);
break;
case 5:
fetchSourceContext = new FetchSourceContext(randomAsciiOfLengthBetween(5, 20));
break;
default:
throw new IllegalStateException();
}
factory.fetchSource(fetchSourceContext);
}
if (randomBoolean()) {
int numSorts = randomIntBetween(1, 5);
for (int i = 0; i < numSorts; i++) {
int branch = randomInt(5);
switch (branch) {
case 0:
factory.sort(SortBuilders.fieldSort(randomAsciiOfLengthBetween(5, 20)).order(randomFrom(SortOrder.values())));
break;
case 1:
factory.sort(SortBuilders.geoDistanceSort(randomAsciiOfLengthBetween(5, 20))
.geohashes(AbstractQueryTestCase.randomGeohash(1, 12)).order(randomFrom(SortOrder.values())));
break;
case 2:
factory.sort(SortBuilders.scoreSort().order(randomFrom(SortOrder.values())));
break;
case 3:
factory.sort(SortBuilders.scriptSort(new Script("foo"), "number").order(randomFrom(SortOrder.values())));
break;
case 4:
factory.sort(randomAsciiOfLengthBetween(5, 20));
break;
case 5:
factory.sort(randomAsciiOfLengthBetween(5, 20), randomFrom(SortOrder.values()));
break;
}
}
}
if (randomBoolean()) {
factory.highlighter(HighlightBuilderTests.randomHighlighterBuilder());
}
return factory;
}
}

View File

@ -284,7 +284,7 @@ public class SearchSourceBuilderTests extends ESTestCase {
// NORELEASE need a random aggregation builder method
builder.aggregation(AggregationBuilders.avg(randomAsciiOfLengthBetween(5, 20)));
}
if (true) {
if (randomBoolean()) {
// NORELEASE need a method to randomly build content for ext
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder();
xContentBuilder.startObject();

View File

@ -18,6 +18,12 @@
*/
package org.elasticsearch.test;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import com.carrotsearch.hppc.ObjectObjectAssociativeContainer;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Query;
@ -46,6 +52,7 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
@ -64,11 +71,7 @@ import org.elasticsearch.search.rescore.RescoreSearchContext;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import com.carrotsearch.hppc.ObjectObjectAssociativeContainer;
public class TestSearchContext extends SearchContext {
@ -548,6 +551,11 @@ public class TestSearchContext extends SearchContext {
return null;
}
@Override
public FetchPhase fetchPhase() {
return null;
}
@Override
public MappedFieldType smartNameFieldType(String name) {
if (mapperService() != null) {