Make aggregation registration more like query registration

Creates a class to hold behavior common to these `ParseFieldRegistry`s.

Also renames some weirdly named methods on StreamInput.
This commit is contained in:
Nik Everett 2016-04-11 09:37:18 -04:00
parent fae0666226
commit 7908759949
29 changed files with 269 additions and 270 deletions

View File

@ -731,14 +731,14 @@ public abstract class StreamInput extends InputStream {
/**
* Reads a {@link AggregatorBuilder} from the current stream
*/
public AggregatorBuilder<?> readAggregatorFactory() throws IOException {
public AggregatorBuilder<?> readAggregatorBuilder() throws IOException {
return readNamedWriteable(AggregatorBuilder.class);
}
/**
* Reads a {@link PipelineAggregatorBuilder} from the current stream
*/
public PipelineAggregatorBuilder<?> readPipelineAggregatorFactory() throws IOException {
public PipelineAggregatorBuilder<?> readPipelineAggregatorBuilder() throws IOException {
return readNamedWriteable(PipelineAggregatorBuilder.class);
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.xcontent;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.collect.Tuple;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
/**
* Registry for looking things up using ParseField semantics.
*/
public class ParseFieldRegistry<T> {
private final Map<String, Tuple<ParseField, T>> registry = new HashMap<>();
private final String registryName;
/**
* Build the registry.
* @param registryName used for error messages
*/
public ParseFieldRegistry(String registryName) {
this.registryName = registryName;
}
/**
* All the names under which values are registered. Expect this to be used mostly for testing.
*/
public Set<String> getNames() {
return registry.keySet();
}
/**
* Register a parser.
*/
public void register(T value, ParseField parseField) {
Tuple<ParseField, T> parseFieldParserTuple = new Tuple<>(parseField, value);
for (String name: parseField.getAllNamesIncludedDeprecated()) {
Tuple<ParseField, T> previousValue = registry.putIfAbsent(name, parseFieldParserTuple);
if (previousValue != null) {
throw new IllegalArgumentException("[" + previousValue.v2() + "] already registered for [" + registryName + "][" + name
+ "] while trying to register [" + value + "]");
}
}
}
/**
* Lookup a value from the registry by name while checking that the name matches the ParseField.
*
* @param name The name of the thing to look up.
* @param parser The parser from which the name was looked up. This is used to resolve the {@link ParseFieldMatcher} and to build nice
* error messages.
* @return The value being looked up. Never null.
* @throws ParsingException if the named thing isn't in the registry or the name was deprecated and deprecated names aren't supported.
*/
public T lookup(String name, XContentParser parser) {
T value = lookupReturningNullIfNotFound(name, parser);
if (value == null) {
throw new ParsingException(parser.getTokenLocation(), "no [" + registryName + "] registered for [" + name + "]");
}
return value;
}
/**
* Lookup a value from the registry by name while checking that the name matches the ParseField.
*
* @param name The name of the thing to look up.
* @param parser The parser from which the name was looked up. This is used to resolve the {@link ParseFieldMatcher} and to build nice
* error messages.
* @return The value being looked up or null if it wasn't found.
* @throws ParsingException if the named thing isn't in the registry or the name was deprecated and deprecated names aren't supported.
*/
public T lookupReturningNullIfNotFound(String name, XContentParser parser) {
Tuple<ParseField, T> parseFieldAndValue = registry.get(name);
if (parseFieldAndValue == null) {
return null;
}
ParseField parseField = parseFieldAndValue.v1();
T value = parseFieldAndValue.v2();
boolean match = parser.getParseFieldMatcher().match(name, parseField);
//this is always expected to match, ParseField is useful for deprecation warnings etc. here
assert match : "ParseField did not match registered name [" + name + "][" + registryName + "]";
return value;
}
}

View File

@ -57,6 +57,9 @@ public class QueryParseContext {
if (parseFieldMatcher == null) {
throw new IllegalArgumentException("parseFieldMatcher must not be null");
}
if (parser != null) {
parser.setParseFieldMatcher(parseFieldMatcher);
}
this.parseFieldMatcher = parseFieldMatcher;
}
@ -117,8 +120,7 @@ public class QueryParseContext {
if (token != XContentParser.Token.START_OBJECT && token != XContentParser.Token.START_ARRAY) {
throw new ParsingException(parser.getTokenLocation(), "[_na] query malformed, no field after start_object");
}
QueryParser queryParser = indicesQueriesRegistry.getQueryParser(queryName, parseFieldMatcher, parser.getTokenLocation());
QueryBuilder result = queryParser.fromXContent(this);
QueryBuilder<?> result = indicesQueriesRegistry.lookup(queryName, parser).fromXContent(this);
if (parser.currentToken() == XContentParser.Token.END_OBJECT || parser.currentToken() == XContentParser.Token.END_ARRAY) {
// if we are at END_OBJECT, move to the next one...
parser.nextToken();

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery.FilterFunction;
import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
import org.elasticsearch.common.lucene.search.function.ScoreFunction;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentLocation;
@ -428,8 +429,8 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
return this;
}
public static FunctionScoreQueryBuilder fromXContent(ScoreFunctionsRegistry scoreFunctionsRegistry, QueryParseContext parseContext)
throws IOException {
public static FunctionScoreQueryBuilder fromXContent(ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionsRegistry,
QueryParseContext parseContext) throws IOException {
XContentParser parser = parseContext.parser();
QueryBuilder<?> query = null;
@ -475,8 +476,7 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
// we try to parse a score function. If there is no score function for the current field name,
// getScoreFunction will throw.
ScoreFunctionBuilder<?> scoreFunction = scoreFunctionsRegistry
.getScoreFunction(currentFieldName, parseContext.parseFieldMatcher(), parseContext.parser().getTokenLocation())
ScoreFunctionBuilder<?> scoreFunction = scoreFunctionsRegistry.lookup(currentFieldName, parseContext.parser())
.fromXContent(parseContext, parser);
filterFunctionBuilders.add(new FunctionScoreQueryBuilder.FilterFunctionBuilder(scoreFunction));
}
@ -554,8 +554,9 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
MISPLACED_FUNCTION_MESSAGE_PREFIX + errorString);
}
private static String parseFiltersAndFunctions(ScoreFunctionsRegistry scoreFunctionsRegistry, QueryParseContext parseContext,
XContentParser parser, List<FunctionScoreQueryBuilder.FilterFunctionBuilder> filterFunctionBuilders) throws IOException {
private static String parseFiltersAndFunctions(ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionsRegistry,
QueryParseContext parseContext, XContentParser parser,
List<FunctionScoreQueryBuilder.FilterFunctionBuilder> filterFunctionBuilders) throws IOException {
String currentFieldName = null;
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
@ -579,8 +580,7 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
"failed to parse function_score functions. already found [{}], now encountering [{}].",
scoreFunction.getName(), currentFieldName);
}
scoreFunction = scoreFunctionsRegistry.getScoreFunction(currentFieldName, parseContext.parseFieldMatcher(),
parseContext.parser().getTokenLocation()).fromXContent(parseContext, parser);
scoreFunction = scoreFunctionsRegistry.lookup(currentFieldName, parser).fromXContent(parseContext, parser);
}
} else if (token.isValue()) {
if (parseContext.parseFieldMatcher().match(currentFieldName, WEIGHT_FIELD)) {

View File

@ -1,62 +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.index.query.functionscore;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.xcontent.XContentLocation;
import java.util.Map;
import static java.util.Collections.unmodifiableMap;
/**
* Registry of all {@link ScoreFunctionParser}s.
*/
public class ScoreFunctionsRegistry {
private final Map<String, Tuple<ParseField, ScoreFunctionParser<?>>> scoreFunctionParsers;
public ScoreFunctionsRegistry(Map<String, Tuple<ParseField, ScoreFunctionParser<?>>> scoreFunctionParsers) {
this.scoreFunctionParsers = unmodifiableMap(scoreFunctionParsers);
}
/**
* Get the {@linkplain ScoreFunctionParser} for a specific type of query registered under its name.
* Uses {@link ParseField} internally so that deprecation warnings/errors can be logged/thrown.
* @param name the name of the parser to retrieve
* @param parseFieldMatcher the {@link ParseFieldMatcher} to match the query name against
* @param xContentLocation the current location of the {@link org.elasticsearch.common.xcontent.XContentParser}
* @return the query parser
* @throws IllegalArgumentException of there's no query or parser registered under the provided name
*/
public ScoreFunctionParser<?> getScoreFunction(String name, ParseFieldMatcher parseFieldMatcher, XContentLocation xContentLocation) {
Tuple<ParseField, ScoreFunctionParser<?>> parserLookup = scoreFunctionParsers.get(name);
if (parserLookup == null) {
throw new ParsingException(xContentLocation, "No function with the name [" + name + "] is registered.");
}
ParseField functionField = parserLookup.v1();
ScoreFunctionParser<?> functionParser = parserLookup.v2();
boolean match = parseFieldMatcher.match(name, functionField);
assert match : "registered ParseField did not match the name it was registered for [" + name + "]";
return functionParser;
}
}

View File

@ -19,44 +19,14 @@
package org.elasticsearch.indices.query;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.index.query.QueryParser;
import java.util.Map;
public class IndicesQueriesRegistry extends AbstractComponent {
private Map<String, Tuple<ParseField, QueryParser<?>>> queryParsers;
public IndicesQueriesRegistry(Settings settings, Map<String, Tuple<ParseField, QueryParser<?>>> queryParsers) {
super(settings);
this.queryParsers = queryParsers;
}
/**
* Get the query parser for a specific type of query registered under its name.
* Uses {@link ParseField} internally so that deprecation warnings/errors can be logged/thrown.
* @param name the name of the parser to retrieve
* @param parseFieldMatcher the {@link ParseFieldMatcher} to match the query name against
* @param xContentLocation the current location of the {@link org.elasticsearch.common.xcontent.XContentParser}
* @return the query parser
* @throws IllegalArgumentException of there's no query or parser registered under the provided name
*/
public QueryParser<?> getQueryParser(String name, ParseFieldMatcher parseFieldMatcher, XContentLocation xContentLocation) {
Tuple<ParseField, QueryParser<?>> parseFieldQueryParserTuple = queryParsers.get(name);
if (parseFieldQueryParserTuple == null) {
throw new ParsingException(xContentLocation, "No query registered for [" + name + "]");
}
ParseField parseField = parseFieldQueryParserTuple.v1();
QueryParser<?> queryParser = parseFieldQueryParserTuple.v2();
boolean match = parseFieldMatcher.match(name, parseField);
//this is always expected to match, ParseField is useful for deprecation warnings etc. here
assert match : "registered ParseField did not match the query name it was registered for: [" + name + "]";
return queryParser;
/**
* Extensions to ParseFieldRegistry to make Guice happy.
*/
public class IndicesQueriesRegistry extends ParseFieldRegistry<QueryParser<?>> {
public IndicesQueriesRegistry() {
super("query");
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search;
import org.apache.lucene.search.BooleanQuery;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.geo.ShapesAvailability;
import org.elasticsearch.common.geo.builders.ShapeBuilders;
import org.elasticsearch.common.inject.AbstractModule;
@ -31,6 +30,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.lucene.search.function.ScoreFunction;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.index.percolator.PercolatorHighlightSubFetchPhase;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.BoostingQueryBuilder;
@ -91,7 +91,6 @@ import org.elasticsearch.index.query.functionscore.LinearDecayFunctionBuilder;
import org.elasticsearch.index.query.functionscore.RandomScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
import org.elasticsearch.index.query.functionscore.ScoreFunctionsRegistry;
import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.WeightBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
@ -100,6 +99,7 @@ import org.elasticsearch.search.aggregations.AggregationBinaryParseElement;
import org.elasticsearch.search.aggregations.AggregationParseElement;
import org.elasticsearch.search.aggregations.AggregationPhase;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.aggregations.bucket.children.ChildrenParser;
import org.elasticsearch.search.aggregations.bucket.children.InternalChildren;
@ -175,6 +175,7 @@ import org.elasticsearch.search.aggregations.metrics.valuecount.InternalValueCou
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountParser;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg.AvgBucketParser;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg.AvgBucketPipelineAggregator;
@ -231,9 +232,7 @@ import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.Suggesters;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
/**
@ -241,21 +240,16 @@ import java.util.Set;
*/
public class SearchModule extends AbstractModule {
private final Set<Aggregator.Parser> aggParsers = new HashSet<>();
private final Set<PipelineAggregator.Parser> pipelineAggParsers = new HashSet<>();
private final Highlighters highlighters = new Highlighters();
private final Suggesters suggesters;
/**
* Map from name to score parser and its ParseField.
*/
private final Map<String, Tuple<ParseField, ScoreFunctionParser<?>>> scoreFunctionParsers = new HashMap<>();
private final ScoreFunctionsRegistry scoreFunctionsRegistry = new ScoreFunctionsRegistry(scoreFunctionParsers);
/**
* Query parsers constructed at configure time. These have to be constructed
* at configure time because they depend on things that are registered by
* plugins (function score parsers).
*/
private final Map<String, Tuple<ParseField, QueryParser<?>>> queryParsers = new HashMap<>();
private final ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionParserRegistry = new ParseFieldRegistry<>("score_function");
private final IndicesQueriesRegistry queryParserRegistry = new IndicesQueriesRegistry();
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry = new ParseFieldRegistry<>("aggregation");
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry = new ParseFieldRegistry<>(
"pipline_aggregation");
private final AggregatorParsers aggregatorParsers = new AggregatorParsers(aggregationParserRegistry, pipelineAggregationParserRegistry);
private final Set<Class<? extends FetchSubPhase>> fetchSubPhases = new HashSet<>();
private final Set<SignificanceHeuristicParser> heuristicParsers = new HashSet<>();
private final Set<MovAvgModel.AbstractModelParser> modelParsers = new HashSet<>();
@ -297,21 +291,15 @@ public class SearchModule extends AbstractModule {
*/
public <T extends ScoreFunctionBuilder<T>> void registerScoreFunction(Writeable.Reader<T> reader, ScoreFunctionParser<T> parser,
ParseField functionName) {
for (String name: functionName.getAllNamesIncludedDeprecated()) {
Tuple<ParseField, ScoreFunctionParser<?>> oldValue = scoreFunctionParsers.putIfAbsent(name, new Tuple<>(functionName, parser));
if (oldValue != null) {
throw new IllegalArgumentException(
"Function score parser [" + oldValue.v2() + "] already registered for name [" + name + "]");
}
}
scoreFunctionParserRegistry.register(parser, functionName);
namedWriteableRegistry.register(ScoreFunctionBuilder.class, functionName.getPreferredName(), reader);
}
/**
* Fetch the registry of {@linkplain ScoreFunction}s. This is public so extensions can access the score functions.
*/
public ScoreFunctionsRegistry getScoreFunctionsRegistry() {
return scoreFunctionsRegistry;
public ParseFieldRegistry<ScoreFunctionParser<?>> getScoreFunctionParserRegistry() {
return scoreFunctionParserRegistry;
}
/**
@ -335,20 +323,12 @@ public class SearchModule extends AbstractModule {
*/
public <QB extends QueryBuilder<QB>> void registerQuery(Writeable.Reader<QB> reader, QueryParser<QB> queryParser,
ParseField queryName) {
Tuple<ParseField, QueryParser<?>> parseFieldQueryParserTuple = new Tuple<>(queryName, queryParser);
for (String name: queryName.getAllNamesIncludedDeprecated()) {
Tuple<ParseField, QueryParser<?>> previousValue = queryParsers.putIfAbsent(name, parseFieldQueryParserTuple);
if (previousValue != null) {
throw new IllegalArgumentException("Query parser [" + previousValue.v2() + "] already registered for name [" +
name + "] while trying to register [" + queryParser + "]");
}
}
queryParserRegistry.register(queryParser, queryName);
namedWriteableRegistry.register(QueryBuilder.class, queryName.getPreferredName(), reader);
}
Set<String> getRegisteredQueries() {
return queryParsers.keySet();
public IndicesQueriesRegistry getQueryParserRegistry() {
return queryParserRegistry;
}
public void registerFetchSubPhase(Class<? extends FetchSubPhase> subPhase) {
@ -364,25 +344,67 @@ public class SearchModule extends AbstractModule {
}
/**
* Enabling extending the get module by adding a custom aggregation parser.
* Register an aggregation.
*
* @param parser The parser for the custom aggregator.
* @param reader reads the aggregation builder from a stream
* @param aggregationParser reads the aggregation builder from XContent
* @param aggregationName names by which the aggregation may be parsed. The first name is special because it is the name that the reader
* is registered under.
*/
public void registerAggregatorParser(Aggregator.Parser parser) {
aggParsers.add(parser);
public <AB extends AggregatorBuilder<AB>> void registerAggregation(Writeable.Reader<AB> reader, Aggregator.Parser aggregationParser,
ParseField aggregationName) {
aggregationParserRegistry.register(aggregationParser, aggregationName);
namedWriteableRegistry.register(AggregatorBuilder.class, aggregationName.getPreferredName(), reader);
}
/**
* Register an aggregation.
*
* @deprecated prefer {@link #registerPipelineAggregation(Writeable.Reader, PipelineAggregator.Parser, ParseField)}. Will be removed
* before 5.0.0GA.
*/
@Deprecated // NORELEASE remove this before 5.0.0GA
public void registerAggregatorParser(Aggregator.Parser parser) {
aggregationParserRegistry.register(parser, new ParseField(parser.type()));
namedWriteableRegistry.registerPrototype(AggregatorBuilder.class, parser.getFactoryPrototypes());
}
/**
* Register a pipeline aggregation.
*
* @param reader reads the aggregation builder from a stream
* @param aggregationParser reads the aggregation builder from XContent
* @param aggregationName names by which the aggregation may be parsed. The first name is special because it is the name that the reader
* is registered under.
*/
public <AB extends PipelineAggregatorBuilder<AB>> void registerPipelineAggregation(Writeable.Reader<AB> reader,
PipelineAggregator.Parser aggregationParser, ParseField aggregationName) {
pipelineAggregationParserRegistry.register(aggregationParser, aggregationName);
namedWriteableRegistry.register(PipelineAggregatorBuilder.class, aggregationName.getPreferredName(), reader);
}
/**
* Register a pipeline aggregation.
*
* @deprecated prefer {@link #registerPipelineAggregation(Writeable.Reader, PipelineAggregator.Parser, ParseField)}. Will be removed
* before 5.0.0GA.
*/
@Deprecated // NORELEASE remove this before 5.0.0GA
public void registerPipelineParser(PipelineAggregator.Parser parser) {
pipelineAggParsers.add(parser);
pipelineAggregationParserRegistry.register(parser, new ParseField(parser.type()));
namedWriteableRegistry.registerPrototype(PipelineAggregatorBuilder.class, parser.getFactoryPrototype());
}
public AggregatorParsers getAggregatorParsers() {
return aggregatorParsers;
}
@Override
protected void configure() {
IndicesQueriesRegistry indicesQueriesRegistry = buildQueryParserRegistry();
bind(IndicesQueriesRegistry.class).toInstance(indicesQueriesRegistry);
bind(IndicesQueriesRegistry.class).toInstance(queryParserRegistry);
bind(Suggesters.class).toInstance(suggesters);
configureSearch();
configureAggs(indicesQueriesRegistry);
configureAggs();
configureHighlighters();
configureFetchSubPhase();
configureShapes();
@ -405,15 +427,11 @@ public class SearchModule extends AbstractModule {
bind(InnerHitsFetchSubPhase.class).asEagerSingleton();
}
public IndicesQueriesRegistry buildQueryParserRegistry() {
return new IndicesQueriesRegistry(settings, queryParsers);
}
protected void configureHighlighters() {
highlighters.bind(binder());
}
protected void configureAggs(IndicesQueriesRegistry indicesQueriesRegistry) {
protected void configureAggs() {
MovAvgModelParserMapper movAvgModelParserMapper = new MovAvgModelParserMapper(modelParsers);
@ -432,11 +450,11 @@ public class SearchModule extends AbstractModule {
registerAggregatorParser(new GlobalParser());
registerAggregatorParser(new MissingParser());
registerAggregatorParser(new FilterParser());
registerAggregatorParser(new FiltersParser(indicesQueriesRegistry));
registerAggregatorParser(new FiltersParser(queryParserRegistry));
registerAggregatorParser(new SamplerParser());
registerAggregatorParser(new DiversifiedSamplerParser());
registerAggregatorParser(new TermsParser());
registerAggregatorParser(new SignificantTermsParser(significanceHeuristicParserMapper, indicesQueriesRegistry));
registerAggregatorParser(new SignificantTermsParser(significanceHeuristicParserMapper, queryParserRegistry));
registerAggregatorParser(new RangeParser());
registerAggregatorParser(new DateRangeParser());
registerAggregatorParser(new IpRangeParser());
@ -466,9 +484,8 @@ public class SearchModule extends AbstractModule {
registerPipelineParser(new BucketSelectorParser());
registerPipelineParser(new SerialDiffParser());
AggregatorParsers aggregatorParsers = new AggregatorParsers(aggParsers, pipelineAggParsers, namedWriteableRegistry);
AggregationParseElement aggParseElement = new AggregationParseElement(aggregatorParsers, indicesQueriesRegistry);
AggregationBinaryParseElement aggBinaryParseElement = new AggregationBinaryParseElement(aggregatorParsers, indicesQueriesRegistry);
AggregationParseElement aggParseElement = new AggregationParseElement(aggregatorParsers, queryParserRegistry);
AggregationBinaryParseElement aggBinaryParseElement = new AggregationBinaryParseElement(aggregatorParsers, queryParserRegistry);
AggregationPhase aggPhase = new AggregationPhase(aggParseElement, aggBinaryParseElement);
bind(AggregatorParsers.class).toInstance(aggregatorParsers);
bind(AggregationParseElement.class).toInstance(aggParseElement);
@ -574,7 +591,7 @@ public class SearchModule extends AbstractModule {
registerQuery(IndicesQueryBuilder::new, IndicesQueryBuilder::fromXContent, IndicesQueryBuilder.QUERY_NAME_FIELD);
registerQuery(CommonTermsQueryBuilder::new, CommonTermsQueryBuilder::fromXContent, CommonTermsQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanMultiTermQueryBuilder::new, SpanMultiTermQueryBuilder::fromXContent, SpanMultiTermQueryBuilder.QUERY_NAME_FIELD);
registerQuery(FunctionScoreQueryBuilder::new, c -> FunctionScoreQueryBuilder.fromXContent(scoreFunctionsRegistry, c),
registerQuery(FunctionScoreQueryBuilder::new, c -> FunctionScoreQueryBuilder.fromXContent(scoreFunctionParserRegistry, c),
FunctionScoreQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SimpleQueryStringBuilder::new, SimpleQueryStringBuilder::fromXContent, SimpleQueryStringBuilder.QUERY_NAME_FIELD);
registerQuery(TemplateQueryBuilder::new, TemplateQueryBuilder::fromXContent, TemplateQueryBuilder.QUERY_NAME_FIELD);

View File

@ -277,12 +277,12 @@ public class AggregatorFactories {
Builder builder = new Builder();
int factoriesSize = in.readVInt();
for (int i = 0; i < factoriesSize; i++) {
AggregatorBuilder<?> factory = in.readAggregatorFactory();
AggregatorBuilder<?> factory = in.readAggregatorBuilder();
builder.addAggregator(factory);
}
int pipelineFactoriesSize = in.readVInt();
for (int i = 0; i < pipelineFactoriesSize; i++) {
PipelineAggregatorBuilder<?> factory = in.readPipelineAggregatorFactory();
PipelineAggregatorBuilder<?> factory = in.readPipelineAggregatorBuilder();
builder.addPipelineAggregator(factory);
}
return builder;

View File

@ -19,8 +19,7 @@
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.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
@ -28,71 +27,45 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import static java.util.Collections.unmodifiableMap;
/**
* A registry for all the aggregator parser, also servers as the main parser for the aggregations module
*/
public class AggregatorParsers {
public static final Pattern VALID_AGG_NAME = Pattern.compile("[^\\[\\]>]+");
private final Map<String, Aggregator.Parser> aggParsers;
private final Map<String, PipelineAggregator.Parser> pipelineAggregatorParsers;
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry;
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry;
/**
* Constructs the AggregatorParsers out of all the given parsers
*
* @param aggParsers
* The available aggregator parsers (dynamically injected by the
* {@link org.elasticsearch.search.SearchModule}
* ).
*/
@Inject
public AggregatorParsers(Set<Aggregator.Parser> aggParsers, Set<PipelineAggregator.Parser> pipelineAggregatorParsers,
NamedWriteableRegistry namedWriteableRegistry) {
Map<String, Aggregator.Parser> aggParsersBuilder = new HashMap<>(aggParsers.size());
for (Aggregator.Parser parser : aggParsers) {
aggParsersBuilder.put(parser.type(), parser);
AggregatorBuilder<?> factoryPrototype = parser.getFactoryPrototypes();
namedWriteableRegistry.registerPrototype(AggregatorBuilder.class, factoryPrototype);
}
this.aggParsers = unmodifiableMap(aggParsersBuilder);
Map<String, PipelineAggregator.Parser> pipelineAggregatorParsersBuilder = new HashMap<>(pipelineAggregatorParsers.size());
for (PipelineAggregator.Parser parser : pipelineAggregatorParsers) {
pipelineAggregatorParsersBuilder.put(parser.type(), parser);
PipelineAggregatorBuilder<?> factoryPrototype = parser.getFactoryPrototype();
namedWriteableRegistry.registerPrototype(PipelineAggregatorBuilder.class, factoryPrototype);
}
this.pipelineAggregatorParsers = unmodifiableMap(pipelineAggregatorParsersBuilder);
public AggregatorParsers(ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry,
ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry) {
this.aggregationParserRegistry = aggregationParserRegistry;
this.pipelineAggregationParserRegistry = pipelineAggregationParserRegistry;
}
/**
* Returns the parser that is registered under the given aggregation type.
*
* @param type The aggregation type
* @return The parser associated with the given aggregation type.
* @param type The aggregation type
* @param parser the parser the type was read from. Used to lookup the ParseFieldMatcher and for making error messages.
* @return The parser associated with the given aggregation type or null if it wasn't found.
*/
public Aggregator.Parser parser(String type) {
return aggParsers.get(type);
public Aggregator.Parser parser(String type, XContentParser parser) {
return aggregationParserRegistry.lookupReturningNullIfNotFound(type, parser);
}
/**
* Returns the parser that is registered under the given pipeline aggregator
* type.
* Returns the parser that is registered under the given pipeline aggregator type.
*
* @param type
* The pipeline aggregator type
* @return The parser associated with the given pipeline aggregator type.
* @param type The pipeline aggregator type
* @param parser the parser the type was read from. Used to lookup the ParseFieldMatcher and for making error messages.
* @return The parser associated with the given pipeline aggregator type or null if it wasn't found.
*/
public PipelineAggregator.Parser pipelineAggregator(String type) {
return pipelineAggregatorParsers.get(type);
public PipelineAggregator.Parser pipelineParser(String type, XContentParser parser) {
return pipelineAggregationParserRegistry.lookupReturningNullIfNotFound(type, parser);
}
/**
@ -109,7 +82,6 @@ public class AggregatorParsers {
return parseAggregators(parser, parseContext, 0);
}
private AggregatorFactories.Builder parseAggregators(XContentParser parser, QueryParseContext parseContext, int level)
throws IOException {
Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
@ -194,9 +166,9 @@ public class AggregatorParsers {
+ aggregationName + "]: [" + pipelineAggregatorFactory + "] and [" + fieldName + "]");
}
Aggregator.Parser aggregatorParser = parser(fieldName);
Aggregator.Parser aggregatorParser = parser(fieldName, parser);
if (aggregatorParser == null) {
PipelineAggregator.Parser pipelineAggregatorParser = pipelineAggregator(fieldName);
PipelineAggregator.Parser pipelineAggregatorParser = pipelineParser(fieldName, parser);
if (pipelineAggregatorParser == null) {
throw new ParsingException(parser.getTokenLocation(),
"Could not find aggregator type [" + fieldName + "] in [" + aggregationName + "]");

View File

@ -168,8 +168,9 @@ public class MultiSearchRequestTests extends ESTestCase {
}
private IndicesQueriesRegistry registry() {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
return new IndicesQueriesRegistry(Settings.EMPTY,
singletonMap(MatchAllQueryBuilder.NAME, new Tuple<>(MatchAllQueryBuilder.QUERY_NAME_FIELD, parser)));
registry.register(parser, MatchAllQueryBuilder.QUERY_NAME_FIELD);
return registry;
}
}

View File

@ -121,7 +121,7 @@ public class IndexModuleTests extends ESTestCase {
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, scriptContextRegistry, scriptSettings);
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, emptyMap());
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry();
return new NodeServicesProvider(threadPool, bigArrays, client, scriptService, indicesQueriesRegistry, circuitBreakerService);
}

View File

@ -101,17 +101,15 @@ public class PercolatorQueryCacheTests extends ESTestCase {
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
.build();
Map<String, Tuple<ParseField, QueryParser<?>>> queryParsers = new HashMap<>();
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry();
QueryParser<TermQueryBuilder> termParser = TermQueryBuilder::fromXContent;
queryParsers.put(TermQueryBuilder.NAME, new Tuple<>(TermQueryBuilder.QUERY_NAME_FIELD, termParser));
indicesQueriesRegistry.register(termParser, TermQueryBuilder.QUERY_NAME_FIELD);
QueryParser<WildcardQueryBuilder> wildcardParser = WildcardQueryBuilder::fromXContent;
queryParsers.put(WildcardQueryBuilder.NAME, new Tuple<>(WildcardQueryBuilder.QUERY_NAME_FIELD, wildcardParser));
indicesQueriesRegistry.register(wildcardParser, WildcardQueryBuilder.QUERY_NAME_FIELD);
QueryParser<BoolQueryBuilder> boolQueryParser = BoolQueryBuilder::fromXContent;
queryParsers.put(BoolQueryBuilder.NAME, new Tuple<>(BoolQueryBuilder.QUERY_NAME_FIELD, boolQueryParser));
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, queryParsers);
indicesQueriesRegistry.register(boolQueryParser, BoolQueryBuilder.QUERY_NAME_FIELD);
Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("_index", ClusterState.UNKNOWN_UUID), indexSettings);
SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap());
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);

View File

@ -69,7 +69,7 @@ public class InnerHitBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -67,7 +67,7 @@ public class InnerHitsBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -496,7 +496,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase {
} catch(IllegalArgumentException e) {
assertThat(e.getMessage(), equalTo("failed to parse filter for alias [invalid_alias]"));
assertThat(e.getCause(), instanceOf(ParsingException.class));
assertThat(e.getCause().getMessage(), equalTo("No query registered for [invalid]"));
assertThat(e.getCause().getMessage(), equalTo("no [query] registered for [invalid]"));
}
}

View File

@ -19,10 +19,13 @@
package org.elasticsearch.search;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
@ -32,6 +35,7 @@ import org.elasticsearch.search.highlight.PlainHighlighter;
import org.elasticsearch.search.suggest.CustomSuggester;
import org.elasticsearch.search.suggest.phrase.PhraseSuggester;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -82,31 +86,32 @@ public class SearchModuleTests extends ModuleTestCase {
SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> module
.registerQuery(TermQueryBuilder::new, TermQueryBuilder::fromXContent, TermQueryBuilder.QUERY_NAME_FIELD));
assertThat(e.getMessage(), containsString("already registered for name [term] while trying to register [org.elasticsearch."));
assertThat(e.getMessage(), containsString("] already registered for [query][term] while trying to register [org.elasticsearch."));
}
public void testRegisteredQueries() {
public void testRegisteredQueries() throws IOException {
SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry());
List<String> allSupportedQueries = new ArrayList<>();
Collections.addAll(allSupportedQueries, NON_DEPRECATED_QUERIES);
Collections.addAll(allSupportedQueries, DEPRECATED_QUERIES);
String[] supportedQueries = allSupportedQueries.toArray(new String[allSupportedQueries.size()]);
assertThat(module.getRegisteredQueries(), containsInAnyOrder(supportedQueries));
assertThat(module.getQueryParserRegistry().getNames(), containsInAnyOrder(supportedQueries));
IndicesQueriesRegistry indicesQueriesRegistry = module.buildQueryParserRegistry();
IndicesQueriesRegistry indicesQueriesRegistry = module.getQueryParserRegistry();
XContentParser dummyParser = XContentHelper.createParser(new BytesArray("{}"));
dummyParser.setParseFieldMatcher(ParseFieldMatcher.EMPTY);
for (String queryName : supportedQueries) {
QueryParser<?> queryParser = indicesQueriesRegistry.getQueryParser(queryName,
ParseFieldMatcher.EMPTY, new XContentLocation(-1, -1));
assertThat(queryParser, notNullValue());
indicesQueriesRegistry.lookup(queryName, dummyParser);
}
dummyParser.setParseFieldMatcher(ParseFieldMatcher.STRICT);
for (String queryName : NON_DEPRECATED_QUERIES) {
QueryParser<?> queryParser = indicesQueriesRegistry.getQueryParser(queryName,
ParseFieldMatcher.STRICT, new XContentLocation(-1, -1));
QueryParser<?> queryParser = indicesQueriesRegistry.lookup(queryName, dummyParser);
assertThat(queryParser, notNullValue());
}
for (String queryName : DEPRECATED_QUERIES) {
try {
indicesQueriesRegistry.getQueryParser(queryName, ParseFieldMatcher.STRICT, new XContentLocation(-1, -1));
indicesQueriesRegistry.lookup(queryName, dummyParser);
fail("query is deprecated, getQueryParser should have failed in strict mode");
} catch(IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("Deprecated field [" + queryName + "] used"));

View File

@ -235,7 +235,7 @@ public abstract class BaseAggregationTestCase<AB extends AggregatorBuilder<AB>>
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.type.name(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
AggregatorBuilder newAgg = aggParsers.parser(testAgg.getType()).parse(testAgg.name, parser, parseContext);
AggregatorBuilder<?> newAgg = aggParsers.parser(testAgg.getType(), parser).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

@ -237,7 +237,7 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(testAgg.type(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
PipelineAggregatorBuilder newAgg = aggParsers.pipelineAggregator(testAgg.getWriteableName()).parse(testAgg.name(), parser,
PipelineAggregatorBuilder<?> newAgg = aggParsers.pipelineParser(testAgg.getWriteableName(), parser).parse(testAgg.name(), parser,
parseContext);
assertSame(XContentParser.Token.END_OBJECT, parser.currentToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());
@ -256,10 +256,9 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
public void testSerialization() throws IOException {
AF testAgg = createTestAggregatorFactory();
try (BytesStreamOutput output = new BytesStreamOutput()) {
testAgg.writeTo(output);
output.writePipelineAggregatorBuilder(testAgg);
try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) {
PipelineAggregatorBuilder prototype = aggParsers.pipelineAggregator(testAgg.getWriteableName()).getFactoryPrototype();
PipelineAggregatorBuilder deserializedQuery = prototype.readFrom(in);
PipelineAggregatorBuilder deserializedQuery = in.readPipelineAggregatorBuilder();
assertEquals(deserializedQuery, testAgg);
assertEquals(deserializedQuery.hashCode(), testAgg.hashCode());
assertNotSame(deserializedQuery, testAgg);
@ -297,11 +296,10 @@ public abstract class BasePipelineAggregationTestCase<AF extends PipelineAggrega
// argument
private AF copyAggregation(AF agg) throws IOException {
try (BytesStreamOutput output = new BytesStreamOutput()) {
agg.writeTo(output);
output.writePipelineAggregatorBuilder(agg);
try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) {
PipelineAggregatorBuilder prototype = aggParsers.pipelineAggregator(agg.getWriteableName()).getFactoryPrototype();
@SuppressWarnings("unchecked")
AF secondAgg = (AF) prototype.readFrom(in);
AF secondAgg = (AF) in.readPipelineAggregatorBuilder();
return secondAgg;
}
}

View File

@ -35,9 +35,9 @@ import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.ChiSquare;
@ -246,7 +246,7 @@ public class SignificanceHeuristicTests extends ESTestCase {
protected void checkParseException(SignificanceHeuristicParserMapper heuristicParserMapper, SearchContext searchContext,
String faultyHeuristicDefinition, String expectedError) throws IOException {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.emptyMap());
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
try {
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"text\", " + faultyHeuristicDefinition + ",\"min_doc_count\":200}");
QueryParseContext parseContext = new QueryParseContext(registry);
@ -271,7 +271,7 @@ public class SignificanceHeuristicTests extends ESTestCase {
private SignificanceHeuristic parseSignificanceHeuristic(SignificanceHeuristicParserMapper heuristicParserMapper,
SearchContext searchContext, XContentParser stParser) throws IOException {
IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.emptyMap());
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
QueryParseContext parseContext = new QueryParseContext(registry);
parseContext.reset(stParser);
parseContext.parseFieldMatcher(ParseFieldMatcher.STRICT);

View File

@ -116,7 +116,7 @@ public class MovAvgTests extends BasePipelineAggregationTestCase<MovAvgPipelineA
assertSame(XContentParser.Token.FIELD_NAME, parser.nextToken());
assertEquals(expected.type(), parser.currentName());
assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
PipelineAggregatorBuilder<?> newAgg = aggParsers.pipelineAggregator(expected.getWriteableName()).parse(expected.name(), parser,
PipelineAggregatorBuilder<?> newAgg = aggParsers.pipelineParser(expected.getWriteableName(), parser).parse(expected.name(), parser,
parseContext);
assertSame(XContentParser.Token.END_OBJECT, parser.currentToken());
assertSame(XContentParser.Token.END_OBJECT, parser.nextToken());

View File

@ -84,7 +84,7 @@ public class HighlightBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -71,7 +71,7 @@ public class QueryRescoreBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -20,7 +20,6 @@
package org.elasticsearch.search.searchafter;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -42,7 +41,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
@ -57,11 +55,9 @@ public class SearchAfterBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new IndicesQueriesRegistry();
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
indicesQueriesRegistry = new IndicesQueriesRegistry(
Settings.builder().build(),
Collections.singletonMap(
MatchAllQueryBuilder.NAME, new Tuple<>(MatchAllQueryBuilder.QUERY_NAME_FIELD, parser)));
indicesQueriesRegistry.register(parser, MatchAllQueryBuilder.QUERY_NAME_FIELD);
}
@AfterClass

View File

@ -100,7 +100,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
};
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -51,7 +51,7 @@ public class SortBuilderTests extends ESTestCase {
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).getQueryParserRegistry();
}
@AfterClass

View File

@ -60,7 +60,7 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
public static void init() throws IOException {
namedWriteableRegistry = new NamedWriteableRegistry();
SearchModule searchModule = new SearchModule(Settings.EMPTY, namedWriteableRegistry);
queriesRegistry = searchModule.buildQueryParserRegistry();
queriesRegistry = searchModule.getQueryParserRegistry();
suggesters = searchModule.getSuggesters();
parseFieldMatcher = ParseFieldMatcher.STRICT;
}

View File

@ -36,7 +36,6 @@ import org.elasticsearch.search.suggest.phrase.PhraseSuggestionContext.DirectCan
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
@ -111,7 +110,7 @@ public class DirectCandidateGeneratorTests extends ESTestCase{
* creates random candidate generator, renders it to xContent and back to new instance that should be equal to original
*/
public void testFromXContent() throws IOException {
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry(Settings.EMPTY, Collections.emptyMap()));
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry());
context.parseFieldMatcher(new ParseFieldMatcher(Settings.EMPTY));
for (int runs = 0; runs < NUMBER_OF_RUNS; runs++) {
DirectCandidateGeneratorBuilder generator = randomCandidateGenerator();
@ -153,7 +152,7 @@ public class DirectCandidateGeneratorTests extends ESTestCase{
* test that bad xContent throws exception
*/
public void testIllegalXContent() throws IOException {
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry(Settings.EMPTY, Collections.emptyMap()));
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry());
context.parseFieldMatcher(new ParseFieldMatcher(Settings.EMPTY));
// test missing fieldname

View File

@ -51,7 +51,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -94,8 +93,7 @@ public abstract class SmoothingModelTestCase extends ESTestCase {
* Test that creates new smoothing model from a random test smoothing model and checks both for equality
*/
public void testFromXContent() throws IOException {
QueryParseContext context = new QueryParseContext(
new IndicesQueriesRegistry(Settings.builder().build(), Collections.emptyMap()));
QueryParseContext context = new QueryParseContext(new IndicesQueriesRegistry());
context.parseFieldMatcher(new ParseFieldMatcher(Settings.EMPTY));
SmoothingModel testModel = createTestModel();

View File

@ -35,7 +35,7 @@ setup:
invalid_query: {}
- is_false: valid
- match: {error: 'org.elasticsearch.common.ParsingException: No query registered for [invalid_query]'}
- match: {error: 'org.elasticsearch.common.ParsingException: no [query] registered for [invalid_query]'}
- do:
indices.validate_query: