Merge pull request #17205 from cbuescher/sort-use-sortbuilders

Switch to using SortBuilder instead of BytesReference in serialization of SearchSource and elsewhere.
This commit is contained in:
Christoph Büscher 2016-03-23 16:25:02 +01:00
commit a3fc4c0370
32 changed files with 242 additions and 1084 deletions

View File

@ -21,7 +21,6 @@ package org.elasticsearch.action.percolate;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
@ -127,7 +126,7 @@ public class PercolateRequestBuilder extends ActionRequestBuilder<PercolateReque
/**
* Delegates to {@link PercolateSourceBuilder#addSort(SortBuilder)}
*/
public PercolateRequestBuilder addSort(SortBuilder sort) {
public PercolateRequestBuilder addSort(SortBuilder<?> sort) {
sourceBuilder().addSort(sort);
return this;
}

View File

@ -48,13 +48,13 @@ import java.util.Map;
public class PercolateSourceBuilder extends ToXContentToBytes {
private DocBuilder docBuilder;
private QueryBuilder queryBuilder;
private QueryBuilder<?> queryBuilder;
private Integer size;
private List<SortBuilder> sorts;
private List<SortBuilder<?>> sorts;
private Boolean trackScores;
private HighlightBuilder highlightBuilder;
private List<AggregatorBuilder<?>> aggregationBuilders;
private List<PipelineAggregatorBuilder> pipelineAggregationBuilders;
private List<PipelineAggregatorBuilder<?>> pipelineAggregationBuilders;
/**
* Sets the document to run the percolate queries against.
@ -68,7 +68,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
* Sets a query to reduce the number of percolate queries to be evaluated and score the queries that match based
* on this query.
*/
public PercolateSourceBuilder setQueryBuilder(QueryBuilder queryBuilder) {
public PercolateSourceBuilder setQueryBuilder(QueryBuilder<?> queryBuilder) {
this.queryBuilder = queryBuilder;
return this;
}
@ -98,7 +98,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
*
* By default the matching percolator queries are returned in an undefined order.
*/
public PercolateSourceBuilder addSort(SortBuilder sort) {
public PercolateSourceBuilder addSort(SortBuilder<?> sort) {
if (sorts == null) {
sorts = new ArrayList<>();
}
@ -137,7 +137,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
/**
* Add an aggregation definition.
*/
public PercolateSourceBuilder addAggregation(PipelineAggregatorBuilder aggregationBuilder) {
public PercolateSourceBuilder addAggregation(PipelineAggregatorBuilder<?> aggregationBuilder) {
if (pipelineAggregationBuilders == null) {
pipelineAggregationBuilders = new ArrayList<>();
}
@ -160,10 +160,8 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
}
if (sorts != null) {
builder.startArray("sort");
for (SortBuilder sort : sorts) {
builder.startObject();
for (SortBuilder<?> sort : sorts) {
sort.toXContent(builder, params);
builder.endObject();
}
builder.endArray();
}
@ -182,7 +180,7 @@ public class PercolateSourceBuilder extends ToXContentToBytes {
}
}
if (pipelineAggregationBuilders != null) {
for (PipelineAggregatorBuilder aggregation : pipelineAggregationBuilders) {
for (PipelineAggregatorBuilder<?> aggregation : pipelineAggregationBuilders) {
aggregation.toXContent(builder, params);
}
}

View File

@ -37,12 +37,13 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.phrase.SmoothingModel;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
@ -687,21 +688,21 @@ public abstract class StreamInput extends InputStream {
/**
* Reads a {@link AggregatorBuilder} from the current stream
*/
public AggregatorBuilder readAggregatorFactory() throws IOException {
public AggregatorBuilder<?> readAggregatorFactory() throws IOException {
return readNamedWriteable(AggregatorBuilder.class);
}
/**
* Reads a {@link PipelineAggregatorBuilder} from the current stream
*/
public PipelineAggregatorBuilder readPipelineAggregatorFactory() throws IOException {
public PipelineAggregatorBuilder<?> readPipelineAggregatorFactory() throws IOException {
return readNamedWriteable(PipelineAggregatorBuilder.class);
}
/**
* Reads a {@link QueryBuilder} from the current stream
*/
public QueryBuilder readQuery() throws IOException {
public QueryBuilder<?> readQuery() throws IOException {
return readNamedWriteable(QueryBuilder.class);
}
@ -726,6 +727,13 @@ public abstract class StreamInput extends InputStream {
return readNamedWriteable(SuggestionBuilder.class);
}
/**
* Reads a {@link SortBuilder} from the current stream
*/
public SortBuilder<?> readSortBuilder() throws IOException {
return readNamedWriteable(SortBuilder.class);
}
/**
* Reads a {@link org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder} from the current stream
*/

View File

@ -36,13 +36,13 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.completion.context.QueryContext;
import org.elasticsearch.search.suggest.phrase.SmoothingModel;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.phrase.SmoothingModel;
import org.elasticsearch.tasks.Task;
import org.joda.time.ReadableInstant;
import java.io.EOFException;
@ -532,7 +532,7 @@ public abstract class StreamOutput extends OutputStream {
}
}
public void writeOptionalWriteable(@Nullable Writeable writeable) throws IOException {
public void writeOptionalWriteable(@Nullable Writeable<?> writeable) throws IOException {
if (writeable != null) {
writeBoolean(true);
writeable.writeTo(this);
@ -663,7 +663,7 @@ public abstract class StreamOutput extends OutputStream {
/**
* Writes a {@link NamedWriteable} to the current stream, by first writing its name and then the object itself
*/
void writeNamedWriteable(NamedWriteable namedWriteable) throws IOException {
void writeNamedWriteable(NamedWriteable<?> namedWriteable) throws IOException {
writeString(namedWriteable.getWriteableName());
namedWriteable.writeTo(this);
}
@ -685,7 +685,7 @@ public abstract class StreamOutput extends OutputStream {
/**
* Writes a {@link QueryBuilder} to the current stream
*/
public void writeQuery(QueryBuilder queryBuilder) throws IOException {
public void writeQuery(QueryBuilder<?> queryBuilder) throws IOException {
writeNamedWriteable(queryBuilder);
}
@ -745,8 +745,15 @@ public abstract class StreamOutput extends OutputStream {
/**
* Writes a {@link SuggestionBuilder} to the current stream
*/
public void writeSuggestion(SuggestionBuilder suggestion) throws IOException {
public void writeSuggestion(SuggestionBuilder<?> suggestion) throws IOException {
writeNamedWriteable(suggestion);
}
/**
* Writes a {@link SortBuilder} to the current stream
*/
public void writeSortBuilder(SortBuilder<?> sort) throws IOException {
writeNamedWriteable(sort);
}
}

View File

@ -27,7 +27,7 @@ import org.elasticsearch.search.fetch.source.FetchSourceParseElement;
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 org.elasticsearch.search.sort.SortBuilder;
import java.io.IOException;
@ -35,7 +35,6 @@ public class InnerHitsQueryParserHelper {
public static final InnerHitsQueryParserHelper INSTANCE = new InnerHitsQueryParserHelper();
private static final SortParseElement sortParseElement = new SortParseElement();
private static final FetchSourceParseElement sourceParseElement = new FetchSourceParseElement();
private static final HighlighterParseElement highlighterParseElement = new HighlighterParseElement();
private static final ScriptFieldsParseElement scriptFieldsParseElement = new ScriptFieldsParseElement();
@ -54,10 +53,10 @@ public class InnerHitsQueryParserHelper {
if ("name".equals(fieldName)) {
innerHitName = parser.textOrNull();
} else {
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sortParseElement, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
}
} else {
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sortParseElement, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
}
}
} catch (Exception e) {
@ -67,10 +66,10 @@ public class InnerHitsQueryParserHelper {
}
public static void parseCommonInnerHitOptions(XContentParser parser, XContentParser.Token token, String fieldName, SubSearchContext subSearchContext,
SortParseElement sortParseElement, FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement,
FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement,
ScriptFieldsParseElement scriptFieldsParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement) throws Exception {
if ("sort".equals(fieldName)) {
sortParseElement.parse(parser, subSearchContext);
SortBuilder.parseSort(parser, subSearchContext);
} else if ("_source".equals(fieldName)) {
sourceParseElement.parse(parser, subSearchContext);
} else if (token == XContentParser.Token.START_OBJECT) {

View File

@ -1,134 +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.support;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import java.io.IOException;
/**
* A helper that helps with parsing inner queries of the nested query.
* 1) Takes into account that type nested path can appear before or after the inner query
* 2) Updates the {@link NestedScope} when parsing the inner query.
*/
public class NestedInnerQueryParseSupport {
protected final QueryShardContext shardContext;
protected final QueryParseContext parseContext;
private BytesReference source;
private Query innerFilter;
protected String path;
private boolean filterParsed = false;
protected boolean filterFound = false;
protected ObjectMapper nestedObjectMapper;
public NestedInnerQueryParseSupport(XContentParser parser, QueryShardContext context) {
shardContext = context;
parseContext = shardContext.parseContext();
shardContext.reset(parser);
}
public void filter() throws IOException {
if (path != null) {
setPathLevel();
try {
innerFilter = QueryBuilder.rewriteQuery(parseContext.parseInnerQueryBuilder(),
this.shardContext).toFilter(this.shardContext);
} finally {
resetPathLevel();
}
filterParsed = true;
} else {
source = XContentFactory.smileBuilder().copyCurrentStructure(parseContext.parser()).bytes();
}
filterFound = true;
}
public Query getInnerFilter() throws IOException {
if (filterParsed) {
return innerFilter;
} else {
if (path == null) {
throw new QueryShardException(shardContext, "[nested] requires 'path' field");
}
if (!filterFound) {
throw new QueryShardException(shardContext, "[nested] requires either 'query' or 'filter' field");
}
setPathLevel();
XContentParser old = parseContext.parser();
try {
XContentParser innerParser = XContentHelper.createParser(source);
parseContext.parser(innerParser);
innerFilter = QueryBuilder.rewriteQuery(parseContext.parseInnerQueryBuilder(),
this.shardContext).toFilter(this.shardContext);
filterParsed = true;
return innerFilter;
} finally {
resetPathLevel();
parseContext.parser(old);
}
}
}
public void setPath(String path) {
this.path = path;
nestedObjectMapper = shardContext.getObjectMapper(path);
if (nestedObjectMapper == null) {
throw new QueryShardException(shardContext, "[nested] failed to find nested object under path [" + path + "]");
}
if (!nestedObjectMapper.nested().isNested()) {
throw new QueryShardException(shardContext, "[nested] nested object under path [" + path + "] is not of nested type");
}
}
public String getPath() {
return path;
}
public ObjectMapper getNestedObjectMapper() {
return nestedObjectMapper;
}
public boolean filterFound() {
return filterFound;
}
private void setPathLevel() {
shardContext.nestedScope().nextLevel(nestedObjectMapper);
}
private void resetPathLevel() {
shardContext.nestedScope().previousLevel();
}
}

View File

@ -226,6 +226,11 @@ import org.elasticsearch.search.highlight.Highlighters;
import org.elasticsearch.search.query.QueryPhase;
import org.elasticsearch.search.rescore.QueryRescorerBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.GeoDistanceSortBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.Suggesters;
import org.elasticsearch.search.suggest.SuggestionBuilder;
@ -346,6 +351,7 @@ public class SearchModule extends AbstractModule {
configureFetchSubPhase();
configureShapes();
configureRescorers();
configureSorts();
}
protected void configureFetchSubPhase() {
@ -489,6 +495,13 @@ public class SearchModule extends AbstractModule {
namedWriteableRegistry.registerPrototype(RescoreBuilder.class, QueryRescorerBuilder.PROTOTYPE);
}
private void configureSorts() {
namedWriteableRegistry.registerPrototype(SortBuilder.class, GeoDistanceSortBuilder.PROTOTYPE);
namedWriteableRegistry.registerPrototype(SortBuilder.class, ScoreSortBuilder.PROTOTYPE);
namedWriteableRegistry.registerPrototype(SortBuilder.class, ScriptSortBuilder.PROTOTYPE);
namedWriteableRegistry.registerPrototype(SortBuilder.class, FieldSortBuilder.PROTOTYPE);
}
private void registerBuiltinFunctionScoreParsers() {
registerFunctionScoreParser(new ScriptScoreFunctionParser());
registerFunctionScoreParser(new GaussDecayFunctionParser());

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search;
import com.carrotsearch.hppc.ObjectFloatHashMap;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
@ -41,7 +42,6 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.XContentParser;
@ -92,6 +92,7 @@ import org.elasticsearch.search.query.QuerySearchResultProvider;
import org.elasticsearch.search.query.ScrollQuerySearchResult;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.Suggesters;
import org.elasticsearch.threadpool.ThreadPool;
@ -99,6 +100,7 @@ import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
@ -683,33 +685,13 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
context.parsedPostFilter(queryShardContext.toQuery(source.postFilter()));
}
if (source.sorts() != null) {
XContentParser completeSortParser = null;
try {
XContentBuilder completeSortBuilder = XContentFactory.jsonBuilder();
completeSortBuilder.startObject();
completeSortBuilder.startArray("sort");
for (BytesReference sort : source.sorts()) {
XContentParser parser = XContentFactory.xContent(sort).createParser(sort);
parser.nextToken();
completeSortBuilder.copyCurrentStructure(parser);
Optional<Sort> optionalSort = SortBuilder.buildSort(source.sorts(), context.getQueryShardContext());
if (optionalSort.isPresent()) {
context.sort(optionalSort.get());
}
completeSortBuilder.endArray();
completeSortBuilder.endObject();
BytesReference completeSortBytes = completeSortBuilder.bytes();
completeSortParser = XContentFactory.xContent(completeSortBytes).createParser(completeSortBytes);
completeSortParser.nextToken();
completeSortParser.nextToken();
completeSortParser.nextToken();
this.elementParsers.get("sort").parse(completeSortParser, context);
} catch (Exception e) {
String sSource = "_na_";
try {
sSource = source.toString();
} catch (Throwable e1) {
// ignore
}
XContentLocation location = completeSortParser != null ? completeSortParser.getTokenLocation() : null;
throw new SearchParseException(context, "failed to parse sort source [" + sSource + "]", location, e);
} catch (IOException e) {
throw new SearchContextException(context, "failed to create sort elements", e);
}
}
context.trackScores(source.trackScores());

View File

@ -21,13 +21,9 @@ package org.elasticsearch.search.aggregations.metrics.tophits;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.AggregatorBuilder;
@ -38,6 +34,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
@ -57,7 +54,7 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
private boolean explain = false;
private boolean version = false;
private boolean trackScores = false;
private List<BytesReference> sorts = null;
private List<SortBuilder<?>> sorts = null;
private HighlightBuilder highlightBuilder;
private List<String> fieldNames;
private List<String> fieldDataFields;
@ -119,6 +116,9 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
if (order == null) {
throw new IllegalArgumentException("sort [order] must not be null: [" + name + "]");
}
if (name.equals(ScoreSortBuilder.NAME)) {
sort(SortBuilders.scoreSort().order(order));
}
sort(SortBuilders.fieldSort(name).order(order));
return this;
}
@ -133,6 +133,9 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
if (name == null) {
throw new IllegalArgumentException("sort [name] must not be null: [" + name + "]");
}
if (name.equals(ScoreSortBuilder.NAME)) {
sort(SortBuilders.scoreSort());
}
sort(SortBuilders.fieldSort(name));
return this;
}
@ -140,39 +143,28 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
/**
* Adds a sort builder.
*/
public TopHitsAggregatorBuilder sort(SortBuilder sort) {
public TopHitsAggregatorBuilder sort(SortBuilder<?> sort) {
if (sort == null) {
throw new IllegalArgumentException("[sort] must not be null: [" + name + "]");
}
try {
if (sorts == null) {
sorts = new ArrayList<>();
}
// NORELEASE when sort has been refactored and made writeable
// add the sortBuilcer to the List directly instead of
// serialising to XContent
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
sort.toXContent(builder, EMPTY_PARAMS);
builder.endObject();
sorts.add(builder.bytes());
} catch (IOException e) {
throw new RuntimeException(e);
}
sorts.add(sort);
return this;
}
/**
* Adds a sort builder.
*/
public TopHitsAggregatorBuilder sorts(List<BytesReference> sorts) {
public TopHitsAggregatorBuilder sorts(List<SortBuilder<?>> sorts) {
if (sorts == null) {
throw new IllegalArgumentException("[sorts] must not be null: [" + name + "]");
}
if (this.sorts == null) {
this.sorts = new ArrayList<>();
}
for (BytesReference sort : sorts) {
for (SortBuilder<?> sort : sorts) {
this.sorts.add(sort);
}
return this;
@ -181,7 +173,7 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
/**
* Gets the bytes representing the sort builders for this request.
*/
public List<BytesReference> sorts() {
public List<SortBuilder<?>> sorts() {
return sorts;
}
@ -509,10 +501,8 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
}
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);
for (SortBuilder<?> sort : sorts) {
sort.toXContent(builder, params);
}
builder.endArray();
}
@ -562,9 +552,9 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
factory.size = in.readVInt();
if (in.readBoolean()) {
int size = in.readVInt();
List<BytesReference> sorts = new ArrayList<>();
List<SortBuilder<?>> sorts = new ArrayList<>();
for (int i = 0; i < size; i++) {
sorts.add(in.readBytesReference());
sorts.add(in.readSortBuilder());
}
factory.sorts = sorts;
}
@ -612,8 +602,8 @@ public class TopHitsAggregatorBuilder extends AggregatorBuilder<TopHitsAggregato
out.writeBoolean(hasSorts);
if (hasSorts) {
out.writeVInt(sorts.size());
for (BytesReference sort : sorts) {
out.writeBytesReference(sort);
for (SortBuilder<?> sort : sorts) {
out.writeSortBuilder(sort);
}
}
out.writeBoolean(trackScores);

View File

@ -19,12 +19,7 @@
package org.elasticsearch.search.aggregations.metrics.tophits;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesReference;
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.apache.lucene.search.Sort;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.Aggregator;
@ -35,27 +30,27 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsContext;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsFetchSubPhase;
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.SubSearchContext;
import org.elasticsearch.search.sort.SortParseElement;
import org.elasticsearch.search.sort.SortBuilder;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregatorFactory> {
private static final SortParseElement sortParseElement = new SortParseElement();
private final int from;
private final int size;
private final boolean explain;
private final boolean version;
private final boolean trackScores;
private final List<BytesReference> sorts;
private final List<SortBuilder<?>> sorts;
private final HighlightBuilder highlightBuilder;
private final List<String> fieldNames;
private final List<String> fieldDataFields;
@ -63,7 +58,7 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
private final FetchSourceContext fetchSourceContext;
public TopHitsAggregatorFactory(String name, Type type, int from, int size, boolean explain, boolean version, boolean trackScores,
List<BytesReference> sorts, HighlightBuilder highlightBuilder, List<String> fieldNames, List<String> fieldDataFields,
List<SortBuilder<?>> sorts, HighlightBuilder highlightBuilder, List<String> fieldNames, List<String> fieldDataFields,
List<ScriptField> scriptFields, FetchSourceContext fetchSourceContext, AggregationContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
@ -90,27 +85,9 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
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);
Optional<Sort> optionalSort = SortBuilder.buildSort(sorts, subSearchContext.getQueryShardContext());
if (optionalSort.isPresent()) {
subSearchContext.sort(optionalSort.get());
}
}
if (fieldNames != null) {

View File

@ -19,9 +19,6 @@
package org.elasticsearch.search.aggregations.metrics.tophits;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
@ -30,6 +27,8 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@ -124,9 +123,7 @@ public class TopHitsParser implements Aggregator.Parser {
} 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());
List<SortBuilder<?>> sorts = SortBuilder.fromXContent(context);
factory.sorts(sorts);
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
@ -157,11 +154,7 @@ public class TopHitsParser implements Aggregator.Parser {
}
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());
}
List<SortBuilder<?>> sorts = SortBuilder.fromXContent(context);
factory.sorts(sorts);
} else if (context.parseFieldMatcher().match(currentFieldName, SearchSourceBuilder._SOURCE_FIELD)) {
factory.fetchSource(FetchSourceContext.parse(parser, context));

View File

@ -52,6 +52,7 @@ import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
@ -139,7 +140,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
private Boolean version;
private List<BytesReference> sorts;
private List<SortBuilder<?>> sorts;
private boolean trackScores = false;
@ -336,6 +337,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
* The sort ordering
*/
public SearchSourceBuilder sort(String name, SortOrder order) {
if (name.equals(ScoreSortBuilder.NAME)) {
return sort(SortBuilders.scoreSort().order(order));
}
return sort(SortBuilders.fieldSort(name).order(order));
}
@ -346,32 +350,27 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
* The name of the field to sort by
*/
public SearchSourceBuilder sort(String name) {
if (name.equals(ScoreSortBuilder.NAME)) {
return sort(SortBuilders.scoreSort());
}
return sort(SortBuilders.fieldSort(name));
}
/**
* Adds a sort builder.
*/
public SearchSourceBuilder sort(SortBuilder sort) {
try {
public SearchSourceBuilder sort(SortBuilder<?> sort) {
if (sorts == null) {
sorts = new ArrayList<>();
}
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
sort.toXContent(builder, EMPTY_PARAMS);
builder.endObject();
sorts.add(builder.bytes());
sorts.add(sort);
return this;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* Gets the bytes representing the sort builders for this request.
*/
public List<BytesReference> sorts() {
public List<SortBuilder<?>> sorts() {
return sorts;
}
@ -907,9 +906,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
} else if (context.parseFieldMatcher().match(currentFieldName, SUGGEST_FIELD)) {
suggestBuilder = SuggestBuilder.fromXContent(context, suggesters);
} else if (context.parseFieldMatcher().match(currentFieldName, SORT_FIELD)) {
sorts = new ArrayList<>();
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
sorts.add(xContentBuilder.bytes());
sorts = new ArrayList<>(SortBuilder.fromXContent(context));
} else if (context.parseFieldMatcher().match(currentFieldName, EXT_FIELD)) {
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
ext = xContentBuilder.bytes();
@ -940,11 +937,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
}
} else if (context.parseFieldMatcher().match(currentFieldName, SORT_FIELD)) {
sorts = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
sorts.add(xContentBuilder.bytes());
}
sorts = new ArrayList<>(SortBuilder.fromXContent(context));
} else if (context.parseFieldMatcher().match(currentFieldName, RESCORE_FIELD)) {
rescoreBuilders = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
@ -1057,10 +1050,8 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
if (sorts != null) {
builder.startArray(SORT_FIELD.getPreferredName());
for (BytesReference sort : sorts) {
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(sort);
parser.nextToken();
builder.copyCurrentStructure(parser);
for (SortBuilder<?> sort : sorts) {
sort.toXContent(builder, params);
}
builder.endArray();
}
@ -1266,9 +1257,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
builder.size = in.readVInt();
if (in.readBoolean()) {
int size = in.readVInt();
List<BytesReference> sorts = new ArrayList<>();
List<SortBuilder<?>> sorts = new ArrayList<>();
for (int i = 0; i < size; i++) {
sorts.add(in.readBytesReference());
sorts.add(in.readSortBuilder());
}
builder.sorts = sorts;
}
@ -1382,8 +1373,8 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
out.writeBoolean(hasSorts);
if (hasSorts) {
out.writeVInt(sorts.size());
for (BytesReference sort : sorts) {
out.writeBytesReference(sort);
for (SortBuilder<?> sort : sorts) {
out.writeSortBuilder(sort);
}
}
boolean hasStats = stats != null;

View File

@ -35,7 +35,6 @@ import org.elasticsearch.search.highlight.HighlighterParseElement;
import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHits;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.SortParseElement;
import java.io.IOException;
import java.util.HashMap;
@ -51,8 +50,8 @@ public class InnerHitsFetchSubPhase implements FetchSubPhase {
private FetchPhase fetchPhase;
@Inject
public InnerHitsFetchSubPhase(SortParseElement sortParseElement, FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement, ScriptFieldsParseElement scriptFieldsParseElement) {
parseElements = singletonMap("inner_hits", new InnerHitsParseElement(sortParseElement, sourceParseElement, highlighterParseElement,
public InnerHitsFetchSubPhase(FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement, ScriptFieldsParseElement scriptFieldsParseElement) {
parseElements = singletonMap("inner_hits", new InnerHitsParseElement(sourceParseElement, highlighterParseElement,
fieldDataFieldsParseElement, scriptFieldsParseElement));
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.search.fetch.source.FetchSourceParseElement;
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.util.HashMap;
import java.util.Map;
@ -43,14 +42,12 @@ import static org.elasticsearch.index.query.support.InnerHitsQueryParserHelper.p
*/
public class InnerHitsParseElement implements SearchParseElement {
private final SortParseElement sortParseElement;
private final FetchSourceParseElement sourceParseElement;
private final HighlighterParseElement highlighterParseElement;
private final FieldDataFieldsParseElement fieldDataFieldsParseElement;
private final ScriptFieldsParseElement scriptFieldsParseElement;
public InnerHitsParseElement(SortParseElement sortParseElement, FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement, ScriptFieldsParseElement scriptFieldsParseElement) {
this.sortParseElement = sortParseElement;
public InnerHitsParseElement(FetchSourceParseElement sourceParseElement, HighlighterParseElement highlighterParseElement, FieldDataFieldsParseElement fieldDataFieldsParseElement, ScriptFieldsParseElement scriptFieldsParseElement) {
this.sourceParseElement = sourceParseElement;
this.highlighterParseElement = highlighterParseElement;
this.fieldDataFieldsParseElement = fieldDataFieldsParseElement;
@ -184,10 +181,10 @@ public class InnerHitsParseElement implements SearchParseElement {
} else if ("inner_hits".equals(fieldName)) {
childInnerHits = parseInnerHits(parser, context, searchContext);
} else {
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sortParseElement, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
}
} else {
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sortParseElement, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
parseCommonInnerHitOptions(parser, token, fieldName, subSearchContext, sourceParseElement, highlighterParseElement, scriptFieldsParseElement, fieldDataFieldsParseElement);
}
}

View File

@ -58,7 +58,6 @@ import org.elasticsearch.search.profile.ProfileShardResult;
import org.elasticsearch.search.profile.Profiler;
import org.elasticsearch.search.rescore.RescorePhase;
import org.elasticsearch.search.rescore.RescoreSearchContext;
import org.elasticsearch.search.sort.SortParseElement;
import org.elasticsearch.search.sort.TrackScoresParseElement;
import org.elasticsearch.search.suggest.SuggestPhase;
@ -98,7 +97,6 @@ public class QueryPhase implements SearchPhase {
parseElements.put("query", new QueryParseElement());
parseElements.put("post_filter", new PostFilterParseElement());
parseElements.put("postFilter", new PostFilterParseElement());
parseElements.put("sort", new SortParseElement());
parseElements.put("trackScores", new TrackScoresParseElement());
parseElements.put("track_scores", new TrackScoresParseElement());
parseElements.put("min_score", new MinScoreParseElement());

View File

@ -20,12 +20,10 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BytesRef;
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.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
@ -44,7 +42,7 @@ import java.util.Objects;
* A sort builder to sort based on a document field.
*/
public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
static final FieldSortBuilder PROTOTYPE = new FieldSortBuilder("");
public static final FieldSortBuilder PROTOTYPE = new FieldSortBuilder("_na_");
public static final String NAME = "field_sort";
public static final ParseField NESTED_PATH = new ParseField("nested_path");
public static final ParseField NESTED_FILTER = new ParseField("nested_filter");
@ -109,19 +107,12 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
* <tt>_first</tt> to sort missing last or first respectively.
*/
public FieldSortBuilder missing(Object missing) {
if (missing instanceof String) {
this.missing = BytesRefs.toBytesRef(missing);
} else {
this.missing = missing;
}
return this;
}
/** Returns the value used when a field is missing in a doc. */
public Object missing() {
if (missing instanceof BytesRef) {
return ((BytesRef) missing).utf8ToString();
}
return missing;
}
@ -208,15 +199,12 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.startObject(fieldName);
builder.field(ORDER_FIELD.getPreferredName(), order);
if (missing != null) {
if (missing instanceof BytesRef) {
builder.field(MISSING.getPreferredName(), ((BytesRef) missing).utf8ToString());
} else {
builder.field(MISSING.getPreferredName(), missing);
}
}
if (unmappedType != null) {
builder.field(UNMAPPED_TYPE.getPreferredName(), unmappedType);
}
@ -230,6 +218,7 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
builder.field(NESTED_PATH.getPreferredName(), nestedPath);
}
builder.endObject();
builder.endObject();
return builder;
}
@ -376,7 +365,7 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
if (context.parseFieldMatcher().match(currentFieldName, NESTED_PATH)) {
nestedPath = parser.text();
} else if (context.parseFieldMatcher().match(currentFieldName, MISSING)) {
missing = parser.objectBytes();
missing = parser.objectText();
} else if (context.parseFieldMatcher().match(currentFieldName, REVERSE)) {
if (parser.booleanValue()) {
order = SortOrder.DESC;

View File

@ -74,7 +74,7 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
public static final ParseField NESTED_PATH_FIELD = new ParseField("nested_path");
public static final ParseField NESTED_FILTER_FIELD = new ParseField("nested_filter");
static final GeoDistanceSortBuilder PROTOTYPE = new GeoDistanceSortBuilder("", -1, -1);
public static final GeoDistanceSortBuilder PROTOTYPE = new GeoDistanceSortBuilder("_na_", -1, -1);
private final String fieldName;
private final List<GeoPoint> points = new ArrayList<>();
@ -300,6 +300,7 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.startObject(NAME);
builder.startArray(fieldName);
@ -325,6 +326,7 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
builder.field(COERCE_FIELD.getPreferredName(), coerce);
builder.field(IGNORE_MALFORMED_FIELD.getPreferredName(), ignoreMalformed);
builder.endObject();
builder.endObject();
return builder;
}

View File

@ -1,220 +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.sort;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoDistance.FixedSourceDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
*
*/
public class GeoDistanceSortParser implements SortParser {
@Override
public String[] names() {
return new String[]{"_geo_distance", "_geoDistance"};
}
@Override
public SortField parse(XContentParser parser, QueryShardContext context) throws IOException {
String fieldName = null;
List<GeoPoint> geoPoints = new ArrayList<>();
DistanceUnit unit = DistanceUnit.DEFAULT;
GeoDistance geoDistance = GeoDistance.DEFAULT;
boolean reverse = false;
MultiValueMode sortMode = null;
NestedInnerQueryParseSupport nestedHelper = null;
final boolean indexCreatedBeforeV2_0 = context.indexVersionCreated().before(Version.V_2_0_0);
boolean coerce = GeoDistanceSortBuilder.DEFAULT_COERCE;
boolean ignoreMalformed = GeoDistanceSortBuilder.DEFAULT_IGNORE_MALFORMED;
XContentParser.Token token;
String currentName = parser.currentName();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentName = parser.currentName();
} else if (token == XContentParser.Token.START_ARRAY) {
GeoDistanceSortBuilder.parseGeoPoints(parser, geoPoints);
fieldName = currentName;
} else if (token == XContentParser.Token.START_OBJECT) {
// the json in the format of -> field : { lat : 30, lon : 12 }
if ("nested_filter".equals(currentName) || "nestedFilter".equals(currentName)) {
if (nestedHelper == null) {
nestedHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedHelper.filter();
} else {
fieldName = currentName;
GeoPoint point = new GeoPoint();
GeoUtils.parseGeoPoint(parser, point);
geoPoints.add(point);
}
} else if (token.isValue()) {
if ("reverse".equals(currentName)) {
reverse = parser.booleanValue();
} else if ("order".equals(currentName)) {
reverse = "desc".equals(parser.text());
} else if (currentName.equals("unit")) {
unit = DistanceUnit.fromString(parser.text());
} else if (currentName.equals("distance_type") || currentName.equals("distanceType")) {
geoDistance = GeoDistance.fromString(parser.text());
} else if ("coerce".equals(currentName) || (indexCreatedBeforeV2_0 && "normalize".equals(currentName))) {
coerce = parser.booleanValue();
if (coerce == true) {
ignoreMalformed = true;
}
} else if ("ignore_malformed".equals(currentName)) {
boolean ignoreMalformedFlag = parser.booleanValue();
if (coerce == false) {
ignoreMalformed = ignoreMalformedFlag;
}
} else if ("sort_mode".equals(currentName) || "sortMode".equals(currentName) || "mode".equals(currentName)) {
sortMode = MultiValueMode.fromString(parser.text());
} else if ("nested_path".equals(currentName) || "nestedPath".equals(currentName)) {
if (nestedHelper == null) {
nestedHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedHelper.setPath(parser.text());
} else {
GeoPoint point = new GeoPoint();
point.resetFromString(parser.text());
geoPoints.add(point);
fieldName = currentName;
}
}
}
// validation was not available prior to 2.x, so to support bwc percolation queries we only ignore_malformed on 2.x created indexes
if (!indexCreatedBeforeV2_0 && !ignoreMalformed) {
for (GeoPoint point : geoPoints) {
if (point.lat() > 90.0 || point.lat() < -90.0) {
throw new ElasticsearchParseException("illegal latitude value [{}] for [GeoDistanceSort]", point.lat());
}
if (point.lon() > 180.0 || point.lon() < -180) {
throw new ElasticsearchParseException("illegal longitude value [{}] for [GeoDistanceSort]", point.lon());
}
}
}
if (coerce) {
for (GeoPoint point : geoPoints) {
GeoUtils.normalizePoint(point, coerce, coerce);
}
}
if (sortMode == null) {
sortMode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN;
}
if (sortMode == MultiValueMode.SUM) {
throw new IllegalArgumentException("sort_mode [sum] isn't supported for sorting by geo distance");
}
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType == null) {
throw new IllegalArgumentException("failed to find mapper for [" + fieldName + "] for geo distance based sort");
}
final MultiValueMode finalSortMode = sortMode; // final reference for use in the anonymous class
final IndexGeoPointFieldData geoIndexFieldData = context.getForField(fieldType);
final FixedSourceDistance[] distances = new FixedSourceDistance[geoPoints.size()];
for (int i = 0; i< geoPoints.size(); i++) {
distances[i] = geoDistance.fixedSourceDistance(geoPoints.get(i).lat(), geoPoints.get(i).lon(), unit);
}
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilter(Queries.newNonNestedFilter());
Query innerDocumentsQuery;
if (nestedHelper.filterFound()) {
// TODO: use queries instead
innerDocumentsQuery = nestedHelper.getInnerFilter();
} else {
innerDocumentsQuery = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, innerDocumentsQuery);
} else {
nested = null;
}
IndexFieldData.XFieldComparatorSource geoDistanceComparatorSource = new IndexFieldData.XFieldComparatorSource() {
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new FieldComparator.DoubleComparator(numHits, null, null) {
@Override
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
final MultiGeoPointValues geoPointValues = geoIndexFieldData.load(context).getGeoPointValues();
final SortedNumericDoubleValues distanceValues = GeoDistance.distanceValues(geoPointValues, distances);
final NumericDoubleValues selectedValues;
if (nested == null) {
selectedValues = finalSortMode.select(distanceValues, Double.MAX_VALUE);
} else {
final BitSet rootDocs = nested.rootDocs(context);
final DocIdSetIterator innerDocs = nested.innerDocs(context);
selectedValues = finalSortMode.select(distanceValues, Double.MAX_VALUE, rootDocs, innerDocs, context.reader().maxDoc());
}
return selectedValues.getRawDoubleValues();
}
};
}
};
return new SortField(fieldName, geoDistanceComparatorSource, reverse);
}
}

View File

@ -39,7 +39,7 @@ import java.util.Objects;
public class ScoreSortBuilder extends SortBuilder<ScoreSortBuilder> {
public static final String NAME = "_score";
static final ScoreSortBuilder PROTOTYPE = new ScoreSortBuilder();
public static final ScoreSortBuilder PROTOTYPE = new ScoreSortBuilder();
public static final ParseField REVERSE_FIELD = new ParseField("reverse");
public static final ParseField ORDER_FIELD = new ParseField("order");
private static final SortField SORT_SCORE = new SortField(null, SortField.Type.SCORE);
@ -53,9 +53,11 @@ public class ScoreSortBuilder extends SortBuilder<ScoreSortBuilder> {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.startObject(NAME);
builder.field(ORDER_FIELD.getPreferredName(), order);
builder.endObject();
builder.endObject();
return builder;
}

View File

@ -67,7 +67,7 @@ import java.util.Objects;
public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
public static final String NAME = "_script";
static final ScriptSortBuilder PROTOTYPE = new ScriptSortBuilder(new Script("_na_"), ScriptSortType.STRING);
public static final ScriptSortBuilder PROTOTYPE = new ScriptSortBuilder(new Script("_na_"), ScriptSortType.STRING);
public static final ParseField TYPE_FIELD = new ParseField("type");
public static final ParseField SCRIPT_FIELD = new ParseField("script");
public static final ParseField SORTMODE_FIELD = new ParseField("mode");
@ -179,6 +179,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) throws IOException {
builder.startObject();
builder.startObject(NAME);
builder.field(SCRIPT_FIELD.getPreferredName(), script);
builder.field(TYPE_FIELD.getPreferredName(), type);
@ -193,6 +194,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
builder.field(NESTED_FILTER_FIELD.getPreferredName(), nestedFilter, builderParams);
}
builder.endObject();
builder.endObject();
return builder;
}

View File

@ -1,210 +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.sort;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptParameterParser;
import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
/**
*
*/
public class ScriptSortParser implements SortParser {
@Override
public String[] names() {
return new String[]{"_script"};
}
@Override
public SortField parse(XContentParser parser, QueryShardContext context) throws IOException {
ScriptParameterParser scriptParameterParser = new ScriptParameterParser();
Script script = null;
ScriptSortType type = null;
Map<String, Object> params = null;
boolean reverse = false;
MultiValueMode sortMode = null;
NestedInnerQueryParseSupport nestedHelper = null;
XContentParser.Token token;
String currentName = parser.currentName();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentName = parser.currentName();
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.parseFieldMatcher().match(currentName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.parseFieldMatcher());
} else if ("params".equals(currentName)) {
params = parser.map();
} else if ("nested_filter".equals(currentName) || "nestedFilter".equals(currentName)) {
if (nestedHelper == null) {
nestedHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedHelper.filter();
}
} else if (token.isValue()) {
if ("reverse".equals(currentName)) {
reverse = parser.booleanValue();
} else if ("order".equals(currentName)) {
reverse = "desc".equals(parser.text());
} else if (scriptParameterParser.token(currentName, token, parser, context.parseFieldMatcher())) {
// Do Nothing (handled by ScriptParameterParser
} else if ("type".equals(currentName)) {
type = ScriptSortType.fromString(parser.text());
} else if ("mode".equals(currentName)) {
sortMode = MultiValueMode.fromString(parser.text());
} else if ("nested_path".equals(currentName) || "nestedPath".equals(currentName)) {
if (nestedHelper == null) {
nestedHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedHelper.setPath(parser.text());
}
}
}
if (script == null) { // Didn't find anything using the new API so try using the old one instead
ScriptParameterValue scriptValue = scriptParameterParser.getDefaultScriptParameterValue();
if (scriptValue != null) {
if (params == null) {
params = new HashMap<>();
}
script = new Script(scriptValue.script(), scriptValue.scriptType(), scriptParameterParser.lang(), params);
}
} else if (params != null) {
throw new ParsingException(parser.getTokenLocation(), "script params must be specified inside script object");
}
if (script == null) {
throw new ParsingException(parser.getTokenLocation(), "_script sorting requires setting the script to sort by");
}
if (type == null) {
throw new ParsingException(parser.getTokenLocation(), "_script sorting requires setting the type of the script");
}
final SearchScript searchScript = context.getScriptService().search(
context.lookup(), script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
if (ScriptSortType.STRING.equals(type) && (sortMode == MultiValueMode.SUM || sortMode == MultiValueMode.AVG)) {
throw new ParsingException(parser.getTokenLocation(), "type [string] doesn't support mode [" + sortMode + "]");
}
if (sortMode == null) {
sortMode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN;
}
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilter(Queries.newNonNestedFilter());
Query innerDocumentsFilter;
if (nestedHelper.filterFound()) {
// TODO: use queries instead
innerDocumentsFilter = nestedHelper.getInnerFilter();
} else {
innerDocumentsFilter = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, innerDocumentsFilter);
} else {
nested = null;
}
final IndexFieldData.XFieldComparatorSource fieldComparatorSource;
switch (type) {
case STRING:
fieldComparatorSource = new BytesRefFieldComparatorSource(null, null, sortMode, nested) {
LeafSearchScript leafScript;
@Override
protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOException {
leafScript = searchScript.getLeafSearchScript(context);
final BinaryDocValues values = new BinaryDocValues() {
final BytesRefBuilder spare = new BytesRefBuilder();
@Override
public BytesRef get(int docID) {
leafScript.setDocument(docID);
spare.copyChars(leafScript.run().toString());
return spare.get();
}
};
return FieldData.singleton(values, null);
}
@Override
protected void setScorer(Scorer scorer) {
leafScript.setScorer(scorer);
}
};
break;
case NUMBER:
fieldComparatorSource = new DoubleValuesComparatorSource(null, Double.MAX_VALUE, sortMode, nested) {
LeafSearchScript leafScript;
@Override
protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws IOException {
leafScript = searchScript.getLeafSearchScript(context);
final NumericDoubleValues values = new NumericDoubleValues() {
@Override
public double get(int docID) {
leafScript.setDocument(docID);
return leafScript.runAsDouble();
}
};
return FieldData.singleton(values, null);
}
@Override
protected void setScorer(Scorer scorer) {
leafScript.setScorer(scorer);
}
};
break;
default:
throw new ParsingException(parser.getTokenLocation(), "custom script sort type [" + type + "] not supported");
}
return new SortField("_script", fieldComparatorSource, reverse);
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.join.BitSetProducer;
import org.elasticsearch.action.support.ToXContentToBytes;
@ -34,6 +35,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -41,6 +43,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import static java.util.Collections.unmodifiableMap;
@ -157,6 +160,41 @@ public abstract class SortBuilder<T extends SortBuilder<?>> extends ToXContentTo
}
}
public static void parseSort(XContentParser parser, SearchContext context) throws IOException {
QueryParseContext parseContext = context.getQueryShardContext().parseContext();
parseContext.reset(parser);
Optional<Sort> sortOptional = buildSort(SortBuilder.fromXContent(parseContext), context.getQueryShardContext());
if (sortOptional.isPresent()) {
context.sort(sortOptional.get());
}
}
public static Optional<Sort> buildSort(List<SortBuilder<?>> sortBuilders, QueryShardContext context) throws IOException {
List<SortField> sortFields = new ArrayList<>(sortBuilders.size());
for (SortBuilder<?> builder : sortBuilders) {
sortFields.add(builder.build(context));
}
if (!sortFields.isEmpty()) {
// optimize if we just sort on score non reversed, we don't really
// need sorting
boolean sort;
if (sortFields.size() > 1) {
sort = true;
} else {
SortField sortField = sortFields.get(0);
if (sortField.getType() == SortField.Type.SCORE && !sortField.getReverse()) {
sort = false;
} else {
sort = true;
}
}
if (sort) {
return Optional.of(new Sort(sortFields.toArray(new SortField[sortFields.size()])));
}
}
return Optional.empty();
}
protected static Nested resolveNested(QueryShardContext context, String nestedPath, QueryBuilder<?> nestedFilter) throws IOException {
Nested nested = null;
if (nestedPath != null) {

View File

@ -1,252 +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.sort;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.join.BitSetProducer;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static java.util.Collections.unmodifiableMap;
/**
*
*/
public class SortParseElement implements SearchParseElement {
private static final SortField SORT_SCORE = new SortField(null, SortField.Type.SCORE);
private static final SortField SORT_SCORE_REVERSE = new SortField(null, SortField.Type.SCORE, true);
private static final SortField SORT_DOC = new SortField(null, SortField.Type.DOC);
private static final SortField SORT_DOC_REVERSE = new SortField(null, SortField.Type.DOC, true);
public static final ParseField UNMAPPED_TYPE = new ParseField("unmapped_type");
public static final String SCORE_FIELD_NAME = "_score";
public static final String DOC_FIELD_NAME = "_doc";
private static final Map<String, SortParser> PARSERS;
static {
Map<String, SortParser> parsers = new HashMap<>();
addParser(parsers, new ScriptSortParser());
addParser(parsers, new GeoDistanceSortParser());
PARSERS = unmodifiableMap(parsers);
}
private static void addParser(Map<String, SortParser> parsers, SortParser parser) {
for (String name : parser.names()) {
parsers.put(name, parser);
}
}
@Override
public void parse(XContentParser parser, SearchContext context) throws IOException {
List<SortField> sortFields = parse(parser, context.getQueryShardContext());
if (!sortFields.isEmpty()) {
// optimize if we just sort on score non reversed, we don't really need sorting
boolean sort;
if (sortFields.size() > 1) {
sort = true;
} else {
SortField sortField = sortFields.get(0);
if (sortField.getType() == SortField.Type.SCORE && !sortField.getReverse()) {
sort = false;
} else {
sort = true;
}
}
if (sort) {
context.sort(new Sort(sortFields.toArray(new SortField[sortFields.size()])));
}
}
}
List<SortField> parse(XContentParser parser, QueryShardContext context) throws IOException {
XContentParser.Token token = parser.currentToken();
List<SortField> sortFields = new ArrayList<>(2);
if (token == XContentParser.Token.START_ARRAY) {
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
addCompoundSortField(parser, context, sortFields);
} else if (token == XContentParser.Token.VALUE_STRING) {
addSortField(context, sortFields, parser.text(), false, null, null, null, null);
} else {
throw new IllegalArgumentException("malformed sort format, within the sort array, an object, or an actual string are allowed");
}
}
} else if (token == XContentParser.Token.VALUE_STRING) {
addSortField(context, sortFields, parser.text(), false, null, null, null, null);
} else if (token == XContentParser.Token.START_OBJECT) {
addCompoundSortField(parser, context, sortFields);
} else {
throw new IllegalArgumentException("malformed sort format, either start with array, object, or an actual string");
}
return sortFields;
}
private void addCompoundSortField(XContentParser parser, QueryShardContext context, List<SortField> sortFields) throws IOException {
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
String fieldName = parser.currentName();
boolean reverse = false;
String missing = null;
String innerJsonName = null;
String unmappedType = null;
MultiValueMode sortMode = null;
NestedInnerQueryParseSupport nestedFilterParseHelper = null;
token = parser.nextToken();
if (token == XContentParser.Token.VALUE_STRING) {
String direction = parser.text();
if (direction.equals("asc")) {
reverse = SCORE_FIELD_NAME.equals(fieldName);
} else if (direction.equals("desc")) {
reverse = !SCORE_FIELD_NAME.equals(fieldName);
} else {
throw new IllegalArgumentException("sort direction [" + fieldName + "] not supported");
}
addSortField(context, sortFields, fieldName, reverse, unmappedType, missing, sortMode, nestedFilterParseHelper);
} else {
if (PARSERS.containsKey(fieldName)) {
sortFields.add(PARSERS.get(fieldName).parse(parser, context));
} else {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
innerJsonName = parser.currentName();
} else if (token.isValue()) {
if ("reverse".equals(innerJsonName)) {
reverse = parser.booleanValue();
} else if ("order".equals(innerJsonName)) {
if ("asc".equals(parser.text())) {
reverse = SCORE_FIELD_NAME.equals(fieldName);
} else if ("desc".equals(parser.text())) {
reverse = !SCORE_FIELD_NAME.equals(fieldName);
}
} else if ("missing".equals(innerJsonName)) {
missing = parser.textOrNull();
} else if (context.parseFieldMatcher().match(innerJsonName, UNMAPPED_TYPE)) {
unmappedType = parser.textOrNull();
} else if ("mode".equals(innerJsonName)) {
sortMode = MultiValueMode.fromString(parser.text());
} else if ("nested_path".equals(innerJsonName) || "nestedPath".equals(innerJsonName)) {
if (nestedFilterParseHelper == null) {
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedFilterParseHelper.setPath(parser.text());
} else {
throw new IllegalArgumentException("sort option [" + innerJsonName + "] not supported");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if ("nested_filter".equals(innerJsonName) || "nestedFilter".equals(innerJsonName)) {
if (nestedFilterParseHelper == null) {
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context);
}
nestedFilterParseHelper.filter();
} else {
throw new IllegalArgumentException("sort option [" + innerJsonName + "] not supported");
}
}
}
addSortField(context, sortFields, fieldName, reverse, unmappedType, missing, sortMode, nestedFilterParseHelper);
}
}
}
}
}
private void addSortField(QueryShardContext context, List<SortField> sortFields, String fieldName, boolean reverse, String unmappedType, @Nullable final String missing, MultiValueMode sortMode, NestedInnerQueryParseSupport nestedHelper) throws IOException {
if (SCORE_FIELD_NAME.equals(fieldName)) {
if (reverse) {
sortFields.add(SORT_SCORE_REVERSE);
} else {
sortFields.add(SORT_SCORE);
}
} else if (DOC_FIELD_NAME.equals(fieldName)) {
if (reverse) {
sortFields.add(SORT_DOC_REVERSE);
} else {
sortFields.add(SORT_DOC);
}
} else {
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType == null) {
if (unmappedType != null) {
fieldType = context.getMapperService().unmappedFieldType(unmappedType);
} else {
throw new QueryShardException(context, "No mapping found for [" + fieldName + "] in order to sort on");
}
}
if (!fieldType.isSortable()) {
throw new QueryShardException(context, "Sorting not supported for field[" + fieldName + "]");
}
// We only support AVG and SUM on number based fields
if (fieldType.isNumeric() == false && (sortMode == MultiValueMode.SUM || sortMode == MultiValueMode.AVG)) {
sortMode = null;
}
if (sortMode == null) {
sortMode = resolveDefaultSortMode(reverse);
}
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilter(Queries.newNonNestedFilter());
Query innerDocumentsQuery;
if (nestedHelper.filterFound()) {
innerDocumentsQuery = nestedHelper.getInnerFilter();
} else {
innerDocumentsQuery = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, innerDocumentsQuery);
} else {
nested = null;
}
IndexFieldData.XFieldComparatorSource fieldComparatorSource = context.getForField(fieldType)
.comparatorSource(missing, sortMode, nested);
sortFields.add(new SortField(fieldType.name(), fieldComparatorSource, reverse));
}
}
private static MultiValueMode resolveDefaultSortMode(boolean reverse) {
return reverse ? MultiValueMode.MAX : MultiValueMode.MIN;
}
}

View File

@ -1,36 +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.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
/**
*
*/
public interface SortParser {
String[] names();
SortField parse(XContentParser parser, QueryShardContext context) throws IOException;
}

View File

@ -74,6 +74,8 @@ import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.rescore.QueryRescoreBuilderTests;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
@ -551,7 +553,7 @@ public class SearchSourceBuilderTests extends ESTestCase {
SearchSourceBuilder searchSourceBuilder = SearchSourceBuilder.parseSearchSource(parser, createParseContext(parser),
aggParsers, suggesters);
assertEquals(1, searchSourceBuilder.sorts().size());
assertEquals("{\"foo\":{\"order\":\"asc\"}}", searchSourceBuilder.sorts().get(0).toUtf8());
assertEquals(new FieldSortBuilder("foo"), searchSourceBuilder.sorts().get(0));
}
}
@ -567,11 +569,11 @@ public class SearchSourceBuilderTests extends ESTestCase {
SearchSourceBuilder searchSourceBuilder = SearchSourceBuilder.parseSearchSource(parser, createParseContext(parser),
aggParsers, suggesters);
assertEquals(5, searchSourceBuilder.sorts().size());
assertEquals("{\"post_date\":{\"order\":\"asc\"}}", searchSourceBuilder.sorts().get(0).toUtf8());
assertEquals("\"user\"", searchSourceBuilder.sorts().get(1).toUtf8());
assertEquals("{\"name\":\"desc\"}", searchSourceBuilder.sorts().get(2).toUtf8());
assertEquals("{\"age\":\"desc\"}", searchSourceBuilder.sorts().get(3).toUtf8());
assertEquals("\"_score\"", searchSourceBuilder.sorts().get(4).toUtf8());
assertEquals(new FieldSortBuilder("post_date"), searchSourceBuilder.sorts().get(0));
assertEquals(new FieldSortBuilder("user"), searchSourceBuilder.sorts().get(1));
assertEquals(new FieldSortBuilder("name").order(SortOrder.DESC), searchSourceBuilder.sorts().get(2));
assertEquals(new FieldSortBuilder("age").order(SortOrder.DESC), searchSourceBuilder.sorts().get(3));
assertEquals(new ScoreSortBuilder(), searchSourceBuilder.sorts().get(4));
}
}
}

View File

@ -66,7 +66,6 @@ import org.junit.BeforeClass;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
@ -78,7 +77,6 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
private static final int NUMBER_OF_TESTBUILDERS = 20;
static IndicesQueriesRegistry indicesQueriesRegistry;
private static SortParseElement parseElement = new SortParseElement();
private static ScriptService scriptService;
@BeforeClass
@ -131,10 +129,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
if (randomBoolean()) {
builder.prettyPrint();
}
builder.startObject();
testItem.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
XContentParser itemParser = XContentHelper.createParser(builder.bytes());
itemParser.nextToken();
@ -163,24 +158,12 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
for (int runs = 0; runs < NUMBER_OF_TESTBUILDERS; runs++) {
T sortBuilder = createTestItem();
SortField sortField = sortBuilder.build(mockShardContext);
XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values()));
if (randomBoolean()) {
builder.prettyPrint();
}
builder.startObject();
sortBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
XContentParser parser = XContentHelper.createParser(builder.bytes());
parser.nextToken();
List<SortField> sortFields = parseElement.parse(parser, mockShardContext);
assertEquals(1, sortFields.size());
SortField sortFieldOldStyle = sortFields.get(0);
assertEquals(sortFieldOldStyle.getField(), sortField.getField());
assertEquals(sortFieldOldStyle.getReverse(), sortField.getReverse());
assertEquals(sortFieldOldStyle.getType(), sortField.getType());
sortFieldAssertions(sortBuilder, sortField);
}
}
protected abstract void sortFieldAssertions(T builder, SortField sortField) throws IOException;
/**
* Test serialization and deserialization of the test sort.
*/

View File

@ -19,6 +19,8 @@ x * Licensed to Elasticsearch under one or more contributor
package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import java.io.IOException;
public class FieldSortBuilderTests extends AbstractSortTestCase<FieldSortBuilder> {
@ -29,7 +31,7 @@ public class FieldSortBuilderTests extends AbstractSortTestCase<FieldSortBuilder
}
public static FieldSortBuilder randomFieldSortBuilder() {
String fieldName = rarely() ? SortParseElement.DOC_FIELD_NAME : randomAsciiOfLengthBetween(1, 10);
String fieldName = rarely() ? FieldSortBuilder.DOC_FIELD_NAME : randomAsciiOfLengthBetween(1, 10);
FieldSortBuilder builder = new FieldSortBuilder(fieldName);
if (randomBoolean()) {
builder.order(RandomSortDataGenerator.order(null));
@ -86,4 +88,19 @@ public class FieldSortBuilderTests extends AbstractSortTestCase<FieldSortBuilder
}
return mutated;
}
@Override
protected void sortFieldAssertions(FieldSortBuilder builder, SortField sortField) throws IOException {
SortField.Type expectedType;
if (builder.getFieldName().equals(FieldSortBuilder.DOC_FIELD_NAME)) {
expectedType = SortField.Type.DOC;
} else {
expectedType = SortField.Type.CUSTOM;
}
assertEquals(expectedType, sortField.getType());
assertEquals(builder.order() == SortOrder.ASC ? false : true, sortField.getReverse());
if (expectedType == SortField.Type.CUSTOM) {
assertEquals(builder.getFieldName(), sortField.getField());
}
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
@ -182,6 +183,13 @@ public class GeoDistanceSortBuilderTests extends AbstractSortTestCase<GeoDistanc
return result;
}
@Override
protected void sortFieldAssertions(GeoDistanceSortBuilder builder, SortField sortField) throws IOException {
assertEquals(SortField.Type.CUSTOM, sortField.getType());
assertEquals(builder.order() == SortOrder.ASC ? false : true, sortField.getReverse());
assertEquals(builder.fieldName(), sortField.getField());
}
public void testSortModeSumIsRejectedInSetter() {
GeoDistanceSortBuilder builder = new GeoDistanceSortBuilder("testname", -1, -1);
GeoPoint point = RandomGeoGenerator.randomPoint(getRandom());

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -79,4 +80,10 @@ public class ScoreSortBuilderTests extends AbstractSortTestCase<ScoreSortBuilder
ScoreSortBuilder scoreSort = ScoreSortBuilder.PROTOTYPE.fromXContent(context, "_score");
assertEquals(order, scoreSort.order());
}
@Override
protected void sortFieldAssertions(ScoreSortBuilder builder, SortField sortField) {
assertEquals(SortField.Type.SCORE, sortField.getType());
assertEquals(builder.order() == SortOrder.DESC ? false : true, sortField.getReverse());
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.settings.Settings;
@ -121,8 +122,11 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
return result;
}
@Rule
public ExpectedException exceptionRule = ExpectedException.none();
@Override
protected void sortFieldAssertions(ScriptSortBuilder builder, SortField sortField) throws IOException {
assertEquals(SortField.Type.CUSTOM, sortField.getType());
assertEquals(builder.order() == SortOrder.ASC ? false : true, sortField.getReverse());
}
public void testScriptSortType() {
// we rely on these ordinals in serialization, so changing them breaks bwc.
@ -140,6 +144,9 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
assertEquals(ScriptSortType.NUMBER, ScriptSortType.fromString("NUMBER"));
}
@Rule
public ExpectedException exceptionRule = ExpectedException.none();
public void testScriptSortTypeNull() {
exceptionRule.expect(NullPointerException.class);
exceptionRule.expectMessage("input string is null");

View File

@ -48,8 +48,6 @@ public class SortBuilderTests extends ESTestCase {
static IndicesQueriesRegistry indicesQueriesRegistry;
SortParseElement parseElement = new SortParseElement();
@BeforeClass
public static void init() {
namedWriteableRegistry = new NamedWriteableRegistry();
@ -84,6 +82,13 @@ public class SortBuilderTests extends ESTestCase {
sortBuilder = result.get(0);
assertEquals(new FieldSortBuilder("field1"), sortBuilder);
// one element array, see https://github.com/elastic/elasticsearch/issues/17257
json = "{ \"sort\" : [\"field1\"] }";
result = parseSort(json);
assertEquals(1, result.size());
sortBuilder = result.get(0);
assertEquals(new FieldSortBuilder("field1"), sortBuilder);
json = "{ \"sort\" : { \"_doc\" : \"" + order + "\" }}";
result = parseSort(json);
assertEquals(1, result.size());
@ -162,15 +167,11 @@ public class SortBuilderTests extends ESTestCase {
xContentBuilder.endObject();
break;
case 2:
xContentBuilder.startObject();
builder.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
xContentBuilder.endObject();
break;
}
} else {
xContentBuilder.startObject();
builder.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
xContentBuilder.endObject();
}
}
if (testBuilders.size() > 1) {
@ -195,7 +196,7 @@ public class SortBuilderTests extends ESTestCase {
list.add(new ScoreSortBuilder());
break;
case 1:
String fieldName = rarely() ? SortParseElement.DOC_FIELD_NAME : randomAsciiOfLengthBetween(1, 10);
String fieldName = rarely() ? FieldSortBuilder.DOC_FIELD_NAME : randomAsciiOfLengthBetween(1, 10);
list.add(new FieldSortBuilder(fieldName));
break;
case 2:

View File

@ -19,12 +19,14 @@
package org.elasticsearch.search.sort;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.TestSearchContext;
@ -47,10 +49,7 @@ public class SortParserTests extends ESSingleNodeTestCase {
sortBuilder.field("unit", "km");
sortBuilder.field("sort_mode", "max");
sortBuilder.endObject();
XContentParser parser = XContentHelper.createParser(sortBuilder.bytes());
parser.nextToken();
GeoDistanceSortParser geoParser = new GeoDistanceSortParser();
geoParser.parse(parser, context.getQueryShardContext());
parse(context, sortBuilder);
sortBuilder = jsonBuilder();
sortBuilder.startObject();
@ -136,9 +135,11 @@ public class SortParserTests extends ESSingleNodeTestCase {
}
protected void parse(TestSearchContext context, XContentBuilder sortBuilder) throws Exception {
QueryParseContext parseContext = context.getQueryShardContext().parseContext();
XContentParser parser = XContentHelper.createParser(sortBuilder.bytes());
parser.setParseFieldMatcher(ParseFieldMatcher.STRICT);
parseContext.reset(parser);
parser.nextToken();
GeoDistanceSortParser geoParser = new GeoDistanceSortParser();
geoParser.parse(parser, context.getQueryShardContext());
GeoDistanceSortBuilder.PROTOTYPE.fromXContent(parseContext, null);
}
}