Percolator refactoring:
* Added percolator field mapper that extracts the query terms and indexes these terms with the percolator query. * At percolate time these extracted terms are used to query percolator queries that are like to be evaluated. This can significantly cut down the time it takes to percolate. Whereas before all percolator queries were evaluated if they matches with the document being percolated. * Changes made to percolator queries are no longer immediately visible, a refresh needs to happen before the changes are visible. * By default the percolate api only returns upto 10 matches instead of returning all matching percolator queries. * Made percolate more modular, so that it is easier to add unit tests. * Added unit tests for the percolator. Closes #12664 Closes #13646
This commit is contained in:
parent
e0599bfb66
commit
2d6adf6428
|
@ -52,7 +52,7 @@ public class PercolateShardRequest extends BroadcastShardRequest {
|
|||
this.startTime = request.startTime;
|
||||
}
|
||||
|
||||
PercolateShardRequest(ShardId shardId, OriginalIndices originalIndices) {
|
||||
public PercolateShardRequest(ShardId shardId, OriginalIndices originalIndices) {
|
||||
super(shardId, originalIndices);
|
||||
}
|
||||
|
||||
|
@ -81,15 +81,15 @@ public class PercolateShardRequest extends BroadcastShardRequest {
|
|||
return onlyCount;
|
||||
}
|
||||
|
||||
void documentType(String documentType) {
|
||||
public void documentType(String documentType) {
|
||||
this.documentType = documentType;
|
||||
}
|
||||
|
||||
void source(BytesReference source) {
|
||||
public void source(BytesReference source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
void docSource(BytesReference docSource) {
|
||||
public void docSource(BytesReference docSource) {
|
||||
this.docSource = docSource;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,11 +18,12 @@
|
|||
*/
|
||||
package org.elasticsearch.action.percolate;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.percolator.PercolateContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
|
@ -43,31 +44,24 @@ import java.util.Map;
|
|||
*/
|
||||
public class PercolateShardResponse extends BroadcastShardResponse {
|
||||
|
||||
private static final BytesRef[] EMPTY_MATCHES = new BytesRef[0];
|
||||
private static final float[] EMPTY_SCORES = new float[0];
|
||||
private static final List<Map<String, HighlightField>> EMPTY_HL = Collections.emptyList();
|
||||
|
||||
private long count;
|
||||
private float[] scores;
|
||||
private BytesRef[] matches;
|
||||
private List<Map<String, HighlightField>> hls;
|
||||
private byte percolatorTypeId;
|
||||
private TopDocs topDocs;
|
||||
private Map<Integer, String> ids;
|
||||
private Map<Integer, Map<String, HighlightField>> hls;
|
||||
private boolean onlyCount;
|
||||
private int requestedSize;
|
||||
|
||||
private InternalAggregations aggregations;
|
||||
private List<SiblingPipelineAggregator> pipelineAggregators;
|
||||
|
||||
PercolateShardResponse() {
|
||||
hls = new ArrayList<>();
|
||||
}
|
||||
|
||||
public PercolateShardResponse(BytesRef[] matches, List<Map<String, HighlightField>> hls, long count, float[] scores, PercolateContext context, ShardId shardId) {
|
||||
super(shardId);
|
||||
this.matches = matches;
|
||||
public PercolateShardResponse(TopDocs topDocs, Map<Integer, String> ids, Map<Integer, Map<String, HighlightField>> hls, PercolateContext context) {
|
||||
super(new ShardId(context.shardTarget().getIndex(), context.shardTarget().getShardId()));
|
||||
this.topDocs = topDocs;
|
||||
this.ids = ids;
|
||||
this.hls = hls;
|
||||
this.count = count;
|
||||
this.scores = scores;
|
||||
this.percolatorTypeId = context.percolatorTypeId;
|
||||
this.onlyCount = context.isOnlyCount();
|
||||
this.requestedSize = context.size();
|
||||
QuerySearchResult result = context.queryResult();
|
||||
if (result != null) {
|
||||
|
@ -78,39 +72,25 @@ public class PercolateShardResponse extends BroadcastShardResponse {
|
|||
}
|
||||
}
|
||||
|
||||
public PercolateShardResponse(BytesRef[] matches, long count, float[] scores, PercolateContext context, ShardId shardId) {
|
||||
this(matches, EMPTY_HL, count, scores, context, shardId);
|
||||
public TopDocs topDocs() {
|
||||
return topDocs;
|
||||
}
|
||||
|
||||
public PercolateShardResponse(BytesRef[] matches, List<Map<String, HighlightField>> hls, long count, PercolateContext context, ShardId shardId) {
|
||||
this(matches, hls, count, EMPTY_SCORES, context, shardId);
|
||||
}
|
||||
|
||||
public PercolateShardResponse(long count, PercolateContext context, ShardId shardId) {
|
||||
this(EMPTY_MATCHES, EMPTY_HL, count, EMPTY_SCORES, context, shardId);
|
||||
}
|
||||
|
||||
public PercolateShardResponse(PercolateContext context, ShardId shardId) {
|
||||
this(EMPTY_MATCHES, EMPTY_HL, 0, EMPTY_SCORES, context, shardId);
|
||||
}
|
||||
|
||||
public BytesRef[] matches() {
|
||||
return matches;
|
||||
}
|
||||
|
||||
public float[] scores() {
|
||||
return scores;
|
||||
}
|
||||
|
||||
public long count() {
|
||||
return count;
|
||||
/**
|
||||
* Returns per match the percolator query id. The key is the Lucene docId of the matching percolator query.
|
||||
*/
|
||||
public Map<Integer, String> ids() {
|
||||
return ids;
|
||||
}
|
||||
|
||||
public int requestedSize() {
|
||||
return requestedSize;
|
||||
}
|
||||
|
||||
public List<Map<String, HighlightField>> hls() {
|
||||
/**
|
||||
* Returns per match the highlight snippets. The key is the Lucene docId of the matching percolator query.
|
||||
*/
|
||||
public Map<Integer, Map<String, HighlightField>> hls() {
|
||||
return hls;
|
||||
}
|
||||
|
||||
|
@ -122,36 +102,35 @@ public class PercolateShardResponse extends BroadcastShardResponse {
|
|||
return pipelineAggregators;
|
||||
}
|
||||
|
||||
public byte percolatorTypeId() {
|
||||
return percolatorTypeId;
|
||||
public boolean onlyCount() {
|
||||
return onlyCount;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return percolatorTypeId == 0x00;
|
||||
return topDocs.totalHits == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
percolatorTypeId = in.readByte();
|
||||
onlyCount = in.readBoolean();
|
||||
requestedSize = in.readVInt();
|
||||
count = in.readVLong();
|
||||
matches = new BytesRef[in.readVInt()];
|
||||
for (int i = 0; i < matches.length; i++) {
|
||||
matches[i] = in.readBytesRef();
|
||||
}
|
||||
scores = new float[in.readVInt()];
|
||||
for (int i = 0; i < scores.length; i++) {
|
||||
scores[i] = in.readFloat();
|
||||
}
|
||||
topDocs = Lucene.readTopDocs(in);
|
||||
int size = in.readVInt();
|
||||
ids = new HashMap<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
ids.put(in.readVInt(), in.readString());
|
||||
}
|
||||
size = in.readVInt();
|
||||
hls = new HashMap<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
int docId = in.readVInt();
|
||||
int mSize = in.readVInt();
|
||||
Map<String, HighlightField> fields = new HashMap<>();
|
||||
for (int j = 0; j < mSize; j++) {
|
||||
fields.put(in.readString(), HighlightField.readHighlightField(in));
|
||||
}
|
||||
hls.add(fields);
|
||||
hls.put(docId, fields);
|
||||
}
|
||||
aggregations = InternalAggregations.readOptionalAggregations(in);
|
||||
if (in.readBoolean()) {
|
||||
|
@ -169,23 +148,21 @@ public class PercolateShardResponse extends BroadcastShardResponse {
|
|||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeByte(percolatorTypeId);
|
||||
out.writeBoolean(onlyCount);
|
||||
out.writeVLong(requestedSize);
|
||||
out.writeVLong(count);
|
||||
out.writeVInt(matches.length);
|
||||
for (BytesRef match : matches) {
|
||||
out.writeBytesRef(match);
|
||||
}
|
||||
out.writeVLong(scores.length);
|
||||
for (float score : scores) {
|
||||
out.writeFloat(score);
|
||||
Lucene.writeTopDocs(out, topDocs);
|
||||
out.writeVInt(ids.size());
|
||||
for (Map.Entry<Integer, String> entry : ids.entrySet()) {
|
||||
out.writeVInt(entry.getKey());
|
||||
out.writeString(entry.getValue());
|
||||
}
|
||||
out.writeVInt(hls.size());
|
||||
for (Map<String, HighlightField> hl : hls) {
|
||||
out.writeVInt(hl.size());
|
||||
for (Map.Entry<String, HighlightField> entry : hl.entrySet()) {
|
||||
out.writeString(entry.getKey());
|
||||
entry.getValue().writeTo(out);
|
||||
for (Map.Entry<Integer, Map<String, HighlightField>> entry1 : hls.entrySet()) {
|
||||
out.writeVInt(entry1.getKey());
|
||||
out.writeVInt(entry1.getValue().size());
|
||||
for (Map.Entry<String, HighlightField> entry2 : entry1.getValue().entrySet()) {
|
||||
out.writeString(entry2.getKey());
|
||||
entry2.getValue().writeTo(out);
|
||||
}
|
||||
}
|
||||
out.writeOptionalStreamable(aggregations);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.elasticsearch.action.percolate;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
import org.elasticsearch.action.get.GetRequest;
|
||||
|
@ -43,6 +44,7 @@ import org.elasticsearch.percolator.PercolatorService;
|
|||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -117,7 +119,7 @@ public class TransportPercolateAction extends TransportBroadcastAction<Percolate
|
|||
List<PercolateShardResponse> shardResults = null;
|
||||
List<ShardOperationFailedException> shardFailures = null;
|
||||
|
||||
byte percolatorTypeId = 0x00;
|
||||
boolean onlyCount = false;
|
||||
for (int i = 0; i < shardsResponses.length(); i++) {
|
||||
Object shardResponse = shardsResponses.get(i);
|
||||
if (shardResponse == null) {
|
||||
|
@ -133,7 +135,7 @@ public class TransportPercolateAction extends TransportBroadcastAction<Percolate
|
|||
successfulShards++;
|
||||
if (!percolateShardResponse.isEmpty()) {
|
||||
if (shardResults == null) {
|
||||
percolatorTypeId = percolateShardResponse.percolatorTypeId();
|
||||
onlyCount = percolateShardResponse.onlyCount();
|
||||
shardResults = new ArrayList<>();
|
||||
}
|
||||
shardResults.add(percolateShardResponse);
|
||||
|
@ -146,7 +148,12 @@ public class TransportPercolateAction extends TransportBroadcastAction<Percolate
|
|||
PercolateResponse.Match[] matches = request.onlyCount() ? null : PercolateResponse.EMPTY;
|
||||
return new PercolateResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, tookInMillis, matches);
|
||||
} else {
|
||||
PercolatorService.ReduceResult result = percolatorService.reduce(percolatorTypeId, shardResults, request);
|
||||
PercolatorService.ReduceResult result = null;
|
||||
try {
|
||||
result = percolatorService.reduce(onlyCount, shardResults, request);
|
||||
} catch (IOException e) {
|
||||
throw new ElasticsearchException("error during reduce phase", e);
|
||||
}
|
||||
long tookInMillis = Math.max(1, System.currentTimeMillis() - request.startTime);
|
||||
return new PercolateResponse(
|
||||
shardsResponses.length(), successfulShards, failedShards, shardFailures,
|
||||
|
|
|
@ -221,7 +221,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
|
|||
SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap());
|
||||
|
||||
try (AnalysisService analysisService = new FakeAnalysisService(indexSettings)) {
|
||||
try (MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry)) {
|
||||
try (MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, () -> null)) {
|
||||
for (ObjectCursor<MappingMetaData> cursor : indexMetaData.getMappings().values()) {
|
||||
MappingMetaData mappingMetaData = cursor.value;
|
||||
mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), false, false);
|
||||
|
|
|
@ -117,7 +117,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
this.indexSettings = indexSettings;
|
||||
this.analysisService = registry.build(indexSettings);
|
||||
this.similarityService = similarityService;
|
||||
this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry);
|
||||
this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, IndexService.this::getQueryShardContext);
|
||||
this.indexFieldData = new IndexFieldDataService(indexSettings, nodeServicesProvider.getIndicesFieldDataCache(), nodeServicesProvider.getCircuitBreakerService(), mapperService);
|
||||
this.shardStoreDeleter = shardStoreDeleter;
|
||||
this.eventListener = eventListener;
|
||||
|
|
|
@ -33,12 +33,14 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.mapper.object.RootObjectMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.mapper.MapperRegistry;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static org.elasticsearch.index.mapper.MapperBuilders.doc;
|
||||
|
@ -49,6 +51,7 @@ public class DocumentMapperParser {
|
|||
final AnalysisService analysisService;
|
||||
private static final ESLogger logger = Loggers.getLogger(DocumentMapperParser.class);
|
||||
private final SimilarityService similarityService;
|
||||
private final Supplier<QueryShardContext> queryShardContextSupplier;
|
||||
|
||||
private final RootObjectMapper.TypeParser rootObjectTypeParser = new RootObjectMapper.TypeParser();
|
||||
|
||||
|
@ -59,18 +62,20 @@ public class DocumentMapperParser {
|
|||
private final Map<String, MetadataFieldMapper.TypeParser> rootTypeParsers;
|
||||
|
||||
public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, AnalysisService analysisService,
|
||||
SimilarityService similarityService, MapperRegistry mapperRegistry) {
|
||||
SimilarityService similarityService, MapperRegistry mapperRegistry,
|
||||
Supplier<QueryShardContext> queryShardContextSupplier) {
|
||||
this.parseFieldMatcher = new ParseFieldMatcher(indexSettings.getSettings());
|
||||
this.mapperService = mapperService;
|
||||
this.analysisService = analysisService;
|
||||
this.similarityService = similarityService;
|
||||
this.queryShardContextSupplier = queryShardContextSupplier;
|
||||
this.typeParsers = mapperRegistry.getMapperParsers();
|
||||
this.rootTypeParsers = mapperRegistry.getMetadataMapperParsers();
|
||||
indexVersionCreated = indexSettings.getIndexVersionCreated();
|
||||
}
|
||||
|
||||
public Mapper.TypeParser.ParserContext parserContext(String type) {
|
||||
return new Mapper.TypeParser.ParserContext(type, analysisService, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher);
|
||||
return new Mapper.TypeParser.ParserContext(type, analysisService, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher, queryShardContextSupplier.get());
|
||||
}
|
||||
|
||||
public DocumentMapper parse(@Nullable String type, CompressedXContent source) throws MapperParsingException {
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.similarity.SimilarityProvider;
|
||||
|
||||
import java.util.Map;
|
||||
|
@ -95,9 +97,11 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
|||
|
||||
private final ParseFieldMatcher parseFieldMatcher;
|
||||
|
||||
public ParserContext(String type, AnalysisService analysisService, Function<String, SimilarityProvider> similarityLookupService,
|
||||
private final QueryShardContext queryShardContext;
|
||||
|
||||
public ParserContext(String type, AnalysisService analysisService, Function<String, SimilarityProvider> similarityLookupService,
|
||||
MapperService mapperService, Function<String, TypeParser> typeParsers,
|
||||
Version indexVersionCreated, ParseFieldMatcher parseFieldMatcher) {
|
||||
Version indexVersionCreated, ParseFieldMatcher parseFieldMatcher, QueryShardContext queryShardContext) {
|
||||
this.type = type;
|
||||
this.analysisService = analysisService;
|
||||
this.similarityLookupService = similarityLookupService;
|
||||
|
@ -105,6 +109,7 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
|||
this.typeParsers = typeParsers;
|
||||
this.indexVersionCreated = indexVersionCreated;
|
||||
this.parseFieldMatcher = parseFieldMatcher;
|
||||
this.queryShardContext = queryShardContext;
|
||||
}
|
||||
|
||||
public String type() {
|
||||
|
@ -135,6 +140,10 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
|||
return parseFieldMatcher;
|
||||
}
|
||||
|
||||
public QueryShardContext queryShardContext() {
|
||||
return queryShardContext;
|
||||
}
|
||||
|
||||
public boolean isWithinMultiField() { return false; }
|
||||
|
||||
protected Function<String, TypeParser> typeParsers() { return typeParsers; }
|
||||
|
@ -150,7 +159,7 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
|||
|
||||
static class MultiFieldParserContext extends ParserContext {
|
||||
MultiFieldParserContext(ParserContext in) {
|
||||
super(in.type(), in.analysisService, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher());
|
||||
super(in.type(), in.analysisService, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher(), in.queryShardContext());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.elasticsearch.index.analysis.AnalysisService;
|
|||
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
|
||||
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
|
||||
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.InvalidTypeNameException;
|
||||
import org.elasticsearch.indices.TypeMissingException;
|
||||
|
@ -64,12 +65,12 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static java.util.Collections.unmodifiableSet;
|
||||
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
|
||||
|
||||
/**
|
||||
|
@ -116,11 +117,12 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
|
|||
final MapperRegistry mapperRegistry;
|
||||
|
||||
public MapperService(IndexSettings indexSettings, AnalysisService analysisService,
|
||||
SimilarityService similarityService, MapperRegistry mapperRegistry) {
|
||||
SimilarityService similarityService, MapperRegistry mapperRegistry,
|
||||
Supplier<QueryShardContext> queryShardContextSupplier) {
|
||||
super(indexSettings);
|
||||
this.analysisService = analysisService;
|
||||
this.fieldTypes = new FieldTypeLookup();
|
||||
this.documentParser = new DocumentMapperParser(indexSettings, this, analysisService, similarityService, mapperRegistry);
|
||||
this.documentParser = new DocumentMapperParser(indexSettings, this, analysisService, similarityService, mapperRegistry, queryShardContextSupplier);
|
||||
this.indexAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultIndexAnalyzer(), p -> p.indexAnalyzer());
|
||||
this.searchAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchAnalyzer(), p -> p.searchAnalyzer());
|
||||
this.searchQuoteAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchQuoteAnalyzer(), p -> p.searchQuoteAnalyzer());
|
||||
|
@ -131,8 +133,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
|
|||
"\"_default_\":{\n" +
|
||||
"\"properties\" : {\n" +
|
||||
"\"query\" : {\n" +
|
||||
"\"type\" : \"object\",\n" +
|
||||
"\"enabled\" : false\n" +
|
||||
"\"type\" : \"percolator\"\n" +
|
||||
"}\n" +
|
||||
"}\n" +
|
||||
"}\n" +
|
||||
|
|
|
@ -0,0 +1,233 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.queries.TermsQuery;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.BoostQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.elasticsearch.common.logging.support.LoggerMessageFormat;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Utility to extract query terms from queries and create queries from documents.
|
||||
*/
|
||||
public final class ExtractQueryTermsService {
|
||||
|
||||
private static final byte FIELD_VALUE_SEPARATOR = 0; // nul code point
|
||||
|
||||
private ExtractQueryTermsService() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts all terms from the specified query and adds it to the specified document.
|
||||
* @param query The query to extract terms from
|
||||
* @param document The document to add the extracted terms to
|
||||
* @param queryTermsFieldField The field in the document holding the extracted terms
|
||||
* @param unknownQueryField The field used to mark a document that not all query terms could be extracted. For example
|
||||
* the query contained an unsupported query (e.g. WildcardQuery).
|
||||
* @param fieldType The field type for the query metadata field
|
||||
*/
|
||||
public static void extractQueryTerms(Query query, ParseContext.Document document, String queryTermsFieldField, String unknownQueryField, FieldType fieldType) {
|
||||
Set<Term> queryTerms;
|
||||
try {
|
||||
queryTerms = extractQueryTerms(query);
|
||||
} catch (UnsupportedQueryException e) {
|
||||
document.add(new Field(unknownQueryField, new BytesRef(), fieldType));
|
||||
return;
|
||||
}
|
||||
for (Term term : queryTerms) {
|
||||
BytesRefBuilder builder = new BytesRefBuilder();
|
||||
builder.append(new BytesRef(term.field()));
|
||||
builder.append(FIELD_VALUE_SEPARATOR);
|
||||
builder.append(term.bytes());
|
||||
document.add(new Field(queryTermsFieldField, builder.toBytesRef(), fieldType));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts all query terms from the provided query and adds it to specified list.
|
||||
*
|
||||
* From boolean query with no should clauses or phrase queries only the the longest term are selected,
|
||||
* since that those terms are likely to be the rarest. Boolean query's must_not clauses are always ignored.
|
||||
*
|
||||
* If from part of the query, no query terms can be extracted then term extraction is stopped and
|
||||
* an UnsupportedQueryException is thrown.
|
||||
*/
|
||||
static Set<Term> extractQueryTerms(Query query) {
|
||||
// TODO: add support for the TermsQuery when it has methods to access the actual terms it encapsulates
|
||||
// TODO: add support for span queries
|
||||
if (query instanceof TermQuery) {
|
||||
return Collections.singleton(((TermQuery) query).getTerm());
|
||||
} else if (query instanceof PhraseQuery) {
|
||||
Term[] terms = ((PhraseQuery) query).getTerms();
|
||||
if (terms.length == 0) {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
// the longest term is likely to be the rarest,
|
||||
// so from a performance perspective it makes sense to extract that
|
||||
Term longestTerm = terms[0];
|
||||
for (Term term : terms) {
|
||||
if (longestTerm.bytes().length < term.bytes().length) {
|
||||
longestTerm = term;
|
||||
}
|
||||
}
|
||||
return Collections.singleton(longestTerm);
|
||||
} else if (query instanceof BooleanQuery) {
|
||||
List<BooleanClause> clauses = ((BooleanQuery) query).clauses();
|
||||
boolean hasRequiredClauses = false;
|
||||
for (BooleanClause clause : clauses) {
|
||||
if (clause.isRequired()) {
|
||||
hasRequiredClauses = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (hasRequiredClauses) {
|
||||
Set<Term> bestClause = null;
|
||||
for (BooleanClause clause : clauses) {
|
||||
if (clause.isRequired() == false) {
|
||||
// skip must_not clauses, we don't need to remember the things that do *not* match...
|
||||
// skip should clauses, this bq has must clauses, so we don't need to remember should clauses, since they are completely optional.
|
||||
continue;
|
||||
}
|
||||
|
||||
Set<Term> temp = extractQueryTerms(clause.getQuery());
|
||||
bestClause = selectTermListWithTheLongestShortestTerm(temp, bestClause);
|
||||
}
|
||||
if (bestClause != null) {
|
||||
return bestClause;
|
||||
} else {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
} else {
|
||||
Set<Term> terms = new HashSet<>();
|
||||
for (BooleanClause clause : clauses) {
|
||||
if (clause.isProhibited()) {
|
||||
// we don't need to remember the things that do *not* match...
|
||||
continue;
|
||||
}
|
||||
terms.addAll(extractQueryTerms(clause.getQuery()));
|
||||
}
|
||||
return terms;
|
||||
}
|
||||
} else if (query instanceof ConstantScoreQuery) {
|
||||
Query wrappedQuery = ((ConstantScoreQuery) query).getQuery();
|
||||
return extractQueryTerms(wrappedQuery);
|
||||
} else if (query instanceof BoostQuery) {
|
||||
Query wrappedQuery = ((BoostQuery) query).getQuery();
|
||||
return extractQueryTerms(wrappedQuery);
|
||||
} else {
|
||||
throw new UnsupportedQueryException(query);
|
||||
}
|
||||
}
|
||||
|
||||
static Set<Term> selectTermListWithTheLongestShortestTerm(Set<Term> terms1, Set<Term> terms2) {
|
||||
if (terms1 == null) {
|
||||
return terms2;
|
||||
} else if (terms2 == null) {
|
||||
return terms1;
|
||||
} else {
|
||||
int terms1ShortestTerm = minTermLength(terms1);
|
||||
int terms2ShortestTerm = minTermLength(terms2);
|
||||
// keep the clause with longest terms, this likely to be rarest.
|
||||
if (terms1ShortestTerm >= terms2ShortestTerm) {
|
||||
return terms1;
|
||||
} else {
|
||||
return terms2;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static int minTermLength(Set<Term> terms) {
|
||||
int min = Integer.MAX_VALUE;
|
||||
for (Term term : terms) {
|
||||
min = Math.min(min, term.bytes().length);
|
||||
}
|
||||
return min;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a boolean query with a should clause for each term on all fields of the specified index reader.
|
||||
*/
|
||||
public static Query createQueryTermsQuery(IndexReader indexReader, String queryMetadataField, String unknownQueryField) throws IOException {
|
||||
List<Term> extractedTerms = new ArrayList<>();
|
||||
extractedTerms.add(new Term(unknownQueryField));
|
||||
Fields fields = MultiFields.getFields(indexReader);
|
||||
for (String field : fields) {
|
||||
Terms terms = fields.terms(field);
|
||||
if (terms == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
BytesRef fieldBr = new BytesRef(field);
|
||||
TermsEnum tenum = terms.iterator();
|
||||
for (BytesRef term = tenum.next(); term != null ; term = tenum.next()) {
|
||||
BytesRefBuilder builder = new BytesRefBuilder();
|
||||
builder.append(fieldBr);
|
||||
builder.append(FIELD_VALUE_SEPARATOR);
|
||||
builder.append(term);
|
||||
extractedTerms.add(new Term(queryMetadataField, builder.toBytesRef()));
|
||||
}
|
||||
}
|
||||
return new TermsQuery(extractedTerms);
|
||||
}
|
||||
|
||||
/**
|
||||
* Exception indicating that none or some query terms couldn't extracted from a percolator query.
|
||||
*/
|
||||
public static class UnsupportedQueryException extends RuntimeException {
|
||||
|
||||
private final Query unsupportedQuery;
|
||||
|
||||
public UnsupportedQueryException(Query unsupportedQuery) {
|
||||
super(LoggerMessageFormat.format("no query terms can be extracted from query [{}]", unsupportedQuery));
|
||||
this.unsupportedQuery = unsupportedQuery;
|
||||
}
|
||||
|
||||
/**
|
||||
* The actual Lucene query that was unsupported and caused this exception to be thrown.
|
||||
*/
|
||||
public Query getUnsupportedQuery() {
|
||||
return unsupportedQuery;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,150 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Mapper;
|
||||
import org.elasticsearch.index.mapper.MapperBuilders;
|
||||
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
import org.elasticsearch.index.mapper.core.StringFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class PercolatorFieldMapper extends FieldMapper {
|
||||
|
||||
public static final String NAME = "query";
|
||||
public static final String CONTENT_TYPE = "percolator";
|
||||
public static final PercolatorFieldType FIELD_TYPE = new PercolatorFieldType();
|
||||
|
||||
private static final String EXTRACTED_TERMS_FIELD_NAME = "extracted_terms";
|
||||
private static final String UNKNOWN_QUERY_FIELD_NAME = "unknown_query";
|
||||
public static final String EXTRACTED_TERMS_FULL_FIELD_NAME = NAME + "." + EXTRACTED_TERMS_FIELD_NAME;
|
||||
public static final String UNKNOWN_QUERY_FULL_FIELD_NAME = NAME + "." + UNKNOWN_QUERY_FIELD_NAME;
|
||||
|
||||
public static class Builder extends FieldMapper.Builder<Builder, PercolatorFieldMapper> {
|
||||
|
||||
private final QueryShardContext queryShardContext;
|
||||
|
||||
public Builder(QueryShardContext queryShardContext) {
|
||||
super(NAME, FIELD_TYPE, FIELD_TYPE);
|
||||
this.queryShardContext = queryShardContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolatorFieldMapper build(BuilderContext context) {
|
||||
context.path().add(name);
|
||||
StringFieldMapper extractedTermsField = createStringFieldBuilder(EXTRACTED_TERMS_FIELD_NAME).build(context);
|
||||
StringFieldMapper unknownQueryField = createStringFieldBuilder(UNKNOWN_QUERY_FIELD_NAME).build(context);
|
||||
context.path().remove();
|
||||
return new PercolatorFieldMapper(name(), fieldType, defaultFieldType, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo, queryShardContext, extractedTermsField, unknownQueryField);
|
||||
}
|
||||
|
||||
static StringFieldMapper.Builder createStringFieldBuilder(String name) {
|
||||
StringFieldMapper.Builder queryMetaDataFieldBuilder = MapperBuilders.stringField(name);
|
||||
queryMetaDataFieldBuilder.docValues(false);
|
||||
queryMetaDataFieldBuilder.store(false);
|
||||
queryMetaDataFieldBuilder.tokenized(false);
|
||||
queryMetaDataFieldBuilder.indexOptions(IndexOptions.DOCS);
|
||||
return queryMetaDataFieldBuilder;
|
||||
}
|
||||
}
|
||||
|
||||
public static class TypeParser implements FieldMapper.TypeParser {
|
||||
|
||||
@Override
|
||||
public Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
|
||||
return new Builder(parserContext.queryShardContext());
|
||||
}
|
||||
}
|
||||
|
||||
public static final class PercolatorFieldType extends MappedFieldType {
|
||||
|
||||
public PercolatorFieldType() {
|
||||
setName(NAME);
|
||||
setIndexOptions(IndexOptions.NONE);
|
||||
setDocValuesType(DocValuesType.NONE);
|
||||
setStored(false);
|
||||
}
|
||||
|
||||
public PercolatorFieldType(MappedFieldType ref) {
|
||||
super(ref);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MappedFieldType clone() {
|
||||
return new PercolatorFieldType(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String typeName() {
|
||||
return CONTENT_TYPE;
|
||||
}
|
||||
}
|
||||
|
||||
private final boolean mapUnmappedFieldAsString;
|
||||
private final QueryShardContext queryShardContext;
|
||||
private final StringFieldMapper queryTermsField;
|
||||
private final StringFieldMapper unknownQueryField;
|
||||
|
||||
public PercolatorFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, MultiFields multiFields, CopyTo copyTo, QueryShardContext queryShardContext, StringFieldMapper queryTermsField, StringFieldMapper unknownQueryField) {
|
||||
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
|
||||
this.queryShardContext = queryShardContext;
|
||||
this.queryTermsField = queryTermsField;
|
||||
this.unknownQueryField = unknownQueryField;
|
||||
this.mapUnmappedFieldAsString = indexSettings.getAsBoolean(PercolatorQueriesRegistry.MAP_UNMAPPED_FIELDS_AS_STRING, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mapper parse(ParseContext context) throws IOException {
|
||||
QueryShardContext queryShardContext = new QueryShardContext(this.queryShardContext);
|
||||
Query query = PercolatorQueriesRegistry.parseQuery(queryShardContext, mapUnmappedFieldAsString, context.parser());
|
||||
if (context.flyweight() == false) {
|
||||
ExtractQueryTermsService.extractQueryTerms(query, context.doc(), queryTermsField.name(), unknownQueryField.name(), queryTermsField.fieldType());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Mapper> iterator() {
|
||||
return Arrays.<Mapper>asList(queryTermsField, unknownQueryField).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void parseCreateField(ParseContext context, List<Field> fields) throws IOException {
|
||||
throw new UnsupportedOperationException("should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String contentType() {
|
||||
return CONTENT_TYPE;
|
||||
}
|
||||
|
||||
}
|
|
@ -31,17 +31,11 @@ import org.elasticsearch.common.bytes.BytesReference;
|
|||
import org.elasticsearch.common.metrics.CounterMetric;
|
||||
import org.elasticsearch.common.metrics.MeanMetric;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.indexing.IndexingOperationListener;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.DocumentTypeListener;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
|
||||
|
@ -56,7 +50,6 @@ import java.io.IOException;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* Each shard will have a percolator registry even if there isn't a {@link PercolatorService#TYPE_NAME} document type in the index.
|
||||
|
@ -67,12 +60,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
*/
|
||||
public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent implements Closeable {
|
||||
|
||||
public final String MAP_UNMAPPED_FIELDS_AS_STRING = "index.percolator.map_unmapped_fields_as_string";
|
||||
|
||||
// This is a shard level service, but these below are index level service:
|
||||
private final MapperService mapperService;
|
||||
private final IndexFieldDataService indexFieldDataService;
|
||||
|
||||
public final static String MAP_UNMAPPED_FIELDS_AS_STRING = "index.percolator.map_unmapped_fields_as_string";
|
||||
|
||||
private final ConcurrentMap<BytesRef, Query> percolateQueries = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
|
||||
private final QueryShardContext queryShardContext;
|
||||
|
@ -81,16 +69,13 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
|
|||
private final CounterMetric currentMetric = new CounterMetric();
|
||||
private final CounterMetric numberOfQueries = new CounterMetric();
|
||||
|
||||
public PercolatorQueriesRegistry(ShardId shardId, IndexSettings indexSettings, MapperService mapperService,
|
||||
QueryShardContext queryShardContext, IndexFieldDataService indexFieldDataService) {
|
||||
public PercolatorQueriesRegistry(ShardId shardId, IndexSettings indexSettings, QueryShardContext queryShardContext) {
|
||||
super(shardId, indexSettings);
|
||||
this.mapperService = mapperService;
|
||||
this.queryShardContext = queryShardContext;
|
||||
this.indexFieldDataService = indexFieldDataService;
|
||||
this.mapUnmappedFieldsAsString = this.indexSettings.getSettings().getAsBoolean(MAP_UNMAPPED_FIELDS_AS_STRING, false);
|
||||
}
|
||||
|
||||
public ConcurrentMap<BytesRef, Query> percolateQueries() {
|
||||
public ConcurrentMap<BytesRef, Query> getPercolateQueries() {
|
||||
return percolateQueries;
|
||||
}
|
||||
|
||||
|
@ -120,9 +105,7 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
|
|||
}
|
||||
}
|
||||
|
||||
Query parsePercolatorDocument(String id, BytesReference source) {
|
||||
String type = null;
|
||||
BytesReference querySource = null;
|
||||
public Query parsePercolatorDocument(String id, BytesReference source) {
|
||||
try (XContentParser sourceParser = XContentHelper.createParser(source)) {
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token = sourceParser.nextToken(); // move the START_OBJECT
|
||||
|
@ -134,38 +117,21 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
|
|||
currentFieldName = sourceParser.currentName();
|
||||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
if ("query".equals(currentFieldName)) {
|
||||
if (type != null) {
|
||||
return parseQuery(type, sourceParser);
|
||||
} else {
|
||||
XContentBuilder builder = XContentFactory.contentBuilder(sourceParser.contentType());
|
||||
builder.copyCurrentStructure(sourceParser);
|
||||
querySource = builder.bytes();
|
||||
builder.close();
|
||||
}
|
||||
return parseQuery(queryShardContext, mapUnmappedFieldsAsString, sourceParser);
|
||||
} else {
|
||||
sourceParser.skipChildren();
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||
sourceParser.skipChildren();
|
||||
} else if (token.isValue()) {
|
||||
if ("type".equals(currentFieldName)) {
|
||||
type = sourceParser.text();
|
||||
}
|
||||
}
|
||||
}
|
||||
try (XContentParser queryParser = XContentHelper.createParser(querySource)) {
|
||||
return parseQuery(type, queryParser);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new PercolatorException(shardId().index(), "failed to parse query [" + id + "]", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Query parseQuery(String type, XContentParser parser) {
|
||||
String[] previousTypes = null;
|
||||
if (type != null) {
|
||||
previousTypes = QueryShardContext.setTypesWithPrevious(type);
|
||||
}
|
||||
public static Query parseQuery(QueryShardContext queryShardContext, boolean mapUnmappedFieldsAsString, XContentParser parser) {
|
||||
QueryShardContext context = new QueryShardContext(queryShardContext);
|
||||
try {
|
||||
context.reset(parser);
|
||||
|
@ -187,9 +153,6 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
|
|||
} catch (IOException e) {
|
||||
throw new ParsingException(parser.getTokenLocation(), "Failed to parse", e);
|
||||
} finally {
|
||||
if (type != null) {
|
||||
QueryShardContext.setTypes(previousTypes);
|
||||
}
|
||||
context.reset(null);
|
||||
}
|
||||
}
|
||||
|
@ -199,7 +162,7 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
|
|||
final int loadedQueries;
|
||||
try {
|
||||
Query query = new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME));
|
||||
QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger, mapperService, indexFieldDataService);
|
||||
QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(reader);
|
||||
indexSearcher.setQueryCache(null);
|
||||
indexSearcher.search(query, queryCollector);
|
||||
|
|
|
@ -45,17 +45,13 @@ final class QueriesLoaderCollector extends SimpleCollector {
|
|||
private final Map<BytesRef, Query> queries = new HashMap<>();
|
||||
private final FieldsVisitor fieldsVisitor = new FieldsVisitor(true);
|
||||
private final PercolatorQueriesRegistry percolator;
|
||||
private final IndexFieldData<?> uidFieldData;
|
||||
private final ESLogger logger;
|
||||
|
||||
private SortedBinaryDocValues uidValues;
|
||||
private LeafReader reader;
|
||||
|
||||
QueriesLoaderCollector(PercolatorQueriesRegistry percolator, ESLogger logger, MapperService mapperService, IndexFieldDataService indexFieldDataService) {
|
||||
QueriesLoaderCollector(PercolatorQueriesRegistry percolator, ESLogger logger) {
|
||||
this.percolator = percolator;
|
||||
this.logger = logger;
|
||||
final MappedFieldType uidMapper = mapperService.fullName(UidFieldMapper.NAME);
|
||||
this.uidFieldData = indexFieldDataService.getForField(uidMapper);
|
||||
}
|
||||
|
||||
public Map<BytesRef, Query> queries() {
|
||||
|
@ -64,35 +60,27 @@ final class QueriesLoaderCollector extends SimpleCollector {
|
|||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
// the _source is the query
|
||||
fieldsVisitor.reset();
|
||||
reader.document(doc, fieldsVisitor);
|
||||
final Uid uid = fieldsVisitor.uid();
|
||||
|
||||
uidValues.setDocument(doc);
|
||||
if (uidValues.count() > 0) {
|
||||
assert uidValues.count() == 1;
|
||||
final BytesRef uid = uidValues.valueAt(0);
|
||||
final BytesRef id = Uid.splitUidIntoTypeAndId(uid)[1];
|
||||
fieldsVisitor.reset();
|
||||
reader.document(doc, fieldsVisitor);
|
||||
|
||||
try {
|
||||
// id is only used for logging, if we fail we log the id in the catch statement
|
||||
final Query parseQuery = percolator.parsePercolatorDocument(null, fieldsVisitor.source());
|
||||
if (parseQuery != null) {
|
||||
queries.put(BytesRef.deepCopyOf(id), parseQuery);
|
||||
} else {
|
||||
logger.warn("failed to add query [{}] - parser returned null", id);
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to add query [{}]", e, id.utf8ToString());
|
||||
try {
|
||||
// id is only used for logging, if we fail we log the id in the catch statement
|
||||
final Query parseQuery = percolator.parsePercolatorDocument(null, fieldsVisitor.source());
|
||||
if (parseQuery != null) {
|
||||
queries.put(new BytesRef(uid.id()), parseQuery);
|
||||
} else {
|
||||
logger.warn("failed to add query [{}] - parser returned null", uid);
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to add query [{}]", e, uid);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
reader = context.reader();
|
||||
uidValues = uidFieldData.load(context).getBytesValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -267,7 +267,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId);
|
||||
this.provider = provider;
|
||||
this.searcherWrapper = indexSearcherWrapper;
|
||||
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, mapperService, newQueryShardContext(), indexFieldDataService);
|
||||
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, newQueryShardContext());
|
||||
// We start up inactive
|
||||
active.set(false);
|
||||
}
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
|||
import org.elasticsearch.index.mapper.internal.VersionFieldMapper;
|
||||
import org.elasticsearch.index.mapper.ip.IpFieldMapper;
|
||||
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||
import org.elasticsearch.index.percolator.PercolatorFieldMapper;
|
||||
import org.elasticsearch.index.query.BoolQueryParser;
|
||||
import org.elasticsearch.index.query.BoostingQueryParser;
|
||||
import org.elasticsearch.index.query.CommonTermsQueryParser;
|
||||
|
@ -211,6 +212,7 @@ public class IndicesModule extends AbstractModule {
|
|||
registerMapper(ObjectMapper.NESTED_CONTENT_TYPE, new ObjectMapper.TypeParser());
|
||||
registerMapper(CompletionFieldMapper.CONTENT_TYPE, new CompletionFieldMapper.TypeParser());
|
||||
registerMapper(GeoPointFieldMapper.CONTENT_TYPE, new GeoPointFieldMapper.TypeParser());
|
||||
registerMapper(PercolatorFieldMapper.CONTENT_TYPE, new PercolatorFieldMapper.TypeParser());
|
||||
|
||||
if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) {
|
||||
registerMapper(GeoShapeFieldMapper.CONTENT_TYPE, new GeoShapeFieldMapper.TypeParser());
|
||||
|
|
|
@ -29,9 +29,15 @@ import org.apache.lucene.index.LeafReader;
|
|||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
|
@ -76,7 +82,17 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex {
|
|||
try {
|
||||
MultiReader mReader = new MultiReader(memoryIndices, true);
|
||||
LeafReader slowReader = SlowCompositeReaderWrapper.wrap(mReader);
|
||||
final IndexSearcher slowSearcher = new IndexSearcher(slowReader);
|
||||
final IndexSearcher slowSearcher = new IndexSearcher(slowReader) {
|
||||
|
||||
@Override
|
||||
public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException {
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
bq.add(query, BooleanClause.Occur.MUST);
|
||||
bq.add(Queries.newNestedFilter(), BooleanClause.Occur.MUST_NOT);
|
||||
return super.createNormalizedWeight(bq.build(), needsScores);
|
||||
}
|
||||
|
||||
};
|
||||
slowSearcher.setQueryCache(null);
|
||||
DocSearcher docSearcher = new DocSearcher(slowSearcher, rootDocMemoryIndex);
|
||||
context.initialize(docSearcher, parsedDocument);
|
||||
|
|
|
@ -88,11 +88,7 @@ import java.util.concurrent.ConcurrentMap;
|
|||
*/
|
||||
public class PercolateContext extends SearchContext {
|
||||
|
||||
private final PercolatorQueriesRegistry percolateQueryRegistry;
|
||||
public boolean limit;
|
||||
private int size;
|
||||
public boolean doSort;
|
||||
public byte percolatorTypeId;
|
||||
private int size = 10;
|
||||
private boolean trackScores;
|
||||
|
||||
private final SearchShardTarget searchShardTarget;
|
||||
|
@ -102,10 +98,12 @@ public class PercolateContext extends SearchContext {
|
|||
private final PageCacheRecycler pageCacheRecycler;
|
||||
private final BigArrays bigArrays;
|
||||
private final ScriptService scriptService;
|
||||
private final MapperService mapperService;
|
||||
private final int numberOfShards;
|
||||
private final Query aliasFilter;
|
||||
private final long originNanoTime = System.nanoTime();
|
||||
private final long startTime;
|
||||
private final boolean onlyCount;
|
||||
private String[] types;
|
||||
|
||||
private Engine.Searcher docSearcher;
|
||||
|
@ -131,8 +129,8 @@ public class PercolateContext extends SearchContext {
|
|||
this.indexShard = indexShard;
|
||||
this.indexService = indexService;
|
||||
this.fieldDataService = indexService.fieldData();
|
||||
this.mapperService = indexService.mapperService();
|
||||
this.searchShardTarget = searchShardTarget;
|
||||
this.percolateQueryRegistry = indexShard.percolateRegistry();
|
||||
this.types = new String[]{request.documentType()};
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays.withCircuitBreaking();
|
||||
|
@ -143,6 +141,24 @@ public class PercolateContext extends SearchContext {
|
|||
this.numberOfShards = request.getNumberOfShards();
|
||||
this.aliasFilter = aliasFilter;
|
||||
this.startTime = request.getStartTime();
|
||||
this.onlyCount = request.onlyCount();
|
||||
}
|
||||
|
||||
// for testing:
|
||||
PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, MapperService mapperService) {
|
||||
super(null, request);
|
||||
this.searchShardTarget = searchShardTarget;
|
||||
this.mapperService = mapperService;
|
||||
this.indexService = null;
|
||||
this.indexShard = null;
|
||||
this.fieldDataService = null;
|
||||
this.pageCacheRecycler = null;
|
||||
this.bigArrays = null;
|
||||
this.scriptService = null;
|
||||
this.aliasFilter = null;
|
||||
this.startTime = 0;
|
||||
this.numberOfShards = 0;
|
||||
this.onlyCount = true;
|
||||
}
|
||||
|
||||
public IndexSearcher docSearcher() {
|
||||
|
@ -177,10 +193,6 @@ public class PercolateContext extends SearchContext {
|
|||
return indexService;
|
||||
}
|
||||
|
||||
public ConcurrentMap<BytesRef, Query> percolateQueries() {
|
||||
return percolateQueryRegistry.percolateQueries();
|
||||
}
|
||||
|
||||
public Query percolateQuery() {
|
||||
return percolateQuery;
|
||||
}
|
||||
|
@ -196,6 +208,14 @@ public class PercolateContext extends SearchContext {
|
|||
return hitContext;
|
||||
}
|
||||
|
||||
public boolean isOnlyCount() {
|
||||
return onlyCount;
|
||||
}
|
||||
|
||||
public Query percolatorTypeFilter(){
|
||||
return indexService().mapperService().documentMapper(PercolatorService.TYPE_NAME).typeFilter();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContextHighlight highlight() {
|
||||
return highlight;
|
||||
|
@ -230,7 +250,7 @@ public class PercolateContext extends SearchContext {
|
|||
|
||||
@Override
|
||||
public MapperService mapperService() {
|
||||
return indexService.mapperService();
|
||||
return mapperService;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -531,7 +551,6 @@ public class PercolateContext extends SearchContext {
|
|||
@Override
|
||||
public SearchContext size(int size) {
|
||||
this.size = size;
|
||||
this.limit = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,224 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.action.percolate.PercolateShardRequest;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
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.index.mapper.DocumentMapperForType;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.search.SearchParseElement;
|
||||
import org.elasticsearch.search.aggregations.AggregationPhase;
|
||||
import org.elasticsearch.search.highlight.HighlightPhase;
|
||||
import org.elasticsearch.search.sort.SortParseElement;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.index.mapper.SourceToParse.source;
|
||||
|
||||
public class PercolateDocumentParser {
|
||||
|
||||
private final HighlightPhase highlightPhase;
|
||||
private final SortParseElement sortParseElement;
|
||||
private final AggregationPhase aggregationPhase;
|
||||
private final MappingUpdatedAction mappingUpdatedAction;
|
||||
|
||||
@Inject
|
||||
public PercolateDocumentParser(HighlightPhase highlightPhase, SortParseElement sortParseElement, AggregationPhase aggregationPhase, MappingUpdatedAction mappingUpdatedAction) {
|
||||
this.highlightPhase = highlightPhase;
|
||||
this.sortParseElement = sortParseElement;
|
||||
this.aggregationPhase = aggregationPhase;
|
||||
this.mappingUpdatedAction = mappingUpdatedAction;
|
||||
}
|
||||
|
||||
public ParsedDocument parse(PercolateShardRequest request, PercolateContext context, MapperService mapperService, QueryShardContext queryShardContext) {
|
||||
BytesReference source = request.source();
|
||||
if (source == null || source.length() == 0) {
|
||||
if (request.docSource() != null && request.docSource().length() != 0) {
|
||||
return parseFetchedDoc(context, request.docSource(), mapperService, request.shardId().getIndex(), request.documentType());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: combine all feature parse elements into one map
|
||||
Map<String, ? extends SearchParseElement> hlElements = highlightPhase.parseElements();
|
||||
Map<String, ? extends SearchParseElement> aggregationElements = aggregationPhase.parseElements();
|
||||
|
||||
ParsedDocument doc = null;
|
||||
// Some queries (function_score query when for decay functions) rely on a SearchContext being set:
|
||||
// We switch types because this context needs to be in the context of the percolate queries in the shard and
|
||||
// not the in memory percolate doc
|
||||
String[] previousTypes = context.types();
|
||||
context.types(new String[]{PercolatorService.TYPE_NAME});
|
||||
try (XContentParser parser = XContentFactory.xContent(source).createParser(source);) {
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
// we need to check the "doc" here, so the next token will be START_OBJECT which is
|
||||
// the actual document starting
|
||||
if ("doc".equals(currentFieldName)) {
|
||||
if (doc != null) {
|
||||
throw new ElasticsearchParseException("Either specify doc or get, not both");
|
||||
}
|
||||
|
||||
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
|
||||
String index = context.shardTarget().index();
|
||||
doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
|
||||
if (docMapper.getMapping() != null) {
|
||||
doc.addDynamicMappingsUpdate(docMapper.getMapping());
|
||||
}
|
||||
if (doc.dynamicMappingsUpdate() != null) {
|
||||
mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate());
|
||||
}
|
||||
// the document parsing exists the "doc" object, so we need to set the new current field.
|
||||
currentFieldName = parser.currentName();
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
SearchParseElement element = hlElements.get(currentFieldName);
|
||||
if (element == null) {
|
||||
element = aggregationElements.get(currentFieldName);
|
||||
}
|
||||
|
||||
if ("query".equals(currentFieldName)) {
|
||||
if (context.percolateQuery() != null) {
|
||||
throw new ElasticsearchParseException("Either specify query or filter, not both");
|
||||
}
|
||||
context.percolateQuery(queryShardContext.parse(parser).query());
|
||||
} else if ("filter".equals(currentFieldName)) {
|
||||
if (context.percolateQuery() != null) {
|
||||
throw new ElasticsearchParseException("Either specify query or filter, not both");
|
||||
}
|
||||
Query filter = queryShardContext.parseInnerFilter(parser).query();
|
||||
context.percolateQuery(new ConstantScoreQuery(filter));
|
||||
} else if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
} else if (element != null) {
|
||||
element.parse(parser, context);
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||
if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
}
|
||||
} else if (token == null) {
|
||||
break;
|
||||
} else if (token.isValue()) {
|
||||
if ("size".equals(currentFieldName)) {
|
||||
context.size(parser.intValue());
|
||||
if (context.size() < 0) {
|
||||
throw new ElasticsearchParseException("size is set to [{}] and is expected to be higher or equal to 0", context.size());
|
||||
}
|
||||
} else if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
} else if ("track_scores".equals(currentFieldName) || "trackScores".equals(currentFieldName)) {
|
||||
context.trackScores(parser.booleanValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// We need to get the actual source from the request body for highlighting, so parse the request body again
|
||||
// and only get the doc source.
|
||||
if (context.highlight() != null) {
|
||||
parser.close();
|
||||
currentFieldName = null;
|
||||
try (XContentParser parserForHighlighter = XContentFactory.xContent(source).createParser(source)) {
|
||||
token = parserForHighlighter.nextToken();
|
||||
assert token == XContentParser.Token.START_OBJECT;
|
||||
while ((token = parserForHighlighter.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parserForHighlighter.currentName();
|
||||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
if ("doc".equals(currentFieldName)) {
|
||||
BytesStreamOutput bStream = new BytesStreamOutput();
|
||||
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.SMILE, bStream);
|
||||
builder.copyCurrentStructure(parserForHighlighter);
|
||||
builder.close();
|
||||
doc.setSource(bStream.bytes());
|
||||
break;
|
||||
} else {
|
||||
parserForHighlighter.skipChildren();
|
||||
}
|
||||
} else if (token == null) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
throw new ElasticsearchParseException("failed to parse request", e);
|
||||
} finally {
|
||||
context.types(previousTypes);
|
||||
}
|
||||
|
||||
if (request.docSource() != null && request.docSource().length() != 0) {
|
||||
if (doc != null) {
|
||||
throw new IllegalArgumentException("Can't specify the document to percolate in the source of the request and as document id");
|
||||
}
|
||||
|
||||
doc = parseFetchedDoc(context, request.docSource(), mapperService, request.shardId().getIndex(), request.documentType());
|
||||
}
|
||||
|
||||
if (doc == null) {
|
||||
throw new IllegalArgumentException("Nothing to percolate");
|
||||
}
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
private void parseSort(XContentParser parser, PercolateContext context) throws Exception {
|
||||
context.trackScores(true);
|
||||
sortParseElement.parse(parser, context);
|
||||
// null, means default sorting by relevancy
|
||||
if (context.sort() != null) {
|
||||
throw new ElasticsearchParseException("Only _score desc is supported");
|
||||
}
|
||||
}
|
||||
|
||||
private ParsedDocument parseFetchedDoc(PercolateContext context, BytesReference fetchedDoc, MapperService mapperService, String index, String type) {
|
||||
try (XContentParser parser = XContentFactory.xContent(fetchedDoc).createParser(fetchedDoc)) {
|
||||
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
|
||||
ParsedDocument doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(type).flyweight(true));
|
||||
if (doc == null) {
|
||||
throw new ElasticsearchParseException("No doc to percolate in the request");
|
||||
}
|
||||
if (context.highlight() != null) {
|
||||
doc.setSource(fetchedDoc);
|
||||
}
|
||||
return doc;
|
||||
} catch (Throwable e) {
|
||||
throw new ElasticsearchParseException("failed to parse request", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -27,6 +27,7 @@ public class PercolatorModule extends AbstractModule {
|
|||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(PercolateDocumentParser.class).asEagerSingleton();
|
||||
bind(PercolatorService.class).asEagerSingleton();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,250 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TwoPhaseIterator;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.percolator.ExtractQueryTermsService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.lucene.search.BooleanClause.Occur.FILTER;
|
||||
import static org.apache.lucene.search.BooleanClause.Occur.MUST;
|
||||
|
||||
final class PercolatorQuery extends Query {
|
||||
|
||||
public static final float MATCH_COST =
|
||||
(1 << 14) // stored field access cost, approximated by the number of bytes in a block
|
||||
+ 1000; // cost of matching the query against the document, arbitrary as it would be really complex to estimate
|
||||
|
||||
static class Builder {
|
||||
|
||||
private final IndexSearcher percolatorIndexSearcher;
|
||||
private final Map<BytesRef, Query> percolatorQueries;
|
||||
|
||||
private Query percolateQuery;
|
||||
private Query queriesMetaDataQuery;
|
||||
private final Query percolateTypeQuery;
|
||||
|
||||
/**
|
||||
* @param percolatorIndexSearcher The index searcher on top of the in-memory index that holds the document being percolated
|
||||
* @param percolatorQueries All the registered percolator queries
|
||||
* @param percolateTypeQuery A query that identifies all document containing percolator queries
|
||||
*/
|
||||
Builder(IndexSearcher percolatorIndexSearcher, Map<BytesRef, Query> percolatorQueries, Query percolateTypeQuery) {
|
||||
this.percolatorIndexSearcher = percolatorIndexSearcher;
|
||||
this.percolatorQueries = percolatorQueries;
|
||||
this.percolateTypeQuery = percolateTypeQuery;
|
||||
}
|
||||
|
||||
/**
|
||||
* Optionally sets a query that reduces the number of queries to percolate based on custom metadata attached
|
||||
* on the percolator documents.
|
||||
*/
|
||||
void setPercolateQuery(Query percolateQuery) {
|
||||
this.percolateQuery = percolateQuery;
|
||||
}
|
||||
|
||||
/**
|
||||
* Optionally sets a query that reduces the number of queries to percolate based on extracted terms from
|
||||
* the document to be percolated.
|
||||
*
|
||||
* @param extractedTermsFieldName The name of the field to get the extracted terms from
|
||||
* @param unknownQueryFieldname The field used to mark documents whose queries couldn't all get extracted
|
||||
*/
|
||||
void extractQueryTermsQuery(String extractedTermsFieldName, String unknownQueryFieldname) throws IOException {
|
||||
this.queriesMetaDataQuery = ExtractQueryTermsService.createQueryTermsQuery(percolatorIndexSearcher.getIndexReader(), extractedTermsFieldName, unknownQueryFieldname);
|
||||
}
|
||||
|
||||
PercolatorQuery build() {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
builder.add(percolateTypeQuery, FILTER);
|
||||
if (queriesMetaDataQuery != null) {
|
||||
builder.add(queriesMetaDataQuery, FILTER);
|
||||
}
|
||||
if (percolateQuery != null){
|
||||
builder.add(percolateQuery, MUST);
|
||||
}
|
||||
return new PercolatorQuery(builder.build(), percolatorIndexSearcher, percolatorQueries);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final Query percolatorQueriesQuery;
|
||||
private final IndexSearcher percolatorIndexSearcher;
|
||||
private final Map<BytesRef, Query> percolatorQueries;
|
||||
|
||||
private PercolatorQuery(Query percolatorQueriesQuery, IndexSearcher percolatorIndexSearcher, Map<BytesRef, Query> percolatorQueries) {
|
||||
this.percolatorQueriesQuery = percolatorQueriesQuery;
|
||||
this.percolatorIndexSearcher = percolatorIndexSearcher;
|
||||
this.percolatorQueries = percolatorQueries;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
if (getBoost() != 1f) {
|
||||
return super.rewrite(reader);
|
||||
}
|
||||
|
||||
Query rewritten = percolatorQueriesQuery.rewrite(reader);
|
||||
if (rewritten != percolatorQueriesQuery) {
|
||||
return new PercolatorQuery(rewritten, percolatorIndexSearcher, percolatorQueries);
|
||||
} else {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
final Weight innerWeight = percolatorQueriesQuery.createWeight(searcher, needsScores);
|
||||
return new Weight(this) {
|
||||
@Override
|
||||
public void extractTerms(Set<Term> set) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext leafReaderContext, int docId) throws IOException {
|
||||
Scorer scorer = scorer(leafReaderContext);
|
||||
if (scorer != null) {
|
||||
int result = scorer.iterator().advance(docId);
|
||||
if (result == docId) {
|
||||
return Explanation.match(scorer.score(), "PercolatorQuery");
|
||||
}
|
||||
}
|
||||
return Explanation.noMatch("PercolatorQuery");
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getValueForNormalization() throws IOException {
|
||||
return innerWeight.getValueForNormalization();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void normalize(float v, float v1) {
|
||||
innerWeight.normalize(v, v1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext leafReaderContext) throws IOException {
|
||||
final Scorer approximation = innerWeight.scorer(leafReaderContext);
|
||||
if (approximation == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final LeafReader leafReader = leafReaderContext.reader();
|
||||
return new Scorer(this) {
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TwoPhaseIterator twoPhaseIterator() {
|
||||
return new TwoPhaseIterator(approximation.iterator()) {
|
||||
@Override
|
||||
public boolean matches() throws IOException {
|
||||
return matchDocId(approximation.docID(), leafReader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float matchCost() {
|
||||
return MATCH_COST;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return approximation.score();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return approximation.freq();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return approximation.docID();
|
||||
}
|
||||
|
||||
boolean matchDocId(int docId, LeafReader leafReader) throws IOException {
|
||||
SingleFieldsVisitor singleFieldsVisitor = new SingleFieldsVisitor(UidFieldMapper.NAME);
|
||||
leafReader.document(docId, singleFieldsVisitor);
|
||||
BytesRef percolatorQueryId = new BytesRef(singleFieldsVisitor.uid().id());
|
||||
return matchQuery(percolatorQueryId);
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
boolean matchQuery(BytesRef percolatorQueryId) throws IOException {
|
||||
Query percolatorQuery = percolatorQueries.get(percolatorQueryId);
|
||||
if (percolatorQuery != null) {
|
||||
return Lucene.exists(percolatorIndexSearcher, percolatorQuery);
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private final Object instance = new Object();
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (!super.equals(o)) return false;
|
||||
|
||||
PercolatorQuery that = (PercolatorQuery) o;
|
||||
|
||||
return instance.equals(that.instance);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = super.hashCode();
|
||||
result = 31 * result + instance.hashCode();
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String s) {
|
||||
return "PercolatorQuery{inner={" + percolatorQueriesQuery.toString(s) + "}}";
|
||||
}
|
||||
}
|
|
@ -18,134 +18,110 @@
|
|||
*/
|
||||
package org.elasticsearch.percolator;
|
||||
|
||||
import com.carrotsearch.hppc.IntObjectHashMap;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.memory.ExtendedMemoryIndex;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.MultiCollector;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.TopScoreDocCollector;
|
||||
import org.apache.lucene.search.TotalHitCountCollector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.percolate.PercolateResponse;
|
||||
import org.elasticsearch.action.percolate.PercolateShardRequest;
|
||||
import org.elasticsearch.action.percolate.PercolateShardResponse;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.HasContextAndHeaders;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.text.Text;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
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.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.index.mapper.DocumentMapperForType;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.percolator.PercolatorFieldMapper;
|
||||
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
|
||||
import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.percolator.QueryCollector.Count;
|
||||
import org.elasticsearch.percolator.QueryCollector.Match;
|
||||
import org.elasticsearch.percolator.QueryCollector.MatchAndScore;
|
||||
import org.elasticsearch.percolator.QueryCollector.MatchAndSort;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.search.SearchParseElement;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.AggregationPhase;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.highlight.HighlightField;
|
||||
import org.elasticsearch.search.highlight.HighlightPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.sort.SortParseElement;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static org.elasticsearch.index.mapper.SourceToParse.source;
|
||||
import static org.elasticsearch.percolator.QueryCollector.count;
|
||||
import static org.elasticsearch.percolator.QueryCollector.match;
|
||||
import static org.elasticsearch.percolator.QueryCollector.matchAndScore;
|
||||
import static org.apache.lucene.search.BooleanClause.Occur.FILTER;
|
||||
import static org.apache.lucene.search.BooleanClause.Occur.MUST;
|
||||
|
||||
public class PercolatorService extends AbstractComponent {
|
||||
|
||||
public final static float NO_SCORE = Float.NEGATIVE_INFINITY;
|
||||
public final static String TYPE_NAME = ".percolator";
|
||||
|
||||
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
||||
private final IndicesService indicesService;
|
||||
private final IntObjectHashMap<PercolatorType> percolatorTypes;
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
private final BigArrays bigArrays;
|
||||
private final ScriptService scriptService;
|
||||
private final IndicesService indicesService;
|
||||
private final ClusterService clusterService;
|
||||
private final HighlightPhase highlightPhase;
|
||||
private final AggregationPhase aggregationPhase;
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
private final ParseFieldMatcher parseFieldMatcher;
|
||||
private final CloseableThreadLocal<MemoryIndex> cache;
|
||||
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
||||
private final PercolateDocumentParser percolateDocumentParser;
|
||||
|
||||
private final PercolatorIndex single;
|
||||
private final PercolatorIndex multi;
|
||||
|
||||
private final HighlightPhase highlightPhase;
|
||||
private final AggregationPhase aggregationPhase;
|
||||
private final SortParseElement sortParseElement;
|
||||
private final ScriptService scriptService;
|
||||
private final MappingUpdatedAction mappingUpdatedAction;
|
||||
|
||||
private final CloseableThreadLocal<MemoryIndex> cache;
|
||||
|
||||
private final ParseFieldMatcher parseFieldMatcher;
|
||||
|
||||
@Inject
|
||||
public PercolatorService(Settings settings, IndexNameExpressionResolver indexNameExpressionResolver, IndicesService indicesService,
|
||||
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays,
|
||||
HighlightPhase highlightPhase, ClusterService clusterService,
|
||||
AggregationPhase aggregationPhase, ScriptService scriptService,
|
||||
MappingUpdatedAction mappingUpdatedAction) {
|
||||
PercolateDocumentParser percolateDocumentParser) {
|
||||
super(settings);
|
||||
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
||||
this.percolateDocumentParser = percolateDocumentParser;
|
||||
this.parseFieldMatcher = new ParseFieldMatcher(settings);
|
||||
this.indicesService = indicesService;
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays;
|
||||
this.clusterService = clusterService;
|
||||
this.highlightPhase = highlightPhase;
|
||||
this.aggregationPhase = aggregationPhase;
|
||||
this.scriptService = scriptService;
|
||||
this.mappingUpdatedAction = mappingUpdatedAction;
|
||||
this.sortParseElement = new SortParseElement();
|
||||
this.aggregationPhase = aggregationPhase;
|
||||
this.highlightPhase = highlightPhase;
|
||||
|
||||
final long maxReuseBytes = settings.getAsBytesSize("indices.memory.memory_index.size_per_thread", new ByteSizeValue(1, ByteSizeUnit.MB)).bytes();
|
||||
cache = new CloseableThreadLocal<MemoryIndex>() {
|
||||
|
@ -157,23 +133,41 @@ public class PercolatorService extends AbstractComponent {
|
|||
};
|
||||
single = new SingleDocumentPercolatorIndex(cache);
|
||||
multi = new MultiDocumentPercolatorIndex(cache);
|
||||
|
||||
percolatorTypes = new IntObjectHashMap<>(6);
|
||||
percolatorTypes.put(countPercolator.id(), countPercolator);
|
||||
percolatorTypes.put(queryCountPercolator.id(), queryCountPercolator);
|
||||
percolatorTypes.put(matchPercolator.id(), matchPercolator);
|
||||
percolatorTypes.put(queryPercolator.id(), queryPercolator);
|
||||
percolatorTypes.put(scoringPercolator.id(), scoringPercolator);
|
||||
percolatorTypes.put(topMatchingPercolator.id(), topMatchingPercolator);
|
||||
}
|
||||
|
||||
public ReduceResult reduce(boolean onlyCount, List<PercolateShardResponse> shardResponses, HasContextAndHeaders headersContext) throws IOException {
|
||||
if (onlyCount) {
|
||||
long finalCount = 0;
|
||||
for (PercolateShardResponse shardResponse : shardResponses) {
|
||||
finalCount += shardResponse.topDocs().totalHits;
|
||||
}
|
||||
|
||||
public ReduceResult reduce(byte percolatorTypeId, List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
PercolatorType percolatorType = percolatorTypes.get(percolatorTypeId);
|
||||
return percolatorType.reduce(shardResults, headersContext);
|
||||
InternalAggregations reducedAggregations = reduceAggregations(shardResponses, headersContext);
|
||||
return new PercolatorService.ReduceResult(finalCount, reducedAggregations);
|
||||
} else {
|
||||
int requestedSize = shardResponses.get(0).requestedSize();
|
||||
TopDocs[] shardResults = new TopDocs[shardResponses.size()];
|
||||
long foundMatches = 0;
|
||||
for (int i = 0; i < shardResults.length; i++) {
|
||||
TopDocs shardResult = shardResponses.get(i).topDocs();
|
||||
foundMatches += shardResult.totalHits;
|
||||
shardResults[i] = shardResult;
|
||||
}
|
||||
TopDocs merged = TopDocs.merge(requestedSize, shardResults);
|
||||
PercolateResponse.Match[] matches = new PercolateResponse.Match[merged.scoreDocs.length];
|
||||
for (int i = 0; i < merged.scoreDocs.length; i++) {
|
||||
ScoreDoc doc = merged.scoreDocs[i];
|
||||
PercolateShardResponse shardResponse = shardResponses.get(doc.shardIndex);
|
||||
String id = shardResponse.ids().get(doc.doc);
|
||||
Map<String, HighlightField> hl = shardResponse.hls().get(doc.doc);
|
||||
matches[i] = new PercolateResponse.Match(new Text(shardResponse.getIndex()), new Text(id), doc.score, hl);
|
||||
}
|
||||
InternalAggregations reducedAggregations = reduceAggregations(shardResponses, headersContext);
|
||||
return new PercolatorService.ReduceResult(foundMatches, matches, reducedAggregations);
|
||||
}
|
||||
}
|
||||
|
||||
public PercolateShardResponse percolate(PercolateShardRequest request) {
|
||||
public PercolateShardResponse percolate(PercolateShardRequest request) throws IOException {
|
||||
IndexService percolateIndexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||
IndexShard indexShard = percolateIndexService.getShard(request.shardId().id());
|
||||
indexShard.readAllowed(); // check if we can read the shard...
|
||||
|
@ -196,29 +190,11 @@ public class PercolatorService extends AbstractComponent {
|
|||
);
|
||||
SearchContext.setCurrent(context);
|
||||
try {
|
||||
ParsedDocument parsedDocument = parseRequest(indexShard, request, context, request.shardId().getIndex());
|
||||
if (context.percolateQueries().isEmpty()) {
|
||||
return new PercolateShardResponse(context, request.shardId());
|
||||
}
|
||||
ParsedDocument parsedDocument = percolateDocumentParser.parse(request, context, percolateIndexService.mapperService(), percolateIndexService.getQueryShardContext());
|
||||
|
||||
if (request.docSource() != null && request.docSource().length() != 0) {
|
||||
parsedDocument = parseFetchedDoc(context, request.docSource(), percolateIndexService, request.shardId().getIndex(), request.documentType());
|
||||
} else if (parsedDocument == null) {
|
||||
throw new IllegalArgumentException("Nothing to percolate");
|
||||
if (context.searcher().getIndexReader().maxDoc() == 0) {
|
||||
return new PercolateShardResponse(Lucene.EMPTY_TOP_DOCS, Collections.emptyMap(), Collections.emptyMap(), context);
|
||||
}
|
||||
|
||||
if (context.percolateQuery() == null && (context.trackScores() || context.doSort || context.aggregations() != null) || context.aliasFilter() != null) {
|
||||
context.percolateQuery(new MatchAllDocsQuery());
|
||||
}
|
||||
|
||||
if (context.doSort && !context.limit) {
|
||||
throw new IllegalArgumentException("Can't sort if size isn't specified");
|
||||
}
|
||||
|
||||
if (context.highlight() != null && !context.limit) {
|
||||
throw new IllegalArgumentException("Can't highlight if size isn't specified");
|
||||
}
|
||||
|
||||
if (context.size() < 0) {
|
||||
context.size(0);
|
||||
}
|
||||
|
@ -232,23 +208,27 @@ public class PercolatorService extends AbstractComponent {
|
|||
} else {
|
||||
percolatorIndex = single;
|
||||
}
|
||||
|
||||
PercolatorType action;
|
||||
if (request.onlyCount()) {
|
||||
action = context.percolateQuery() != null ? queryCountPercolator : countPercolator;
|
||||
} else {
|
||||
if (context.doSort) {
|
||||
action = topMatchingPercolator;
|
||||
} else if (context.percolateQuery() != null) {
|
||||
action = context.trackScores() ? scoringPercolator : queryPercolator;
|
||||
} else {
|
||||
action = matchPercolator;
|
||||
}
|
||||
}
|
||||
context.percolatorTypeId = action.id();
|
||||
|
||||
percolatorIndex.prepare(context, parsedDocument);
|
||||
return action.doPercolate(request, context, isNested);
|
||||
|
||||
BucketCollector aggregatorCollector = null;
|
||||
if (context.aggregations() != null) {
|
||||
AggregationContext aggregationContext = new AggregationContext(context);
|
||||
context.aggregations().aggregationContext(aggregationContext);
|
||||
|
||||
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
|
||||
List<Aggregator> aggregatorCollectors = new ArrayList<>(aggregators.length);
|
||||
for (int i = 0; i < aggregators.length; i++) {
|
||||
if (!(aggregators[i] instanceof GlobalAggregator)) {
|
||||
Aggregator aggregator = aggregators[i];
|
||||
aggregatorCollectors.add(aggregator);
|
||||
}
|
||||
}
|
||||
context.aggregations().aggregators(aggregators);
|
||||
aggregatorCollector = BucketCollector.wrap(aggregatorCollectors);
|
||||
aggregatorCollector.preCollection();
|
||||
}
|
||||
PercolatorQueriesRegistry queriesRegistry = indexShard.percolateRegistry();
|
||||
return doPercolate(context, queriesRegistry, aggregationPhase, aggregatorCollector, highlightPhase);
|
||||
} finally {
|
||||
SearchContext.removeCurrent();
|
||||
context.close();
|
||||
|
@ -256,566 +236,101 @@ public class PercolatorService extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
private ParsedDocument parseRequest(IndexShard shard, PercolateShardRequest request, PercolateContext context, String index) {
|
||||
BytesReference source = request.source();
|
||||
if (source == null || source.length() == 0) {
|
||||
return null;
|
||||
// moved the core percolation logic to a pck protected method to make testing easier:
|
||||
static PercolateShardResponse doPercolate(PercolateContext context, PercolatorQueriesRegistry queriesRegistry, AggregationPhase aggregationPhase, @Nullable BucketCollector aggregatorCollector, HighlightPhase highlightPhase) throws IOException {
|
||||
PercolatorQuery.Builder builder = new PercolatorQuery.Builder(context.docSearcher(), queriesRegistry.getPercolateQueries(), context.percolatorTypeFilter());
|
||||
if (queriesRegistry.indexSettings().getSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null).onOrAfter(Version.V_3_0_0)) {
|
||||
builder.extractQueryTermsQuery(PercolatorFieldMapper.EXTRACTED_TERMS_FULL_FIELD_NAME, PercolatorFieldMapper.UNKNOWN_QUERY_FULL_FIELD_NAME);
|
||||
}
|
||||
|
||||
// TODO: combine all feature parse elements into one map
|
||||
Map<String, ? extends SearchParseElement> hlElements = highlightPhase.parseElements();
|
||||
Map<String, ? extends SearchParseElement> aggregationElements = aggregationPhase.parseElements();
|
||||
|
||||
ParsedDocument doc = null;
|
||||
XContentParser parser = null;
|
||||
|
||||
// Some queries (function_score query when for decay functions) rely on a SearchContext being set:
|
||||
// We switch types because this context needs to be in the context of the percolate queries in the shard and
|
||||
// not the in memory percolate doc
|
||||
String[] previousTypes = context.types();
|
||||
context.types(new String[]{TYPE_NAME});
|
||||
QueryShardContext queryShardContext = shard.getQueryShardContext();
|
||||
try {
|
||||
parser = XContentFactory.xContent(source).createParser(source);
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
// we need to check the "doc" here, so the next token will be START_OBJECT which is
|
||||
// the actual document starting
|
||||
if ("doc".equals(currentFieldName)) {
|
||||
if (doc != null) {
|
||||
throw new ElasticsearchParseException("Either specify doc or get, not both");
|
||||
}
|
||||
|
||||
MapperService mapperService = shard.mapperService();
|
||||
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
|
||||
doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
|
||||
if (docMapper.getMapping() != null) {
|
||||
doc.addDynamicMappingsUpdate(docMapper.getMapping());
|
||||
}
|
||||
if (doc.dynamicMappingsUpdate() != null) {
|
||||
mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate());
|
||||
}
|
||||
// the document parsing exists the "doc" object, so we need to set the new current field.
|
||||
currentFieldName = parser.currentName();
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
SearchParseElement element = hlElements.get(currentFieldName);
|
||||
if (element == null) {
|
||||
element = aggregationElements.get(currentFieldName);
|
||||
}
|
||||
|
||||
if ("query".equals(currentFieldName)) {
|
||||
if (context.percolateQuery() != null) {
|
||||
throw new ElasticsearchParseException("Either specify query or filter, not both");
|
||||
}
|
||||
context.percolateQuery(queryShardContext.parse(parser).query());
|
||||
} else if ("filter".equals(currentFieldName)) {
|
||||
if (context.percolateQuery() != null) {
|
||||
throw new ElasticsearchParseException("Either specify query or filter, not both");
|
||||
}
|
||||
Query filter = queryShardContext.parseInnerFilter(parser).query();
|
||||
context.percolateQuery(new ConstantScoreQuery(filter));
|
||||
} else if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
} else if (element != null) {
|
||||
element.parse(parser, context);
|
||||
}
|
||||
} else if (token == XContentParser.Token.START_ARRAY) {
|
||||
if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
}
|
||||
} else if (token == null) {
|
||||
break;
|
||||
} else if (token.isValue()) {
|
||||
if ("size".equals(currentFieldName)) {
|
||||
context.size(parser.intValue());
|
||||
if (context.size() < 0) {
|
||||
throw new ElasticsearchParseException("size is set to [{}] and is expected to be higher or equal to 0", context.size());
|
||||
}
|
||||
} else if ("sort".equals(currentFieldName)) {
|
||||
parseSort(parser, context);
|
||||
} else if ("track_scores".equals(currentFieldName) || "trackScores".equals(currentFieldName)) {
|
||||
context.trackScores(parser.booleanValue());
|
||||
}
|
||||
}
|
||||
if (context.percolateQuery() != null || context.aliasFilter() != null) {
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
if (context.percolateQuery() != null) {
|
||||
bq.add(context.percolateQuery(), MUST);
|
||||
}
|
||||
|
||||
// We need to get the actual source from the request body for highlighting, so parse the request body again
|
||||
// and only get the doc source.
|
||||
if (context.highlight() != null) {
|
||||
parser.close();
|
||||
currentFieldName = null;
|
||||
parser = XContentFactory.xContent(source).createParser(source);
|
||||
token = parser.nextToken();
|
||||
assert token == XContentParser.Token.START_OBJECT;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
if ("doc".equals(currentFieldName)) {
|
||||
BytesStreamOutput bStream = new BytesStreamOutput();
|
||||
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.SMILE, bStream);
|
||||
builder.copyCurrentStructure(parser);
|
||||
builder.close();
|
||||
doc.setSource(bStream.bytes());
|
||||
break;
|
||||
} else {
|
||||
parser.skipChildren();
|
||||
}
|
||||
} else if (token == null) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
throw new ElasticsearchParseException("failed to parse request", e);
|
||||
} finally {
|
||||
context.types(previousTypes);
|
||||
if (parser != null) {
|
||||
parser.close();
|
||||
if (context.aliasFilter() != null) {
|
||||
bq.add(context.aliasFilter(), FILTER);
|
||||
}
|
||||
builder.setPercolateQuery(bq.build());
|
||||
}
|
||||
PercolatorQuery percolatorQuery = builder.build();
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
private void parseSort(XContentParser parser, PercolateContext context) throws Exception {
|
||||
sortParseElement.parse(parser, context);
|
||||
// null, means default sorting by relevancy
|
||||
if (context.sort() == null) {
|
||||
context.doSort = true;
|
||||
if (context.isOnlyCount()) {
|
||||
TotalHitCountCollector collector = new TotalHitCountCollector();
|
||||
context.searcher().search(percolatorQuery, MultiCollector.wrap(collector, aggregatorCollector));
|
||||
if (aggregatorCollector != null) {
|
||||
aggregatorCollector.postCollection();
|
||||
aggregationPhase.execute(context);
|
||||
}
|
||||
return new PercolateShardResponse(new TopDocs(collector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0f), Collections.emptyMap(), Collections.emptyMap(), context);
|
||||
} else {
|
||||
throw new ElasticsearchParseException("Only _score desc is supported");
|
||||
}
|
||||
}
|
||||
|
||||
private ParsedDocument parseFetchedDoc(PercolateContext context, BytesReference fetchedDoc, IndexService documentIndexService, String index, String type) {
|
||||
ParsedDocument doc = null;
|
||||
XContentParser parser = null;
|
||||
try {
|
||||
parser = XContentFactory.xContent(fetchedDoc).createParser(fetchedDoc);
|
||||
MapperService mapperService = documentIndexService.mapperService();
|
||||
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
|
||||
doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(type).flyweight(true));
|
||||
|
||||
if (context.highlight() != null) {
|
||||
doc.setSource(fetchedDoc);
|
||||
int size = context.size();
|
||||
if (size > context.searcher().getIndexReader().maxDoc()) {
|
||||
// prevent easy OOM if more than the total number of docs that exist is requested...
|
||||
size = context.searcher().getIndexReader().maxDoc();
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
throw new ElasticsearchParseException("failed to parse request", e);
|
||||
} finally {
|
||||
if (parser != null) {
|
||||
parser.close();
|
||||
TopScoreDocCollector collector = TopScoreDocCollector.create(size);
|
||||
context.searcher().search(percolatorQuery, MultiCollector.wrap(collector, aggregatorCollector));
|
||||
if (aggregatorCollector != null) {
|
||||
aggregatorCollector.postCollection();
|
||||
aggregationPhase.execute(context);
|
||||
}
|
||||
}
|
||||
|
||||
if (doc == null) {
|
||||
throw new ElasticsearchParseException("No doc to percolate in the request");
|
||||
}
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
Map<Integer, String> ids = new HashMap<>(topDocs.scoreDocs.length);
|
||||
Map<Integer, Map<String, HighlightField>> hls = new HashMap<>(topDocs.scoreDocs.length);
|
||||
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
|
||||
if (context.trackScores() == false) {
|
||||
// No sort or tracking scores was provided, so use special value to indicate to not show the scores:
|
||||
scoreDoc.score = NO_SCORE;
|
||||
}
|
||||
|
||||
return doc;
|
||||
int segmentIdx = ReaderUtil.subIndex(scoreDoc.doc, context.searcher().getIndexReader().leaves());
|
||||
LeafReaderContext atomicReaderContext = context.searcher().getIndexReader().leaves().get(segmentIdx);
|
||||
final int segmentDocId = scoreDoc.doc - atomicReaderContext.docBase;
|
||||
SingleFieldsVisitor fieldsVisitor = new SingleFieldsVisitor(UidFieldMapper.NAME);
|
||||
atomicReaderContext.reader().document(segmentDocId, fieldsVisitor);
|
||||
String id = fieldsVisitor.uid().id();
|
||||
ids.put(scoreDoc.doc, id);
|
||||
if (context.highlight() != null) {
|
||||
Query query = queriesRegistry.getPercolateQueries().get(new BytesRef(id));
|
||||
context.parsedQuery(new ParsedQuery(query));
|
||||
context.hitContext().cache().clear();
|
||||
highlightPhase.hitExecute(context, context.hitContext());
|
||||
hls.put(scoreDoc.doc, context.hitContext().hit().getHighlightFields());
|
||||
}
|
||||
}
|
||||
return new PercolateShardResponse(topDocs, ids, hls, context);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
cache.close();
|
||||
}
|
||||
|
||||
interface PercolatorType {
|
||||
|
||||
// 0x00 is reserved for empty type.
|
||||
byte id();
|
||||
|
||||
ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext);
|
||||
|
||||
PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested);
|
||||
|
||||
}
|
||||
|
||||
private final PercolatorType countPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x01;
|
||||
private InternalAggregations reduceAggregations(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
if (shardResults.get(0).aggregations() == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
long finalCount = 0;
|
||||
for (PercolateShardResponse shardResponse : shardResults) {
|
||||
finalCount += shardResponse.count();
|
||||
}
|
||||
|
||||
assert !shardResults.isEmpty();
|
||||
InternalAggregations reducedAggregations = reduceAggregations(shardResults, headersContext);
|
||||
return new ReduceResult(finalCount, reducedAggregations);
|
||||
List<InternalAggregations> aggregationsList = new ArrayList<>(shardResults.size());
|
||||
for (PercolateShardResponse shardResult : shardResults) {
|
||||
aggregationsList.add(shardResult.aggregations());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
long count = 0;
|
||||
for (Map.Entry<BytesRef, Query> entry : context.percolateQueries().entrySet()) {
|
||||
try {
|
||||
Query existsQuery = entry.getValue();
|
||||
if (isNested) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
if (Lucene.exists(context.docSearcher(), existsQuery)) {
|
||||
count ++;
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
logger.debug("[" + entry.getKey() + "] failed to execute query", e);
|
||||
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
|
||||
InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, new InternalAggregation.ReduceContext(bigArrays, scriptService, headersContext));
|
||||
if (aggregations != null) {
|
||||
List<SiblingPipelineAggregator> pipelineAggregators = shardResults.get(0).pipelineAggregators();
|
||||
if (pipelineAggregators != null) {
|
||||
List<InternalAggregation> newAggs = StreamSupport.stream(aggregations.spliterator(), false).map((p) -> {
|
||||
return (InternalAggregation) p;
|
||||
}).collect(Collectors.toList());
|
||||
for (SiblingPipelineAggregator pipelineAggregator : pipelineAggregators) {
|
||||
InternalAggregation newAgg = pipelineAggregator.doReduce(new InternalAggregations(newAggs), new InternalAggregation.ReduceContext(bigArrays, scriptService, headersContext));
|
||||
newAggs.add(newAgg);
|
||||
}
|
||||
}
|
||||
return new PercolateShardResponse(count, context, request.shardId());
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
private final PercolatorType queryCountPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x02;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
return countPercolator.reduce(shardResults, headersContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
long count = 0;
|
||||
Engine.Searcher percolatorSearcher = context.indexShard().acquireSearcher("percolate");
|
||||
try {
|
||||
Count countCollector = count(logger, context, isNested);
|
||||
queryBasedPercolating(percolatorSearcher, context, countCollector);
|
||||
count = countCollector.counter();
|
||||
} catch (Throwable e) {
|
||||
logger.warn("failed to execute", e);
|
||||
} finally {
|
||||
percolatorSearcher.close();
|
||||
}
|
||||
return new PercolateShardResponse(count, context, request.shardId());
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
private final PercolatorType matchPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x03;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
long foundMatches = 0;
|
||||
int numMatches = 0;
|
||||
for (PercolateShardResponse response : shardResults) {
|
||||
foundMatches += response.count();
|
||||
numMatches += response.matches().length;
|
||||
}
|
||||
int requestedSize = shardResults.get(0).requestedSize();
|
||||
|
||||
// Use a custom impl of AbstractBigArray for Object[]?
|
||||
List<PercolateResponse.Match> finalMatches = new ArrayList<>(requestedSize == 0 ? numMatches : requestedSize);
|
||||
outer:
|
||||
for (PercolateShardResponse response : shardResults) {
|
||||
Text index = new Text(response.getIndex());
|
||||
for (int i = 0; i < response.matches().length; i++) {
|
||||
float score = response.scores().length == 0 ? NO_SCORE : response.scores()[i];
|
||||
Text match = new Text(new BytesArray(response.matches()[i]));
|
||||
Map<String, HighlightField> hl = response.hls().isEmpty() ? null : response.hls().get(i);
|
||||
finalMatches.add(new PercolateResponse.Match(index, match, score, hl));
|
||||
if (requestedSize != 0 && finalMatches.size() == requestedSize) {
|
||||
break outer;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert !shardResults.isEmpty();
|
||||
InternalAggregations reducedAggregations = reduceAggregations(shardResults, headersContext);
|
||||
return new ReduceResult(foundMatches, finalMatches.toArray(new PercolateResponse.Match[finalMatches.size()]), reducedAggregations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
long count = 0;
|
||||
List<BytesRef> matches = new ArrayList<>();
|
||||
List<Map<String, HighlightField>> hls = new ArrayList<>();
|
||||
|
||||
for (Map.Entry<BytesRef, Query> entry : context.percolateQueries().entrySet()) {
|
||||
if (context.highlight() != null) {
|
||||
context.parsedQuery(new ParsedQuery(entry.getValue()));
|
||||
context.hitContext().cache().clear();
|
||||
}
|
||||
try {
|
||||
Query existsQuery = entry.getValue();
|
||||
if (isNested) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
if (Lucene.exists(context.docSearcher(), existsQuery)) {
|
||||
if (!context.limit || count < context.size()) {
|
||||
matches.add(entry.getKey());
|
||||
if (context.highlight() != null) {
|
||||
highlightPhase.hitExecute(context, context.hitContext());
|
||||
hls.add(context.hitContext().hit().getHighlightFields());
|
||||
}
|
||||
}
|
||||
count++;
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
logger.debug("[" + entry.getKey() + "] failed to execute query", e);
|
||||
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
|
||||
}
|
||||
}
|
||||
|
||||
BytesRef[] finalMatches = matches.toArray(new BytesRef[matches.size()]);
|
||||
return new PercolateShardResponse(finalMatches, hls, count, context, request.shardId());
|
||||
}
|
||||
};
|
||||
|
||||
private final PercolatorType queryPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x04;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
return matchPercolator.reduce(shardResults, headersContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
Engine.Searcher percolatorSearcher = context.indexShard().acquireSearcher("percolate");
|
||||
try {
|
||||
Match match = match(logger, context, highlightPhase, isNested);
|
||||
queryBasedPercolating(percolatorSearcher, context, match);
|
||||
List<BytesRef> matches = match.matches();
|
||||
List<Map<String, HighlightField>> hls = match.hls();
|
||||
long count = match.counter();
|
||||
|
||||
BytesRef[] finalMatches = matches.toArray(new BytesRef[matches.size()]);
|
||||
return new PercolateShardResponse(finalMatches, hls, count, context, request.shardId());
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to execute", e);
|
||||
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
|
||||
} finally {
|
||||
percolatorSearcher.close();
|
||||
aggregations = new InternalAggregations(newAggs);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
private final PercolatorType scoringPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x05;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
return matchPercolator.reduce(shardResults, headersContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
Engine.Searcher percolatorSearcher = context.indexShard().acquireSearcher("percolate");
|
||||
try {
|
||||
MatchAndScore matchAndScore = matchAndScore(logger, context, highlightPhase, isNested);
|
||||
queryBasedPercolating(percolatorSearcher, context, matchAndScore);
|
||||
List<BytesRef> matches = matchAndScore.matches();
|
||||
List<Map<String, HighlightField>> hls = matchAndScore.hls();
|
||||
float[] scores = matchAndScore.scores().toArray();
|
||||
long count = matchAndScore.counter();
|
||||
|
||||
BytesRef[] finalMatches = matches.toArray(new BytesRef[matches.size()]);
|
||||
return new PercolateShardResponse(finalMatches, hls, count, scores, context, request.shardId());
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to execute", e);
|
||||
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
|
||||
} finally {
|
||||
percolatorSearcher.close();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
private final PercolatorType topMatchingPercolator = new PercolatorType() {
|
||||
|
||||
@Override
|
||||
public byte id() {
|
||||
return 0x06;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReduceResult reduce(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
long foundMatches = 0;
|
||||
int nonEmptyResponses = 0;
|
||||
int firstNonEmptyIndex = 0;
|
||||
for (int i = 0; i < shardResults.size(); i++) {
|
||||
PercolateShardResponse response = shardResults.get(i);
|
||||
foundMatches += response.count();
|
||||
if (response.matches().length != 0) {
|
||||
if (firstNonEmptyIndex == 0) {
|
||||
firstNonEmptyIndex = i;
|
||||
}
|
||||
nonEmptyResponses++;
|
||||
}
|
||||
}
|
||||
|
||||
int requestedSize = shardResults.get(0).requestedSize();
|
||||
|
||||
// Use a custom impl of AbstractBigArray for Object[]?
|
||||
List<PercolateResponse.Match> finalMatches = new ArrayList<>(requestedSize);
|
||||
if (nonEmptyResponses == 1) {
|
||||
PercolateShardResponse response = shardResults.get(firstNonEmptyIndex);
|
||||
Text index = new Text(response.getIndex());
|
||||
for (int i = 0; i < response.matches().length; i++) {
|
||||
float score = response.scores().length == 0 ? Float.NaN : response.scores()[i];
|
||||
Text match = new Text(new BytesArray(response.matches()[i]));
|
||||
if (!response.hls().isEmpty()) {
|
||||
Map<String, HighlightField> hl = response.hls().get(i);
|
||||
finalMatches.add(new PercolateResponse.Match(index, match, score, hl));
|
||||
} else {
|
||||
finalMatches.add(new PercolateResponse.Match(index, match, score));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
int[] slots = new int[shardResults.size()];
|
||||
while (true) {
|
||||
float lowestScore = Float.NEGATIVE_INFINITY;
|
||||
int requestIndex = -1;
|
||||
int itemIndex = -1;
|
||||
for (int i = 0; i < shardResults.size(); i++) {
|
||||
int scoreIndex = slots[i];
|
||||
float[] scores = shardResults.get(i).scores();
|
||||
if (scoreIndex >= scores.length) {
|
||||
continue;
|
||||
}
|
||||
|
||||
float score = scores[scoreIndex];
|
||||
int cmp = Float.compare(lowestScore, score);
|
||||
// TODO: Maybe add a tie?
|
||||
if (cmp < 0) {
|
||||
requestIndex = i;
|
||||
itemIndex = scoreIndex;
|
||||
lowestScore = score;
|
||||
}
|
||||
}
|
||||
|
||||
// This means the shard matches have been exhausted and we should bail
|
||||
if (requestIndex == -1) {
|
||||
break;
|
||||
}
|
||||
|
||||
slots[requestIndex]++;
|
||||
|
||||
PercolateShardResponse shardResponse = shardResults.get(requestIndex);
|
||||
Text index = new Text(shardResponse.getIndex());
|
||||
Text match = new Text(new BytesArray(shardResponse.matches()[itemIndex]));
|
||||
float score = shardResponse.scores()[itemIndex];
|
||||
if (!shardResponse.hls().isEmpty()) {
|
||||
Map<String, HighlightField> hl = shardResponse.hls().get(itemIndex);
|
||||
finalMatches.add(new PercolateResponse.Match(index, match, score, hl));
|
||||
} else {
|
||||
finalMatches.add(new PercolateResponse.Match(index, match, score));
|
||||
}
|
||||
if (finalMatches.size() == requestedSize) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert !shardResults.isEmpty();
|
||||
InternalAggregations reducedAggregations = reduceAggregations(shardResults, headersContext);
|
||||
return new ReduceResult(foundMatches, finalMatches.toArray(new PercolateResponse.Match[finalMatches.size()]), reducedAggregations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PercolateShardResponse doPercolate(PercolateShardRequest request, PercolateContext context, boolean isNested) {
|
||||
Engine.Searcher percolatorSearcher = context.indexShard().acquireSearcher("percolate");
|
||||
try {
|
||||
MatchAndSort matchAndSort = QueryCollector.matchAndSort(logger, context, isNested);
|
||||
queryBasedPercolating(percolatorSearcher, context, matchAndSort);
|
||||
TopDocs topDocs = matchAndSort.topDocs();
|
||||
long count = topDocs.totalHits;
|
||||
List<BytesRef> matches = new ArrayList<>(topDocs.scoreDocs.length);
|
||||
float[] scores = new float[topDocs.scoreDocs.length];
|
||||
List<Map<String, HighlightField>> hls = null;
|
||||
if (context.highlight() != null) {
|
||||
hls = new ArrayList<>(topDocs.scoreDocs.length);
|
||||
}
|
||||
|
||||
final MappedFieldType uidMapper = context.mapperService().fullName(UidFieldMapper.NAME);
|
||||
final IndexFieldData<?> uidFieldData = context.fieldData().getForField(uidMapper);
|
||||
int i = 0;
|
||||
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
|
||||
int segmentIdx = ReaderUtil.subIndex(scoreDoc.doc, percolatorSearcher.reader().leaves());
|
||||
LeafReaderContext atomicReaderContext = percolatorSearcher.reader().leaves().get(segmentIdx);
|
||||
SortedBinaryDocValues values = uidFieldData.load(atomicReaderContext).getBytesValues();
|
||||
final int localDocId = scoreDoc.doc - atomicReaderContext.docBase;
|
||||
values.setDocument(localDocId);
|
||||
final int numValues = values.count();
|
||||
assert numValues == 1;
|
||||
BytesRef bytes = Uid.splitUidIntoTypeAndId(values.valueAt(0))[1];
|
||||
matches.add(BytesRef.deepCopyOf(bytes));
|
||||
if (hls != null) {
|
||||
Query query = context.percolateQueries().get(bytes);
|
||||
context.parsedQuery(new ParsedQuery(query));
|
||||
context.hitContext().cache().clear();
|
||||
highlightPhase.hitExecute(context, context.hitContext());
|
||||
hls.add(i, context.hitContext().hit().getHighlightFields());
|
||||
}
|
||||
scores[i++] = scoreDoc.score;
|
||||
}
|
||||
if (hls != null) {
|
||||
return new PercolateShardResponse(matches.toArray(new BytesRef[matches.size()]), hls, count, scores, context, request.shardId());
|
||||
} else {
|
||||
return new PercolateShardResponse(matches.toArray(new BytesRef[matches.size()]), count, scores, context, request.shardId());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to execute", e);
|
||||
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
|
||||
} finally {
|
||||
percolatorSearcher.close();
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
private void queryBasedPercolating(Engine.Searcher percolatorSearcher, PercolateContext context, QueryCollector percolateCollector) throws IOException {
|
||||
Query percolatorTypeFilter = context.indexService().mapperService().documentMapper(TYPE_NAME).typeFilter();
|
||||
|
||||
final Query filter;
|
||||
if (context.aliasFilter() != null) {
|
||||
BooleanQuery.Builder booleanFilter = new BooleanQuery.Builder();
|
||||
booleanFilter.add(context.aliasFilter(), BooleanClause.Occur.MUST);
|
||||
booleanFilter.add(percolatorTypeFilter, BooleanClause.Occur.MUST);
|
||||
filter = booleanFilter.build();
|
||||
} else {
|
||||
filter = percolatorTypeFilter;
|
||||
}
|
||||
|
||||
Query query = Queries.filtered(context.percolateQuery(), filter);
|
||||
percolatorSearcher.searcher().search(query, percolateCollector);
|
||||
percolateCollector.aggregatorCollector.postCollection();
|
||||
if (context.aggregations() != null) {
|
||||
aggregationPhase.execute(context);
|
||||
}
|
||||
return aggregations;
|
||||
}
|
||||
|
||||
public final static class ReduceResult {
|
||||
|
@ -849,32 +364,5 @@ public class PercolatorService extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
private InternalAggregations reduceAggregations(List<PercolateShardResponse> shardResults, HasContextAndHeaders headersContext) {
|
||||
if (shardResults.get(0).aggregations() == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
List<InternalAggregations> aggregationsList = new ArrayList<>(shardResults.size());
|
||||
for (PercolateShardResponse shardResult : shardResults) {
|
||||
aggregationsList.add(shardResult.aggregations());
|
||||
}
|
||||
InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, new ReduceContext(bigArrays, scriptService,
|
||||
headersContext));
|
||||
if (aggregations != null) {
|
||||
List<SiblingPipelineAggregator> pipelineAggregators = shardResults.get(0).pipelineAggregators();
|
||||
if (pipelineAggregators != null) {
|
||||
List<InternalAggregation> newAggs = StreamSupport.stream(aggregations.spliterator(), false).map((p) -> {
|
||||
return (InternalAggregation) p;
|
||||
}).collect(Collectors.toList());
|
||||
for (SiblingPipelineAggregator pipelineAggregator : pipelineAggregators) {
|
||||
InternalAggregation newAgg = pipelineAggregator.doReduce(new InternalAggregations(newAggs), new ReduceContext(
|
||||
bigArrays, scriptService, headersContext));
|
||||
newAggs.add(newAgg);
|
||||
}
|
||||
aggregations = new InternalAggregations(newAggs);
|
||||
}
|
||||
}
|
||||
return aggregations;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,403 +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.percolator;
|
||||
|
||||
import com.carrotsearch.hppc.FloatArrayList;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.SimpleCollector;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.TopScoreDocCollector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||
import org.elasticsearch.search.highlight.HighlightField;
|
||||
import org.elasticsearch.search.highlight.HighlightPhase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
abstract class QueryCollector extends SimpleCollector {
|
||||
|
||||
final IndexFieldData<?> uidFieldData;
|
||||
final IndexSearcher searcher;
|
||||
final ConcurrentMap<BytesRef, Query> queries;
|
||||
final ESLogger logger;
|
||||
boolean isNestedDoc = false;
|
||||
|
||||
BytesRef current;
|
||||
|
||||
SortedBinaryDocValues values;
|
||||
|
||||
final BucketCollector aggregatorCollector;
|
||||
LeafCollector aggregatorLeafCollector;
|
||||
|
||||
QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
this.logger = logger;
|
||||
this.queries = context.percolateQueries();
|
||||
this.searcher = context.docSearcher();
|
||||
final MappedFieldType uidMapper = context.mapperService().fullName(UidFieldMapper.NAME);
|
||||
this.uidFieldData = context.fieldData().getForField(uidMapper);
|
||||
this.isNestedDoc = isNestedDoc;
|
||||
|
||||
List<Aggregator> aggregatorCollectors = new ArrayList<>();
|
||||
|
||||
if (context.aggregations() != null) {
|
||||
AggregationContext aggregationContext = new AggregationContext(context);
|
||||
context.aggregations().aggregationContext(aggregationContext);
|
||||
|
||||
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
|
||||
for (int i = 0; i < aggregators.length; i++) {
|
||||
if (!(aggregators[i] instanceof GlobalAggregator)) {
|
||||
Aggregator aggregator = aggregators[i];
|
||||
aggregatorCollectors.add(aggregator);
|
||||
}
|
||||
}
|
||||
context.aggregations().aggregators(aggregators);
|
||||
}
|
||||
aggregatorCollector = BucketCollector.wrap(aggregatorCollectors);
|
||||
aggregatorCollector.preCollection();
|
||||
}
|
||||
|
||||
public void postMatch(int doc) throws IOException {
|
||||
aggregatorLeafCollector.collect(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
aggregatorLeafCollector.setScorer(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return aggregatorCollector.needsScores();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
// we use the UID because id might not be indexed
|
||||
values = uidFieldData.load(context).getBytesValues();
|
||||
aggregatorLeafCollector = aggregatorCollector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
|
||||
return new Match(logger, context, highlightPhase, isNestedDoc);
|
||||
}
|
||||
|
||||
static Count count(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
return new Count(logger, context, isNestedDoc);
|
||||
}
|
||||
|
||||
static MatchAndScore matchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
|
||||
return new MatchAndScore(logger, context, highlightPhase, isNestedDoc);
|
||||
}
|
||||
|
||||
static MatchAndSort matchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
return new MatchAndSort(logger, context, isNestedDoc);
|
||||
}
|
||||
|
||||
|
||||
protected final Query getQuery(int doc) {
|
||||
values.setDocument(doc);
|
||||
final int numValues = values.count();
|
||||
if (numValues == 0) {
|
||||
return null;
|
||||
}
|
||||
assert numValues == 1;
|
||||
current = Uid.splitUidIntoTypeAndId(values.valueAt(0))[1];
|
||||
return queries.get(current);
|
||||
}
|
||||
|
||||
|
||||
|
||||
final static class Match extends QueryCollector {
|
||||
|
||||
final PercolateContext context;
|
||||
final HighlightPhase highlightPhase;
|
||||
|
||||
final List<BytesRef> matches = new ArrayList<>();
|
||||
final List<Map<String, HighlightField>> hls = new ArrayList<>();
|
||||
final boolean limit;
|
||||
final int size;
|
||||
long counter = 0;
|
||||
|
||||
Match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
|
||||
super(logger, context, isNestedDoc);
|
||||
this.limit = context.limit;
|
||||
this.size = context.size();
|
||||
this.context = context;
|
||||
this.highlightPhase = highlightPhase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final Query query = getQuery(doc);
|
||||
if (query == null) {
|
||||
// log???
|
||||
return;
|
||||
}
|
||||
Query existsQuery = query;
|
||||
if (isNestedDoc) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
// run the query
|
||||
try {
|
||||
if (context.highlight() != null) {
|
||||
context.parsedQuery(new ParsedQuery(query));
|
||||
context.hitContext().cache().clear();
|
||||
}
|
||||
|
||||
if (Lucene.exists(searcher, existsQuery)) {
|
||||
if (!limit || counter < size) {
|
||||
matches.add(BytesRef.deepCopyOf(current));
|
||||
if (context.highlight() != null) {
|
||||
highlightPhase.hitExecute(context, context.hitContext());
|
||||
hls.add(context.hitContext().hit().getHighlightFields());
|
||||
}
|
||||
}
|
||||
counter++;
|
||||
postMatch(doc);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.warn("[" + current.utf8ToString() + "] failed to execute query", e);
|
||||
}
|
||||
}
|
||||
|
||||
long counter() {
|
||||
return counter;
|
||||
}
|
||||
|
||||
List<BytesRef> matches() {
|
||||
return matches;
|
||||
}
|
||||
|
||||
List<Map<String, HighlightField>> hls() {
|
||||
return hls;
|
||||
}
|
||||
}
|
||||
|
||||
final static class MatchAndSort extends QueryCollector {
|
||||
|
||||
private final TopScoreDocCollector topDocsCollector;
|
||||
private LeafCollector topDocsLeafCollector;
|
||||
|
||||
MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
super(logger, context, isNestedDoc);
|
||||
// TODO: Use TopFieldCollector.create(...) for ascending and descending scoring?
|
||||
topDocsCollector = TopScoreDocCollector.create(context.size());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return super.needsScores() || topDocsCollector.needsScores();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final Query query = getQuery(doc);
|
||||
if (query == null) {
|
||||
// log???
|
||||
return;
|
||||
}
|
||||
Query existsQuery = query;
|
||||
if (isNestedDoc) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
// run the query
|
||||
try {
|
||||
if (Lucene.exists(searcher, existsQuery)) {
|
||||
topDocsLeafCollector.collect(doc);
|
||||
postMatch(doc);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.warn("[" + current.utf8ToString() + "] failed to execute query", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
topDocsLeafCollector = topDocsCollector.getLeafCollector(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
topDocsLeafCollector.setScorer(scorer);
|
||||
}
|
||||
|
||||
TopDocs topDocs() {
|
||||
return topDocsCollector.topDocs();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
final static class MatchAndScore extends QueryCollector {
|
||||
|
||||
final PercolateContext context;
|
||||
final HighlightPhase highlightPhase;
|
||||
|
||||
final List<BytesRef> matches = new ArrayList<>();
|
||||
final List<Map<String, HighlightField>> hls = new ArrayList<>();
|
||||
// TODO: Use thread local in order to cache the scores lists?
|
||||
final FloatArrayList scores = new FloatArrayList();
|
||||
final boolean limit;
|
||||
final int size;
|
||||
long counter = 0;
|
||||
|
||||
private Scorer scorer;
|
||||
|
||||
MatchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
|
||||
super(logger, context, isNestedDoc);
|
||||
this.limit = context.limit;
|
||||
this.size = context.size();
|
||||
this.context = context;
|
||||
this.highlightPhase = highlightPhase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final Query query = getQuery(doc);
|
||||
if (query == null) {
|
||||
// log???
|
||||
return;
|
||||
}
|
||||
Query existsQuery = query;
|
||||
if (isNestedDoc) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
// run the query
|
||||
try {
|
||||
if (context.highlight() != null) {
|
||||
context.parsedQuery(new ParsedQuery(query));
|
||||
context.hitContext().cache().clear();
|
||||
}
|
||||
if (Lucene.exists(searcher, existsQuery)) {
|
||||
if (!limit || counter < size) {
|
||||
matches.add(BytesRef.deepCopyOf(current));
|
||||
scores.add(scorer.score());
|
||||
if (context.highlight() != null) {
|
||||
highlightPhase.hitExecute(context, context.hitContext());
|
||||
hls.add(context.hitContext().hit().getHighlightFields());
|
||||
}
|
||||
}
|
||||
counter++;
|
||||
postMatch(doc);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.warn("[" + current.utf8ToString() + "] failed to execute query", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
long counter() {
|
||||
return counter;
|
||||
}
|
||||
|
||||
List<BytesRef> matches() {
|
||||
return matches;
|
||||
}
|
||||
|
||||
FloatArrayList scores() {
|
||||
return scores;
|
||||
}
|
||||
|
||||
List<Map<String, HighlightField>> hls() {
|
||||
return hls;
|
||||
}
|
||||
}
|
||||
|
||||
final static class Count extends QueryCollector {
|
||||
|
||||
private long counter = 0;
|
||||
|
||||
Count(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
|
||||
super(logger, context, isNestedDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final Query query = getQuery(doc);
|
||||
if (query == null) {
|
||||
// log???
|
||||
return;
|
||||
}
|
||||
Query existsQuery = query;
|
||||
if (isNestedDoc) {
|
||||
existsQuery = new BooleanQuery.Builder()
|
||||
.add(existsQuery, Occur.MUST)
|
||||
.add(Queries.newNonNestedFilter(), Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
// run the query
|
||||
try {
|
||||
if (Lucene.exists(searcher, existsQuery)) {
|
||||
counter++;
|
||||
postMatch(doc);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.warn("[" + current.utf8ToString() + "] failed to execute query", e);
|
||||
}
|
||||
}
|
||||
|
||||
long counter() {
|
||||
return counter;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -111,7 +111,7 @@ public class CodecTests extends ESTestCase {
|
|||
SimilarityService similarityService = new SimilarityService(settings, Collections.emptyMap());
|
||||
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(settings);
|
||||
MapperRegistry mapperRegistry = new MapperRegistry(Collections.emptyMap(), Collections.emptyMap());
|
||||
MapperService service = new MapperService(settings, analysisService, similarityService, mapperRegistry);
|
||||
MapperService service = new MapperService(settings, analysisService, similarityService, mapperRegistry, () -> null);
|
||||
return new CodecService(service, ESLoggerFactory.getLogger("test"));
|
||||
}
|
||||
|
||||
|
|
|
@ -1928,7 +1928,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
AnalysisService analysisService = new AnalysisService(indexSettings, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
|
||||
SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap());
|
||||
MapperRegistry mapperRegistry = new IndicesModule().getMapperRegistry();
|
||||
MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry);
|
||||
MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, () -> null);
|
||||
DocumentMapper.Builder b = new DocumentMapper.Builder(rootBuilder, mapperService);
|
||||
this.docMapper = b.build(mapperService);
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
|
|||
Collections.singletonMap(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser()));
|
||||
|
||||
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
|
||||
XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject(ExternalMetadataMapper.CONTENT_TYPE)
|
||||
|
@ -101,7 +101,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
|
|||
MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap());
|
||||
|
||||
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
|
||||
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
|
||||
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")
|
||||
|
@ -160,7 +160,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
|
|||
MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap());
|
||||
|
||||
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
|
||||
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
|
||||
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")
|
||||
|
|
|
@ -236,9 +236,9 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
|
|||
IndicesModule indicesModule = new IndicesModule();
|
||||
indicesModule.registerMetadataMapper("_dummy", new DummyMetadataFieldMapper.TypeParser());
|
||||
final MapperRegistry mapperRegistry = indicesModule.getMapperRegistry();
|
||||
MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService,
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string();
|
||||
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
|
||||
ParsedDocument parsedDocument = mapper.parse("index", "type", "id", new BytesArray("{}"));
|
||||
|
|
|
@ -0,0 +1,287 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.queries.TermsQuery;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.BoostQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.hasKey;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.Matchers.sameInstance;
|
||||
|
||||
public class ExtractQueryTermsServiceTests extends ESTestCase {
|
||||
|
||||
public final static String QUERY_TERMS_FIELD = "extracted_terms";
|
||||
public final static String UNKNOWN_QUERY_FIELD = "unknown_query";
|
||||
public static FieldType QUERY_TERMS_FIELD_TYPE = new FieldType();
|
||||
|
||||
static {
|
||||
QUERY_TERMS_FIELD_TYPE.setTokenized(false);
|
||||
QUERY_TERMS_FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
|
||||
QUERY_TERMS_FIELD_TYPE.freeze();
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata() {
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
TermQuery termQuery1 = new TermQuery(new Term("field1", "term1"));
|
||||
bq.add(termQuery1, BooleanClause.Occur.SHOULD);
|
||||
TermQuery termQuery2 = new TermQuery(new Term("field2", "term2"));
|
||||
bq.add(termQuery2, BooleanClause.Occur.SHOULD);
|
||||
|
||||
ParseContext.Document document = new ParseContext.Document();
|
||||
ExtractQueryTermsService.extractQueryTerms(bq.build(), document, QUERY_TERMS_FIELD, UNKNOWN_QUERY_FIELD, QUERY_TERMS_FIELD_TYPE);
|
||||
Collections.sort(document.getFields(), (field1, field2) -> field1.binaryValue().compareTo(field2.binaryValue()));
|
||||
assertThat(document.getFields().size(), equalTo(2));
|
||||
assertThat(document.getFields().get(0).name(), equalTo(QUERY_TERMS_FIELD));
|
||||
assertThat(document.getFields().get(0).binaryValue().utf8ToString(), equalTo("field1\u0000term1"));
|
||||
assertThat(document.getFields().get(1).name(), equalTo(QUERY_TERMS_FIELD));
|
||||
assertThat(document.getFields().get(1).binaryValue().utf8ToString(), equalTo("field2\u0000term2"));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_unsupported() {
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
TermQuery termQuery1 = new TermQuery(new Term("field1", "term1"));
|
||||
bq.add(termQuery1, BooleanClause.Occur.SHOULD);
|
||||
TermQuery termQuery2 = new TermQuery(new Term("field2", "term2"));
|
||||
bq.add(termQuery2, BooleanClause.Occur.SHOULD);
|
||||
|
||||
TermRangeQuery query = new TermRangeQuery("field1", new BytesRef("a"), new BytesRef("z"), true, true);
|
||||
ParseContext.Document document = new ParseContext.Document();
|
||||
ExtractQueryTermsService.extractQueryTerms(query, document, QUERY_TERMS_FIELD, UNKNOWN_QUERY_FIELD, QUERY_TERMS_FIELD_TYPE);
|
||||
assertThat(document.getFields().size(), equalTo(1));
|
||||
assertThat(document.getFields().get(0).name(), equalTo(UNKNOWN_QUERY_FIELD));
|
||||
assertThat(document.getFields().get(0).binaryValue().utf8ToString(), equalTo(""));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_termQuery() {
|
||||
TermQuery termQuery = new TermQuery(new Term("_field", "_term"));
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(termQuery));
|
||||
assertThat(terms.size(), equalTo(1));
|
||||
assertThat(terms.get(0).field(), equalTo(termQuery.getTerm().field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(termQuery.getTerm().bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_phraseQuery() {
|
||||
PhraseQuery phraseQuery = new PhraseQuery("_field", "_term1", "term2");
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(phraseQuery));
|
||||
assertThat(terms.size(), equalTo(1));
|
||||
assertThat(terms.get(0).field(), equalTo(phraseQuery.getTerms()[0].field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(phraseQuery.getTerms()[0].bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_booleanQuery() {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term"));
|
||||
builder.add(termQuery1, BooleanClause.Occur.SHOULD);
|
||||
PhraseQuery phraseQuery = new PhraseQuery("_field", "_term1", "term2");
|
||||
builder.add(phraseQuery, BooleanClause.Occur.SHOULD);
|
||||
|
||||
BooleanQuery.Builder subBuilder = new BooleanQuery.Builder();
|
||||
TermQuery termQuery2 = new TermQuery(new Term("_field1", "_term"));
|
||||
subBuilder.add(termQuery2, BooleanClause.Occur.MUST);
|
||||
TermQuery termQuery3 = new TermQuery(new Term("_field3", "_long_term"));
|
||||
subBuilder.add(termQuery3, BooleanClause.Occur.MUST);
|
||||
builder.add(subBuilder.build(), BooleanClause.Occur.SHOULD);
|
||||
|
||||
BooleanQuery booleanQuery = builder.build();
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(booleanQuery));
|
||||
Collections.sort(terms);
|
||||
assertThat(terms.size(), equalTo(3));
|
||||
assertThat(terms.get(0).field(), equalTo(termQuery1.getTerm().field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(termQuery1.getTerm().bytes()));
|
||||
assertThat(terms.get(1).field(), equalTo(phraseQuery.getTerms()[0].field()));
|
||||
assertThat(terms.get(1).bytes(), equalTo(phraseQuery.getTerms()[0].bytes()));
|
||||
assertThat(terms.get(2).field(), equalTo(termQuery3.getTerm().field()));
|
||||
assertThat(terms.get(2).bytes(), equalTo(termQuery3.getTerm().bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_booleanQuery_onlyShould() {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term1"));
|
||||
builder.add(termQuery1, BooleanClause.Occur.SHOULD);
|
||||
TermQuery termQuery2 = new TermQuery(new Term("_field", "_term2"));
|
||||
builder.add(termQuery2, BooleanClause.Occur.SHOULD);
|
||||
|
||||
BooleanQuery.Builder subBuilder = new BooleanQuery.Builder();
|
||||
TermQuery termQuery3 = new TermQuery(new Term("_field1", "_term"));
|
||||
subBuilder.add(termQuery3, BooleanClause.Occur.SHOULD);
|
||||
TermQuery termQuery4 = new TermQuery(new Term("_field3", "_long_term"));
|
||||
subBuilder.add(termQuery4, BooleanClause.Occur.SHOULD);
|
||||
builder.add(subBuilder.build(), BooleanClause.Occur.SHOULD);
|
||||
|
||||
BooleanQuery booleanQuery = builder.build();
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(booleanQuery));
|
||||
Collections.sort(terms);
|
||||
assertThat(terms.size(), equalTo(4));
|
||||
assertThat(terms.get(0).field(), equalTo(termQuery1.getTerm().field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(termQuery1.getTerm().bytes()));
|
||||
assertThat(terms.get(1).field(), equalTo(termQuery2.getTerm().field()));
|
||||
assertThat(terms.get(1).bytes(), equalTo(termQuery2.getTerm().bytes()));
|
||||
assertThat(terms.get(2).field(), equalTo(termQuery3.getTerm().field()));
|
||||
assertThat(terms.get(2).bytes(), equalTo(termQuery3.getTerm().bytes()));
|
||||
assertThat(terms.get(3).field(), equalTo(termQuery4.getTerm().field()));
|
||||
assertThat(terms.get(3).bytes(), equalTo(termQuery4.getTerm().bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_booleanQueryWithMustNot() {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term"));
|
||||
builder.add(termQuery1, BooleanClause.Occur.MUST_NOT);
|
||||
PhraseQuery phraseQuery = new PhraseQuery("_field", "_term1", "term2");
|
||||
builder.add(phraseQuery, BooleanClause.Occur.SHOULD);
|
||||
|
||||
BooleanQuery booleanQuery = builder.build();
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(booleanQuery));
|
||||
assertThat(terms.size(), equalTo(1));
|
||||
assertThat(terms.get(0).field(), equalTo(phraseQuery.getTerms()[0].field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(phraseQuery.getTerms()[0].bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_constantScoreQuery() {
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term"));
|
||||
ConstantScoreQuery constantScoreQuery = new ConstantScoreQuery(termQuery1);
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(constantScoreQuery));
|
||||
assertThat(terms.size(), equalTo(1));
|
||||
assertThat(terms.get(0).field(), equalTo(termQuery1.getTerm().field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(termQuery1.getTerm().bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_boostQuery() {
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term"));
|
||||
BoostQuery constantScoreQuery = new BoostQuery(termQuery1, 1f);
|
||||
List<Term> terms = new ArrayList<>(ExtractQueryTermsService.extractQueryTerms(constantScoreQuery));
|
||||
assertThat(terms.size(), equalTo(1));
|
||||
assertThat(terms.get(0).field(), equalTo(termQuery1.getTerm().field()));
|
||||
assertThat(terms.get(0).bytes(), equalTo(termQuery1.getTerm().bytes()));
|
||||
}
|
||||
|
||||
public void testExtractQueryMetadata_unsupportedQuery() {
|
||||
TermRangeQuery termRangeQuery = new TermRangeQuery("_field", null, null, true, false);
|
||||
|
||||
try {
|
||||
ExtractQueryTermsService.extractQueryTerms(termRangeQuery);
|
||||
fail("UnsupportedQueryException expected");
|
||||
} catch (ExtractQueryTermsService.UnsupportedQueryException e) {
|
||||
assertThat(e.getUnsupportedQuery(), sameInstance(termRangeQuery));
|
||||
}
|
||||
|
||||
TermQuery termQuery1 = new TermQuery(new Term("_field", "_term"));
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();;
|
||||
builder.add(termQuery1, BooleanClause.Occur.SHOULD);
|
||||
builder.add(termRangeQuery, BooleanClause.Occur.SHOULD);
|
||||
BooleanQuery bq = builder.build();
|
||||
|
||||
try {
|
||||
ExtractQueryTermsService.extractQueryTerms(bq);
|
||||
fail("UnsupportedQueryException expected");
|
||||
} catch (ExtractQueryTermsService.UnsupportedQueryException e) {
|
||||
assertThat(e.getUnsupportedQuery(), sameInstance(termRangeQuery));
|
||||
}
|
||||
}
|
||||
|
||||
public void testCreateQueryMetadataQuery() throws Exception {
|
||||
MemoryIndex memoryIndex = new MemoryIndex(false);
|
||||
memoryIndex.addField("field1", "the quick brown fox jumps over the lazy dog", new WhitespaceAnalyzer());
|
||||
memoryIndex.addField("field2", "some more text", new WhitespaceAnalyzer());
|
||||
memoryIndex.addField("_field3", "unhide me", new WhitespaceAnalyzer());
|
||||
memoryIndex.addField("field4", "123", new WhitespaceAnalyzer());
|
||||
|
||||
IndexReader indexReader = memoryIndex.createSearcher().getIndexReader();
|
||||
Query query = ExtractQueryTermsService.createQueryTermsQuery(indexReader, QUERY_TERMS_FIELD, UNKNOWN_QUERY_FIELD);
|
||||
assertThat(query, instanceOf(TermsQuery.class));
|
||||
|
||||
// no easy way to get to the terms in TermsQuery,
|
||||
// if there a less then 16 terms then it gets rewritten to bq and then we can easily check the terms
|
||||
BooleanQuery booleanQuery = (BooleanQuery) ((ConstantScoreQuery) query.rewrite(indexReader)).getQuery();
|
||||
assertThat(booleanQuery.clauses().size(), equalTo(15));
|
||||
assertClause(booleanQuery, 0, QUERY_TERMS_FIELD, "_field3\u0000me");
|
||||
assertClause(booleanQuery, 1, QUERY_TERMS_FIELD, "_field3\u0000unhide");
|
||||
assertClause(booleanQuery, 2, QUERY_TERMS_FIELD, "field1\u0000brown");
|
||||
assertClause(booleanQuery, 3, QUERY_TERMS_FIELD, "field1\u0000dog");
|
||||
assertClause(booleanQuery, 4, QUERY_TERMS_FIELD, "field1\u0000fox");
|
||||
assertClause(booleanQuery, 5, QUERY_TERMS_FIELD, "field1\u0000jumps");
|
||||
assertClause(booleanQuery, 6, QUERY_TERMS_FIELD, "field1\u0000lazy");
|
||||
assertClause(booleanQuery, 7, QUERY_TERMS_FIELD, "field1\u0000over");
|
||||
assertClause(booleanQuery, 8, QUERY_TERMS_FIELD, "field1\u0000quick");
|
||||
assertClause(booleanQuery, 9, QUERY_TERMS_FIELD, "field1\u0000the");
|
||||
assertClause(booleanQuery, 10, QUERY_TERMS_FIELD, "field2\u0000more");
|
||||
assertClause(booleanQuery, 11, QUERY_TERMS_FIELD, "field2\u0000some");
|
||||
assertClause(booleanQuery, 12, QUERY_TERMS_FIELD, "field2\u0000text");
|
||||
assertClause(booleanQuery, 13, QUERY_TERMS_FIELD, "field4\u0000123");
|
||||
assertClause(booleanQuery, 14, UNKNOWN_QUERY_FIELD, "");
|
||||
}
|
||||
|
||||
public void testSelectTermsListWithHighestSumOfTermLength() {
|
||||
Set<Term> terms1 = new HashSet<>();
|
||||
int shortestTerms1Length = Integer.MAX_VALUE;
|
||||
int sumTermLength = randomIntBetween(1, 128);
|
||||
while (sumTermLength > 0) {
|
||||
int length = randomInt(sumTermLength);
|
||||
shortestTerms1Length = Math.min(shortestTerms1Length, length);
|
||||
terms1.add(new Term("field", randomAsciiOfLength(length)));
|
||||
sumTermLength -= length;
|
||||
}
|
||||
|
||||
Set<Term> terms2 = new HashSet<>();
|
||||
int shortestTerms2Length = Integer.MAX_VALUE;
|
||||
sumTermLength = randomIntBetween(1, 128);
|
||||
while (sumTermLength > 0) {
|
||||
int length = randomInt(sumTermLength);
|
||||
shortestTerms2Length = Math.min(shortestTerms2Length, length);
|
||||
terms2.add(new Term("field", randomAsciiOfLength(length)));
|
||||
sumTermLength -= length;
|
||||
}
|
||||
|
||||
Set<Term> result = ExtractQueryTermsService.selectTermListWithTheLongestShortestTerm(terms1, terms2);
|
||||
Set<Term> expected = shortestTerms1Length >= shortestTerms2Length ? terms1 : terms2;
|
||||
assertThat(result, sameInstance(expected));
|
||||
}
|
||||
|
||||
private void assertClause(BooleanQuery booleanQuery, int i, String expectedField, String expectedValue) {
|
||||
assertThat(booleanQuery.clauses().get(i).getOccur(), equalTo(BooleanClause.Occur.SHOULD));
|
||||
assertThat(((TermQuery) booleanQuery.clauses().get(i).getQuery()).getTerm().field(), equalTo(expectedField));
|
||||
assertThat(((TermQuery) booleanQuery.clauses().get(i).getQuery()).getTerm().bytes().utf8ToString(), equalTo(expectedValue));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.elasticsearch.common.compress.CompressedXContent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.percolator.PercolatorService;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
import org.junit.Before;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class PercolatorFieldMapperTests extends ESSingleNodeTestCase {
|
||||
|
||||
private MapperService mapperService;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
IndexService indexService = createIndex("test", Settings.EMPTY);
|
||||
mapperService = indexService.mapperService();
|
||||
|
||||
String mapper = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties").startObject("field").field("type", "string").endObject().endObject()
|
||||
.endObject().endObject().string();
|
||||
mapperService.merge("type", new CompressedXContent(mapper), true, true);
|
||||
|
||||
String percolatorMapper = XContentFactory.jsonBuilder().startObject().startObject(PercolatorService.TYPE_NAME)
|
||||
.startObject("properties").startObject("query").field("type", "percolator").endObject().endObject()
|
||||
.endObject().endObject().string();
|
||||
mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), true, true);
|
||||
}
|
||||
|
||||
public void testPercolatorFieldMapper() throws Exception {
|
||||
ParsedDocument doc = mapperService.documentMapper(PercolatorService.TYPE_NAME).parse("test", PercolatorService.TYPE_NAME, "1", XContentFactory.jsonBuilder().startObject()
|
||||
.field("query", termQuery("field", "value"))
|
||||
.endObject().bytes());
|
||||
|
||||
assertThat(doc.rootDoc().getFields(PercolatorFieldMapper.EXTRACTED_TERMS_FULL_FIELD_NAME).length, equalTo(1));
|
||||
assertThat(doc.rootDoc().getFields(PercolatorFieldMapper.EXTRACTED_TERMS_FULL_FIELD_NAME)[0].binaryValue().utf8ToString(), equalTo("field\0value"));
|
||||
}
|
||||
|
||||
public void testPercolatorFieldMapper_noQuery() throws Exception {
|
||||
ParsedDocument doc = mapperService.documentMapper(PercolatorService.TYPE_NAME).parse("test", PercolatorService.TYPE_NAME, "1", XContentFactory.jsonBuilder().startObject()
|
||||
.endObject().bytes());
|
||||
assertThat(doc.rootDoc().getFields(PercolatorFieldMapper.EXTRACTED_TERMS_FULL_FIELD_NAME).length, equalTo(0));
|
||||
|
||||
try {
|
||||
mapperService.documentMapper(PercolatorService.TYPE_NAME).parse("test", PercolatorService.TYPE_NAME, "1", XContentFactory.jsonBuilder().startObject()
|
||||
.nullField("query")
|
||||
.endObject().bytes());
|
||||
} catch (MapperParsingException e) {
|
||||
assertThat(e.getDetailedMessage(), containsString("query malformed, must start with start_object"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testAllowNoAdditionalSettings() throws Exception {
|
||||
IndexService indexService = createIndex("test1", Settings.EMPTY);
|
||||
MapperService mapperService = indexService.mapperService();
|
||||
|
||||
String percolatorMapper = XContentFactory.jsonBuilder().startObject().startObject(PercolatorService.TYPE_NAME)
|
||||
.startObject("properties").startObject("query").field("type", "percolator").field("index", "no").endObject().endObject()
|
||||
.endObject().endObject().string();
|
||||
try {
|
||||
mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), true, true);
|
||||
fail("MapperParsingException expected");
|
||||
} catch (MapperParsingException e) {
|
||||
assertThat(e.getMessage(), equalTo("Mapping definition for [query] has unsupported parameters: [index : no]"));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -255,7 +255,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
|
|||
ScriptService scriptService = injector.getInstance(ScriptService.class);
|
||||
SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap());
|
||||
MapperRegistry mapperRegistry = injector.getInstance(MapperRegistry.class);
|
||||
MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry);
|
||||
MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, () -> queryShardContext);
|
||||
indexFieldDataService = new IndexFieldDataService(idxSettings, injector.getInstance(IndicesFieldDataCache.class), injector.getInstance(CircuitBreakerService.class), mapperService);
|
||||
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() {
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.percolator;
|
|||
import org.elasticsearch.action.delete.DeleteResponse;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.percolate.PercolateResponse;
|
||||
import org.elasticsearch.action.percolate.PercolateSourceBuilder;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -187,18 +188,21 @@ public class ConcurrentPercolatorIT extends ESIntegTestCase {
|
|||
case 0:
|
||||
response = client().prepareIndex("index", PercolatorService.TYPE_NAME, id)
|
||||
.setSource(onlyField1)
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
type1.incrementAndGet();
|
||||
break;
|
||||
case 1:
|
||||
response = client().prepareIndex("index", PercolatorService.TYPE_NAME, id)
|
||||
.setSource(onlyField2)
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
type2.incrementAndGet();
|
||||
break;
|
||||
case 2:
|
||||
response = client().prepareIndex("index", PercolatorService.TYPE_NAME, id)
|
||||
.setSource(field1And2)
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
type3.incrementAndGet();
|
||||
break;
|
||||
|
@ -327,6 +331,7 @@ public class ConcurrentPercolatorIT extends ESIntegTestCase {
|
|||
} while (!liveIds.remove(id));
|
||||
|
||||
DeleteResponse response = client().prepareDelete("index", PercolatorService.TYPE_NAME, id)
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
assertThat(response.getId(), equalTo(id));
|
||||
assertThat("doc[" + id + "] should have been deleted, but isn't", response.isFound(), equalTo(true));
|
||||
|
@ -334,6 +339,7 @@ public class ConcurrentPercolatorIT extends ESIntegTestCase {
|
|||
String id = Integer.toString(idGen.getAndIncrement());
|
||||
IndexResponse response = client().prepareIndex("index", PercolatorService.TYPE_NAME, id)
|
||||
.setSource(doc)
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
liveIds.add(id);
|
||||
assertThat(response.isCreated(), equalTo(true)); // We only add new docs
|
||||
|
@ -357,9 +363,9 @@ public class ConcurrentPercolatorIT extends ESIntegTestCase {
|
|||
indexThreads[i].start();
|
||||
}
|
||||
|
||||
XContentBuilder percolateDoc = XContentFactory.jsonBuilder().startObject().startObject("doc")
|
||||
String percolateDoc = XContentFactory.jsonBuilder().startObject()
|
||||
.field("field1", "value")
|
||||
.endObject().endObject();
|
||||
.endObject().string();
|
||||
for (int counter = 0; counter < numberPercolateOperation; counter++) {
|
||||
Thread.sleep(5);
|
||||
semaphore.acquire(numIndexThreads);
|
||||
|
@ -369,7 +375,9 @@ public class ConcurrentPercolatorIT extends ESIntegTestCase {
|
|||
}
|
||||
int atLeastExpected = liveIds.size();
|
||||
PercolateResponse response = client().preparePercolate().setIndices("index").setDocumentType("type")
|
||||
.setSource(percolateDoc).execute().actionGet();
|
||||
.setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(percolateDoc))
|
||||
.setSize(atLeastExpected)
|
||||
.get();
|
||||
assertThat(response.getShardFailures(), emptyArray());
|
||||
assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards()));
|
||||
assertThat(response.getMatches().length, equalTo(atLeastExpected));
|
||||
|
|
|
@ -73,6 +73,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
MultiPercolateResponse response = client().prepareMultiPercolate()
|
||||
.add(client().preparePercolate()
|
||||
|
@ -146,6 +147,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
.setRouting("a")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
MultiPercolateResponse response = client().prepareMultiPercolate()
|
||||
.add(client().preparePercolate()
|
||||
|
@ -214,6 +216,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", "type", "1")
|
||||
.setSource(jsonBuilder().startObject().field("field", "a"))
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
MultiPercolateRequestBuilder builder = client().prepareMultiPercolate();
|
||||
int numPercolateRequest = randomIntBetween(50, 100);
|
||||
|
@ -221,7 +224,9 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setGetRequest(Requests.getRequest("test").type("type").id("1"))
|
||||
.setIndices("test").setDocumentType("type"));
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setSize(numQueries)
|
||||
);
|
||||
}
|
||||
|
||||
MultiPercolateResponse response = builder.execute().actionGet();
|
||||
|
@ -238,7 +243,8 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setGetRequest(Requests.getRequest("test").type("type").id("2"))
|
||||
.setIndices("test").setDocumentType("type"));
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries)
|
||||
);
|
||||
}
|
||||
|
||||
response = builder.execute().actionGet();
|
||||
|
@ -255,12 +261,14 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setGetRequest(Requests.getRequest("test").type("type").id("2"))
|
||||
.setIndices("test").setDocumentType("type"));
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries)
|
||||
);
|
||||
}
|
||||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setGetRequest(Requests.getRequest("test").type("type").id("1"))
|
||||
.setIndices("test").setDocumentType("type"));
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries)
|
||||
);
|
||||
|
||||
response = builder.execute().actionGet();
|
||||
assertThat(response.items().length, equalTo(numPercolateRequest + 1));
|
||||
|
@ -282,6 +290,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
}
|
||||
refresh();
|
||||
|
||||
MultiPercolateRequestBuilder builder = client().prepareMultiPercolate();
|
||||
int numPercolateRequest = randomIntBetween(50, 100);
|
||||
|
@ -289,6 +298,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setSize(numQueries)
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field", "a").endObject())));
|
||||
}
|
||||
|
||||
|
@ -331,6 +341,7 @@ public class MultiPercolatorIT extends ESIntegTestCase {
|
|||
}
|
||||
builder.add(
|
||||
client().preparePercolate()
|
||||
.setSize(numQueries)
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field", "a").endObject())));
|
||||
|
||||
|
|
|
@ -0,0 +1,196 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.percolate.PercolateShardRequest;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.analysis.AnalyzerProvider;
|
||||
import org.elasticsearch.index.analysis.CharFilterFactory;
|
||||
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
||||
import org.elasticsearch.index.analysis.TokenizerFactory;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.query.QueryParser;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.query.TermQueryParser;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.IndicesModule;
|
||||
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.AggregationBinaryParseElement;
|
||||
import org.elasticsearch.search.aggregations.AggregationParseElement;
|
||||
import org.elasticsearch.search.aggregations.AggregationPhase;
|
||||
import org.elasticsearch.search.aggregations.AggregatorParsers;
|
||||
import org.elasticsearch.search.highlight.HighlightPhase;
|
||||
import org.elasticsearch.search.highlight.Highlighters;
|
||||
import org.elasticsearch.search.sort.SortParseElement;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.Before;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class PercolateDocumentParserTests extends ESTestCase {
|
||||
|
||||
private Index index;
|
||||
private MapperService mapperService;
|
||||
private PercolateDocumentParser parser;
|
||||
private QueryShardContext queryShardContext;
|
||||
|
||||
@Before
|
||||
public void init() {
|
||||
index = new Index("_index");
|
||||
IndexSettings indexSettings = new IndexSettings(new IndexMetaData.Builder("_index").settings(
|
||||
Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.build(),
|
||||
Settings.EMPTY, Collections.emptyList()
|
||||
);
|
||||
AnalysisService analysisService = new AnalysisService(indexSettings, Collections.<String, AnalyzerProvider>emptyMap(), Collections.<String, TokenizerFactory>emptyMap(), Collections.<String, CharFilterFactory>emptyMap(), Collections.<String, TokenFilterFactory>emptyMap());
|
||||
IndicesModule indicesModule = new IndicesModule();
|
||||
mapperService = new MapperService(indexSettings, analysisService, new SimilarityService(indexSettings, Collections.emptyMap()), indicesModule.getMapperRegistry(), () -> null);
|
||||
|
||||
Set<QueryParser> parsers = Collections.singleton(new TermQueryParser());
|
||||
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(indexSettings.getSettings(), parsers, new NamedWriteableRegistry());
|
||||
|
||||
queryShardContext = new QueryShardContext(indexSettings, null, null, null, mapperService, null, null, indicesQueriesRegistry);
|
||||
|
||||
HighlightPhase highlightPhase = new HighlightPhase(Settings.EMPTY, new Highlighters());
|
||||
AggregatorParsers aggregatorParsers = new AggregatorParsers(Collections.emptySet(), Collections.emptySet());
|
||||
AggregationPhase aggregationPhase = new AggregationPhase(new AggregationParseElement(aggregatorParsers), new AggregationBinaryParseElement(aggregatorParsers));
|
||||
MappingUpdatedAction mappingUpdatedAction = Mockito.mock(MappingUpdatedAction.class);
|
||||
parser = new PercolateDocumentParser(
|
||||
highlightPhase, new SortParseElement(), aggregationPhase, mappingUpdatedAction
|
||||
);
|
||||
}
|
||||
|
||||
public void testParseDoc() throws Exception {
|
||||
XContentBuilder source = jsonBuilder().startObject()
|
||||
.startObject("doc")
|
||||
.field("field1", "value1")
|
||||
.endObject()
|
||||
.endObject();
|
||||
PercolateShardRequest request = new PercolateShardRequest(new ShardId(index, 0), null);
|
||||
request.documentType("type");
|
||||
request.source(source.bytes());
|
||||
|
||||
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", "_index", 0), mapperService);
|
||||
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
|
||||
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
|
||||
}
|
||||
|
||||
public void testParseDocAndOtherOptions() throws Exception {
|
||||
XContentBuilder source = jsonBuilder().startObject()
|
||||
.startObject("doc")
|
||||
.field("field1", "value1")
|
||||
.endObject()
|
||||
.startObject("query")
|
||||
.startObject("term").field("field1", "value1").endObject()
|
||||
.endObject()
|
||||
.field("track_scores", true)
|
||||
.field("size", 123)
|
||||
.startObject("sort").startObject("_score").endObject().endObject()
|
||||
.endObject();
|
||||
PercolateShardRequest request = new PercolateShardRequest(new ShardId(index, 0), null);
|
||||
request.documentType("type");
|
||||
request.source(source.bytes());
|
||||
|
||||
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", "_index", 0), mapperService);
|
||||
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
|
||||
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
|
||||
assertThat(context.percolateQuery(), equalTo(new TermQuery(new Term("field1", "value1"))));
|
||||
assertThat(context.trackScores(), is(true));
|
||||
assertThat(context.size(), is(123));
|
||||
assertThat(context.sort(), nullValue());
|
||||
}
|
||||
|
||||
public void testParseDocSource() throws Exception {
|
||||
XContentBuilder source = jsonBuilder().startObject()
|
||||
.startObject("query")
|
||||
.startObject("term").field("field1", "value1").endObject()
|
||||
.endObject()
|
||||
.field("track_scores", true)
|
||||
.field("size", 123)
|
||||
.startObject("sort").startObject("_score").endObject().endObject()
|
||||
.endObject();
|
||||
XContentBuilder docSource = jsonBuilder().startObject()
|
||||
.field("field1", "value1")
|
||||
.endObject();
|
||||
PercolateShardRequest request = new PercolateShardRequest(new ShardId(index, 0), null);
|
||||
request.documentType("type");
|
||||
request.source(source.bytes());
|
||||
request.docSource(docSource.bytes());
|
||||
|
||||
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", "_index", 0), mapperService);
|
||||
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
|
||||
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
|
||||
assertThat(context.percolateQuery(), equalTo(new TermQuery(new Term("field1", "value1"))));
|
||||
assertThat(context.trackScores(), is(true));
|
||||
assertThat(context.size(), is(123));
|
||||
assertThat(context.sort(), nullValue());
|
||||
}
|
||||
|
||||
public void testParseDocSourceAndSource() throws Exception {
|
||||
XContentBuilder source = jsonBuilder().startObject()
|
||||
.startObject("doc")
|
||||
.field("field1", "value1")
|
||||
.endObject()
|
||||
.startObject("query")
|
||||
.startObject("term").field("field1", "value1").endObject()
|
||||
.endObject()
|
||||
.field("track_scores", true)
|
||||
.field("size", 123)
|
||||
.startObject("sort").startObject("_score").endObject().endObject()
|
||||
.endObject();
|
||||
XContentBuilder docSource = jsonBuilder().startObject()
|
||||
.field("field1", "value1")
|
||||
.endObject();
|
||||
PercolateShardRequest request = new PercolateShardRequest(new ShardId(index, 0), null);
|
||||
request.documentType("type");
|
||||
request.source(source.bytes());
|
||||
request.docSource(docSource.bytes());
|
||||
|
||||
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", "_index", 0), mapperService);
|
||||
try {
|
||||
parser.parse(request, context, mapperService, queryShardContext);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertThat(e.getMessage(), equalTo("Can't specify the document to percolate in the source of the request and as document id"));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -49,9 +49,10 @@ import static org.hamcrest.Matchers.notNullValue;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class PercolatorFacetsAndAggregationsIT extends ESIntegTestCase {
|
||||
public class PercolatorAggregationsIT extends ESIntegTestCase {
|
||||
|
||||
// Just test the integration with facets and aggregations, not the facet and aggregation functionality!
|
||||
public void testFacetsAndAggregations() throws Exception {
|
||||
public void testAggregations() throws Exception {
|
||||
assertAcked(prepareCreate("test").addMapping("type", "field1", "type=string", "field2", "type=string"));
|
||||
ensureGreen();
|
||||
|
||||
|
@ -72,12 +73,15 @@ public class PercolatorFacetsAndAggregationsIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute()
|
||||
.actionGet();
|
||||
}
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
for (int i = 0; i < numQueries; i++) {
|
||||
String value = values[i % numUniqueQueries];
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate()
|
||||
.setIndices("test")
|
||||
.setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()))
|
||||
.setSize(expectedCount[i % numUniqueQueries]);
|
||||
|
||||
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
|
||||
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode));
|
||||
|
@ -134,12 +138,15 @@ public class PercolatorFacetsAndAggregationsIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute()
|
||||
.actionGet();
|
||||
}
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
for (int i = 0; i < numQueries; i++) {
|
||||
String value = values[i % numUniqueQueries];
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate()
|
||||
.setIndices("test")
|
||||
.setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()))
|
||||
.setSize(expectedCount[i % numUniqueQueries]);
|
||||
|
||||
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
|
||||
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode));
|
||||
|
@ -210,12 +217,15 @@ public class PercolatorFacetsAndAggregationsIT extends ESIntegTestCase {
|
|||
.execute()
|
||||
.actionGet();
|
||||
}
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
for (int i = 0; i < numQueries; i++) {
|
||||
String value = "value0";
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
|
||||
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate()
|
||||
.setIndices("test")
|
||||
.setDocumentType("type")
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()))
|
||||
.setSize(numQueries);
|
||||
|
||||
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
|
||||
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("terms").field("field2").collectMode(aggCollectionMode)
|
|
@ -18,54 +18,104 @@
|
|||
*/
|
||||
package org.elasticsearch.percolator;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.percolate.PercolateResponse;
|
||||
import org.elasticsearch.action.percolate.PercolateSourceBuilder;
|
||||
import org.elasticsearch.index.percolator.PercolatorException;
|
||||
import org.elasticsearch.index.query.QueryShardException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.MappingMetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertMatchCount;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0)
|
||||
@LuceneTestCase.SuppressFileSystems("ExtrasFS")
|
||||
public class PercolatorBackwardsCompatibilityIT extends ESIntegTestCase {
|
||||
public void testPercolatorUpgrading() throws Exception {
|
||||
// Simulates an index created on an node before 1.4.0 where the field resolution isn't strict.
|
||||
assertAcked(prepareCreate("test")
|
||||
.setSettings(settings(Version.V_1_3_0).put(indexSettings())));
|
||||
ensureGreen();
|
||||
int numDocs = randomIntBetween(100, 150);
|
||||
IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs];
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
docs[i] = client().prepareIndex("test", PercolatorService.TYPE_NAME)
|
||||
.setSource(jsonBuilder().startObject().field("query", termQuery("field1", "value")).endObject());
|
||||
}
|
||||
indexRandom(true, docs);
|
||||
PercolateResponse response = client().preparePercolate().setIndices("test").setDocumentType("type")
|
||||
.setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc("field1", "value"))
|
||||
.get();
|
||||
assertMatchCount(response, numDocs);
|
||||
|
||||
// After upgrade indices, indices created before the upgrade allow that queries refer to fields not available in mapping
|
||||
client().prepareIndex("test", PercolatorService.TYPE_NAME)
|
||||
.setSource(jsonBuilder().startObject().field("query", termQuery("field2", "value")).endObject()).get();
|
||||
private final static String INDEX_NAME = "percolator_index";
|
||||
|
||||
// However on new indices, the field resolution is strict, no queries with unmapped fields are allowed
|
||||
createIndex("test2");
|
||||
try {
|
||||
client().prepareIndex("test2", PercolatorService.TYPE_NAME)
|
||||
.setSource(jsonBuilder().startObject().field("query", termQuery("field1", "value")).endObject()).get();
|
||||
fail();
|
||||
} catch (PercolatorException e) {
|
||||
e.printStackTrace();
|
||||
assertThat(e.getRootCause(), instanceOf(QueryShardException.class));
|
||||
public void testOldPercolatorIndex() throws Exception {
|
||||
setupNode();
|
||||
|
||||
// verify cluster state:
|
||||
ClusterState state = client().admin().cluster().prepareState().get().getState();
|
||||
assertThat(state.metaData().indices().size(), equalTo(1));
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME), notNullValue());
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME).getCreationVersion(), equalTo(Version.V_2_0_0));
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME).getUpgradedVersion(), equalTo(Version.CURRENT));
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME).getMappings().size(), equalTo(2));
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME).getMappings().get(".percolator"), notNullValue());
|
||||
// important: verify that the query field in the .percolator mapping is of type object (from 3.0.0 this is of type percolator)
|
||||
MappingMetaData mappingMetaData = state.metaData().indices().get(INDEX_NAME).getMappings().get(".percolator");
|
||||
assertThat(XContentMapValues.extractValue("properties.query.type", mappingMetaData.sourceAsMap()), equalTo("object"));
|
||||
assertThat(state.metaData().indices().get(INDEX_NAME).getMappings().get("message"), notNullValue());
|
||||
|
||||
// verify existing percolator queries:
|
||||
SearchResponse searchResponse = client().prepareSearch(INDEX_NAME)
|
||||
.setTypes(".percolator")
|
||||
.addSort("_id", SortOrder.ASC)
|
||||
.get();
|
||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1"));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), equalTo("2"));
|
||||
assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3"));
|
||||
|
||||
// verify percolate response
|
||||
PercolateResponse percolateResponse = client().preparePercolate()
|
||||
.setIndices(INDEX_NAME)
|
||||
.setDocumentType("message")
|
||||
.setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc("message", "the quick brown fox jumps over the lazy dog"))
|
||||
.get();
|
||||
|
||||
assertThat(percolateResponse.getCount(), equalTo(2L));
|
||||
assertThat(percolateResponse.getMatches().length, equalTo(2));
|
||||
assertThat(percolateResponse.getMatches()[0].getId().string(), equalTo("1"));
|
||||
assertThat(percolateResponse.getMatches()[1].getId().string(), equalTo("2"));
|
||||
|
||||
// add an extra query and verify the results
|
||||
client().prepareIndex(INDEX_NAME, ".percolator", "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchQuery("message", "fox jumps")).endObject())
|
||||
.get();
|
||||
refresh();
|
||||
|
||||
percolateResponse = client().preparePercolate()
|
||||
.setIndices(INDEX_NAME)
|
||||
.setDocumentType("message")
|
||||
.setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc("message", "the quick brown fox jumps over the lazy dog"))
|
||||
.get();
|
||||
|
||||
assertThat(percolateResponse.getCount(), equalTo(3L));
|
||||
assertThat(percolateResponse.getMatches().length, equalTo(3));
|
||||
assertThat(percolateResponse.getMatches()[0].getId().string(), equalTo("1"));
|
||||
assertThat(percolateResponse.getMatches()[1].getId().string(), equalTo("2"));
|
||||
assertThat(percolateResponse.getMatches()[2].getId().string(), equalTo("4"));
|
||||
}
|
||||
|
||||
private void setupNode() throws Exception {
|
||||
Path dataDir = createTempDir();
|
||||
Path clusterDir = Files.createDirectory(dataDir.resolve(cluster().getClusterName()));
|
||||
try (InputStream stream = PercolatorBackwardsCompatibilityIT.class.getResourceAsStream("/indices/percolator/bwc_index_2.0.0.zip")) {
|
||||
TestUtil.unzip(stream, clusterDir);
|
||||
}
|
||||
|
||||
Settings.Builder nodeSettings = Settings.builder()
|
||||
.put("path.data", dataDir);
|
||||
internalCluster().startNode(nodeSettings.build());
|
||||
ensureGreen(INDEX_NAME);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.elasticsearch.percolator;
|
||||
|
||||
import org.apache.lucene.search.join.ScoreMode;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||
import org.elasticsearch.action.admin.indices.alias.Alias;
|
||||
|
@ -39,7 +38,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.engine.DocumentMissingException;
|
||||
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
||||
import org.elasticsearch.index.percolator.PercolatorException;
|
||||
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||
import org.elasticsearch.index.query.Operator;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.index.query.QueryShardException;
|
||||
|
@ -102,6 +101,7 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
*
|
||||
*/
|
||||
public class PercolatorIT extends ESIntegTestCase {
|
||||
|
||||
public void testSimple1() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").execute().actionGet();
|
||||
ensureGreen();
|
||||
|
@ -125,7 +125,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc with field1=b");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -200,6 +200,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "test1")
|
||||
.setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field2", "value")).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type1")
|
||||
|
@ -212,6 +213,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "test2")
|
||||
.setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field1", 1)).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type1")
|
||||
|
@ -223,6 +225,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
|
||||
|
||||
client().prepareDelete("test", PercolatorService.TYPE_NAME, "test2").execute().actionGet();
|
||||
refresh();
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type1")
|
||||
.setSource(doc).execute().actionGet();
|
||||
|
@ -244,11 +247,13 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setRouting(Integer.toString(i % 2))
|
||||
.execute().actionGet();
|
||||
}
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc with no routing");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject())
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()))
|
||||
.setSize(100)
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, 100l);
|
||||
assertThat(response.getMatches(), arrayWithSize(100));
|
||||
|
@ -256,23 +261,25 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
logger.info("--> Percolate doc with routing=0");
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject())
|
||||
.setRouting("0")
|
||||
.execute().actionGet();
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()))
|
||||
.setSize(100)
|
||||
.setRouting("0")
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, 50l);
|
||||
assertThat(response.getMatches(), arrayWithSize(50));
|
||||
|
||||
logger.info("--> Percolate doc with routing=1");
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
.setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject())
|
||||
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()))
|
||||
.setSize(100)
|
||||
.setRouting("1")
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, 50l);
|
||||
assertThat(response.getMatches(), arrayWithSize(50));
|
||||
}
|
||||
|
||||
public void testStorePeroclateQueriesOnRecreatedIndex() throws Exception {
|
||||
public void storePercolateQueriesOnRecreatedIndex() throws Exception {
|
||||
createIndex("test");
|
||||
ensureGreen();
|
||||
|
||||
|
@ -326,6 +333,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.endObject())
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
PercolateResponse percolate = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("doc")
|
||||
|
@ -352,7 +360,6 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.field("query", termQuery("field1", "value1"))
|
||||
.endObject())
|
||||
.execute().actionGet();
|
||||
|
||||
refresh();
|
||||
SearchResponse countResponse = client().prepareSearch().setSize(0)
|
||||
.setQuery(matchAllQuery()).setTypes(PercolatorService.TYPE_NAME)
|
||||
|
@ -511,7 +518,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "1")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> First percolate request");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -613,7 +620,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate existing doc with id 1");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -686,7 +693,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate existing doc with id 1");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -751,7 +758,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate existing doc with id 2 and version 1");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -796,6 +803,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
}
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc to index test1");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -962,7 +970,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Count percolate doc with field1=b");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -1033,7 +1041,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "4")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Count percolate existing doc with id 1");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -1083,12 +1091,14 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.execute().actionGet();
|
||||
}
|
||||
}
|
||||
refresh();
|
||||
|
||||
boolean onlyCount = randomBoolean();
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
.setIndices("my-index").setDocumentType("my-type")
|
||||
.setOnlyCount(onlyCount)
|
||||
.setPercolateDoc(docBuilder().setDoc("field", "value"))
|
||||
.setSize((int) totalQueries)
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, totalQueries);
|
||||
if (!onlyCount) {
|
||||
|
@ -1108,7 +1118,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
// The query / filter capabilities are NOT in realtime
|
||||
client().admin().indices().prepareRefresh("my-index").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
int runs = randomIntBetween(3, 16);
|
||||
for (int i = 0; i < runs; i++) {
|
||||
|
@ -1118,6 +1128,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setOnlyCount(onlyCount)
|
||||
.setPercolateDoc(docBuilder().setDoc("field", "value"))
|
||||
.setPercolateQuery(termQuery("level", 1 + randomInt(numLevels - 1)))
|
||||
.setSize((int) numQueriesPerLevel)
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, numQueriesPerLevel);
|
||||
if (!onlyCount) {
|
||||
|
@ -1132,6 +1143,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setOnlyCount(onlyCount)
|
||||
.setPercolateDoc(docBuilder().setDoc("field", "value"))
|
||||
.setPercolateQuery(termQuery("level", 1 + randomInt(numLevels - 1)))
|
||||
.setSize((int) numQueriesPerLevel)
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, numQueriesPerLevel);
|
||||
if (!onlyCount) {
|
||||
|
@ -1268,18 +1280,6 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
assertThat(response.getMatches()[0].getScore(), equalTo(2f));
|
||||
assertThat(response.getMatches()[1].getId().string(), equalTo("1"));
|
||||
assertThat(response.getMatches()[1].getScore(), equalTo(1f));
|
||||
|
||||
response = client().preparePercolate().setIndices("my-index").setDocumentType("my-type")
|
||||
.setSortByScore(true)
|
||||
.setPercolateDoc(docBuilder().setDoc("field", "value"))
|
||||
.setPercolateQuery(QueryBuilders.functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("level")))
|
||||
.execute().actionGet();
|
||||
assertThat(response.getCount(), equalTo(0l));
|
||||
assertThat(response.getShardFailures().length, greaterThan(0));
|
||||
for (ShardOperationFailedException failure : response.getShardFailures()) {
|
||||
assertThat(failure.status(), equalTo(RestStatus.BAD_REQUEST));
|
||||
assertThat(failure.reason(), containsString("Can't sort if size isn't specified"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testPercolateSortingUnsupportedField() throws Exception {
|
||||
|
@ -1322,25 +1322,6 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
assertMatchCount(response, 0l);
|
||||
}
|
||||
|
||||
public void testPercolateNotEmptyIndexButNoRefresh() throws Exception {
|
||||
client().admin().indices().prepareCreate("my-index")
|
||||
.setSettings(settingsBuilder().put("index.refresh_interval", -1))
|
||||
.execute().actionGet();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("my-index", PercolatorService.TYPE_NAME, "1")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).field("level", 1).endObject())
|
||||
.execute().actionGet();
|
||||
|
||||
PercolateResponse response = client().preparePercolate().setIndices("my-index").setDocumentType("my-type")
|
||||
.setSortByScore(true)
|
||||
.setSize(2)
|
||||
.setPercolateDoc(docBuilder().setDoc("field", "value"))
|
||||
.setPercolateQuery(QueryBuilders.functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("level")))
|
||||
.execute().actionGet();
|
||||
assertMatchCount(response, 0l);
|
||||
}
|
||||
|
||||
public void testPercolatorWithHighlighting() throws Exception {
|
||||
StringBuilder fieldMapping = new StringBuilder("type=string")
|
||||
.append(",store=").append(randomBoolean());
|
||||
|
@ -1367,6 +1348,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "5")
|
||||
.setSource(jsonBuilder().startObject().field("query", termQuery("field1", "fox")).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc with field1=The quick brown fox jumps over the lazy dog");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -1393,9 +1375,6 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
assertThat(matches[3].getHighlightFields().get("field1").fragments()[0].string(), equalTo("The quick brown fox jumps over the lazy <em>dog</em>"));
|
||||
assertThat(matches[4].getHighlightFields().get("field1").fragments()[0].string(), equalTo("The quick brown <em>fox</em> jumps over the lazy dog"));
|
||||
|
||||
// Anything with percolate query isn't realtime
|
||||
client().admin().indices().prepareRefresh("test").execute().actionGet();
|
||||
|
||||
logger.info("--> Query percolate doc with field1=The quick brown fox jumps over the lazy dog");
|
||||
response = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
|
@ -1522,6 +1501,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", "type", "1")
|
||||
.setSource(jsonBuilder().startObject().field("field1", "The quick brown fox jumps over the lazy dog").endObject())
|
||||
.get();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Top percolate for doc with field1=The quick brown fox jumps over the lazy dog");
|
||||
response = client().preparePercolate()
|
||||
|
@ -1569,6 +1549,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.endObject())
|
||||
.setRefresh(true)
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
PercolateResponse percolate = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("doc")
|
||||
|
@ -1638,8 +1619,9 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", QueryBuilders.queryStringQuery("color:red")).endObject())
|
||||
.get();
|
||||
fail();
|
||||
} catch (PercolatorException e) {
|
||||
} catch (MapperParsingException e) {
|
||||
}
|
||||
refresh();
|
||||
|
||||
PercolateResponse percolateResponse = client().preparePercolate().setDocumentType("type")
|
||||
.setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(jsonBuilder().startObject().startObject("custom").field("color", "blue").endObject().endObject()))
|
||||
|
@ -1656,6 +1638,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("idx", PercolatorService.TYPE_NAME, "2")
|
||||
.setSource(jsonBuilder().startObject().field("query", QueryBuilders.queryStringQuery("custom.color:blue")).field("type", "type").endObject())
|
||||
.get();
|
||||
refresh();
|
||||
|
||||
// The second request will yield a match, since the query during the proper field during parsing.
|
||||
percolateResponse = client().preparePercolate().setDocumentType("type")
|
||||
|
@ -1725,7 +1708,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", termQuery("field1", "value")).endObject())
|
||||
.get();
|
||||
fail();
|
||||
} catch (PercolatorException e) {
|
||||
} catch (MapperParsingException e) {
|
||||
assertThat(e.getRootCause(), instanceOf(QueryShardException.class));
|
||||
}
|
||||
|
||||
|
@ -1734,7 +1717,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", rangeQuery("field1").from(0).to(1)).endObject())
|
||||
.get();
|
||||
fail();
|
||||
} catch (PercolatorException e) {
|
||||
} catch (MapperParsingException e) {
|
||||
assertThat(e.getRootCause(), instanceOf(QueryShardException.class));
|
||||
}
|
||||
}
|
||||
|
@ -1751,6 +1734,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("test", PercolatorService.TYPE_NAME, "2")
|
||||
.setSource(jsonBuilder().startObject().field("query", constantScoreQuery(rangeQuery("timestamp").from("now-1d").to("now"))).endObject())
|
||||
.get();
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc with field1=b");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -1797,7 +1781,6 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
return doc;
|
||||
}
|
||||
|
||||
// issue
|
||||
public void testNestedDocFilter() throws IOException {
|
||||
String mapping = "{\n" +
|
||||
" \"doc\": {\n" +
|
||||
|
@ -1943,6 +1926,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
.setSettings(settings));
|
||||
client().prepareIndex("test", PercolatorService.TYPE_NAME)
|
||||
.setSource(jsonBuilder().startObject().field("query", matchQuery("field1", "value")).endObject()).get();
|
||||
refresh();
|
||||
logger.info("--> Percolate doc with field1=value");
|
||||
PercolateResponse response1 = client().preparePercolate()
|
||||
.setIndices("test").setDocumentType("type")
|
||||
|
@ -1994,6 +1978,7 @@ public class PercolatorIT extends ESIntegTestCase {
|
|||
client().prepareIndex("index", PercolatorService.TYPE_NAME, "1")
|
||||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.execute().actionGet();
|
||||
refresh();
|
||||
|
||||
// Just percolating a document that has a _parent field in its mapping should just work:
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
|
|
@ -0,0 +1,258 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.percolator.ExtractQueryTermsService;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class PercolatorQueryTests extends ESTestCase {
|
||||
|
||||
public final static String EXTRACTED_TERMS_FIELD_NAME = "extracted_terms";
|
||||
public final static String UNKNOWN_QUERY_FIELD_NAME = "unknown_query";
|
||||
public static FieldType EXTRACTED_TERMS_FIELD_TYPE = new FieldType();
|
||||
|
||||
static {
|
||||
EXTRACTED_TERMS_FIELD_TYPE.setTokenized(false);
|
||||
EXTRACTED_TERMS_FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
|
||||
EXTRACTED_TERMS_FIELD_TYPE.freeze();
|
||||
}
|
||||
|
||||
private Directory directory;
|
||||
private IndexWriter indexWriter;
|
||||
private Map<BytesRef, Query> queries;
|
||||
private DirectoryReader directoryReader;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
directory = newDirectory();
|
||||
queries = new HashMap<>();
|
||||
IndexWriterConfig config = new IndexWriterConfig(new WhitespaceAnalyzer());
|
||||
config.setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
indexWriter = new IndexWriter(directory, config);
|
||||
}
|
||||
|
||||
@After
|
||||
public void destroy() throws Exception {
|
||||
directoryReader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testVariousQueries() throws Exception {
|
||||
addPercolatorQuery("1", new TermQuery(new Term("field", "brown")));
|
||||
addPercolatorQuery("2", new TermQuery(new Term("field", "monkey")));
|
||||
addPercolatorQuery("3", new TermQuery(new Term("field", "fox")));
|
||||
BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
|
||||
bq1.add(new TermQuery(new Term("field", "fox")), BooleanClause.Occur.SHOULD);
|
||||
bq1.add(new TermQuery(new Term("field", "monkey")), BooleanClause.Occur.SHOULD);
|
||||
addPercolatorQuery("4", bq1.build());
|
||||
BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
|
||||
bq2.add(new TermQuery(new Term("field", "fox")), BooleanClause.Occur.MUST);
|
||||
bq2.add(new TermQuery(new Term("field", "monkey")), BooleanClause.Occur.MUST);
|
||||
addPercolatorQuery("5", bq2.build());
|
||||
BooleanQuery.Builder bq3 = new BooleanQuery.Builder();
|
||||
bq3.add(new TermQuery(new Term("field", "fox")), BooleanClause.Occur.MUST);
|
||||
bq3.add(new TermQuery(new Term("field", "apes")), BooleanClause.Occur.MUST_NOT);
|
||||
addPercolatorQuery("6", bq3.build());
|
||||
BooleanQuery.Builder bq4 = new BooleanQuery.Builder();
|
||||
bq4.add(new TermQuery(new Term("field", "fox")), BooleanClause.Occur.MUST_NOT);
|
||||
bq4.add(new TermQuery(new Term("field", "apes")), BooleanClause.Occur.MUST);
|
||||
addPercolatorQuery("7", bq4.build());
|
||||
PhraseQuery.Builder pq1 = new PhraseQuery.Builder();
|
||||
pq1.add(new Term("field", "lazy"));
|
||||
pq1.add(new Term("field", "dog"));
|
||||
addPercolatorQuery("8", pq1.build());
|
||||
|
||||
indexWriter.close();
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
IndexSearcher shardSearcher = newSearcher(directoryReader);
|
||||
|
||||
MemoryIndex memoryIndex = new MemoryIndex();
|
||||
memoryIndex.addField("field", "the quick brown fox jumps over the lazy dog", new WhitespaceAnalyzer());
|
||||
IndexSearcher percolateSearcher = memoryIndex.createSearcher();
|
||||
|
||||
PercolatorQuery.Builder builder = new PercolatorQuery.Builder(
|
||||
percolateSearcher,
|
||||
queries,
|
||||
new MatchAllDocsQuery()
|
||||
);
|
||||
builder.extractQueryTermsQuery(EXTRACTED_TERMS_FIELD_NAME, UNKNOWN_QUERY_FIELD_NAME);
|
||||
TopDocs topDocs = shardSearcher.search(builder.build(), 10);
|
||||
assertThat(topDocs.totalHits, equalTo(5));
|
||||
assertThat(topDocs.scoreDocs.length, equalTo(5));
|
||||
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
|
||||
assertThat(topDocs.scoreDocs[1].doc, equalTo(2));
|
||||
assertThat(topDocs.scoreDocs[2].doc, equalTo(3));
|
||||
assertThat(topDocs.scoreDocs[3].doc, equalTo(5));
|
||||
assertThat(topDocs.scoreDocs[4].doc, equalTo(7));
|
||||
}
|
||||
|
||||
public void testWithScoring() throws Exception {
|
||||
addPercolatorQuery("1", new TermQuery(new Term("field", "brown")), "field", "value1");
|
||||
|
||||
indexWriter.close();
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
IndexSearcher shardSearcher = newSearcher(directoryReader);
|
||||
|
||||
MemoryIndex memoryIndex = new MemoryIndex();
|
||||
memoryIndex.addField("field", "the quick brown fox jumps over the lazy dog", new WhitespaceAnalyzer());
|
||||
IndexSearcher percolateSearcher = memoryIndex.createSearcher();
|
||||
|
||||
PercolatorQuery.Builder builder = new PercolatorQuery.Builder(
|
||||
percolateSearcher,
|
||||
queries,
|
||||
new MatchAllDocsQuery()
|
||||
);
|
||||
builder.extractQueryTermsQuery(EXTRACTED_TERMS_FIELD_NAME, UNKNOWN_QUERY_FIELD_NAME);
|
||||
builder.setPercolateQuery(new TermQuery(new Term("field", "value1")));
|
||||
|
||||
PercolatorQuery percolatorQuery = builder.build();
|
||||
TopDocs topDocs = shardSearcher.search(percolatorQuery, 1);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
assertThat(topDocs.scoreDocs.length, equalTo(1));
|
||||
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
|
||||
assertThat(topDocs.scoreDocs[0].score, not(1f));
|
||||
|
||||
Explanation explanation = shardSearcher.explain(percolatorQuery, 0);
|
||||
assertThat(explanation.isMatch(), is(true));
|
||||
assertThat(explanation.getValue(), equalTo(topDocs.scoreDocs[0].score));
|
||||
}
|
||||
|
||||
public void testDuel() throws Exception {
|
||||
int numQueries = scaledRandomIntBetween(32, 256);
|
||||
for (int i = 0; i < numQueries; i++) {
|
||||
String id = Integer.toString(i);
|
||||
Query query;
|
||||
if (randomBoolean()) {
|
||||
query = new PrefixQuery(new Term("field", id));
|
||||
} else if (randomBoolean()) {
|
||||
query = new WildcardQuery(new Term("field", id + "*"));
|
||||
} else if (randomBoolean()) {
|
||||
query = new CustomQuery(new Term("field", id + "*"));
|
||||
} else {
|
||||
query = new TermQuery(new Term("field", id));
|
||||
}
|
||||
addPercolatorQuery(id, query);
|
||||
}
|
||||
|
||||
indexWriter.close();
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
IndexSearcher shardSearcher = newSearcher(directoryReader);
|
||||
|
||||
for (int i = 0; i < numQueries; i++) {
|
||||
MemoryIndex memoryIndex = new MemoryIndex();
|
||||
String id = Integer.toString(i);
|
||||
memoryIndex.addField("field", id, new WhitespaceAnalyzer());
|
||||
IndexSearcher percolateSearcher = memoryIndex.createSearcher();
|
||||
|
||||
PercolatorQuery.Builder builder1 = new PercolatorQuery.Builder(
|
||||
percolateSearcher,
|
||||
queries,
|
||||
new MatchAllDocsQuery()
|
||||
);
|
||||
// enables the optimization that prevents queries from being evaluated that don't match
|
||||
builder1.extractQueryTermsQuery(EXTRACTED_TERMS_FIELD_NAME, UNKNOWN_QUERY_FIELD_NAME);
|
||||
TopDocs topDocs1 = shardSearcher.search(builder1.build(), 10);
|
||||
|
||||
PercolatorQuery.Builder builder2 = new PercolatorQuery.Builder(
|
||||
percolateSearcher,
|
||||
queries,
|
||||
new MatchAllDocsQuery()
|
||||
);
|
||||
TopDocs topDocs2 = shardSearcher.search(builder2.build(), 10);
|
||||
|
||||
assertThat(topDocs1.totalHits, equalTo(topDocs2.totalHits));
|
||||
assertThat(topDocs1.scoreDocs.length, equalTo(topDocs2.scoreDocs.length));
|
||||
for (int j = 0; j < topDocs1.scoreDocs.length; j++) {
|
||||
assertThat(topDocs1.scoreDocs[j].doc, equalTo(topDocs2.scoreDocs[j].doc));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void addPercolatorQuery(String id, Query query, String... extraFields) throws IOException {
|
||||
queries.put(new BytesRef(id), query);
|
||||
ParseContext.Document document = new ParseContext.Document();
|
||||
ExtractQueryTermsService.extractQueryTerms(query, document, EXTRACTED_TERMS_FIELD_NAME, UNKNOWN_QUERY_FIELD_NAME, EXTRACTED_TERMS_FIELD_TYPE);
|
||||
document.add(new StoredField(UidFieldMapper.NAME, Uid.createUid(PercolatorService.TYPE_NAME, id)));
|
||||
assert extraFields.length % 2 == 0;
|
||||
for (int i = 0; i < extraFields.length; i++) {
|
||||
document.add(new StringField(extraFields[i], extraFields[++i], Field.Store.NO));
|
||||
}
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
|
||||
private final static class CustomQuery extends Query {
|
||||
|
||||
private final Term term;
|
||||
|
||||
private CustomQuery(Term term) {
|
||||
this.term = term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
return new TermQuery(term);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "custom{" + field + "}";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,176 @@
|
|||
/*
|
||||
* 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.percolator;
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.percolate.PercolateShardResponse;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.analysis.AnalyzerProvider;
|
||||
import org.elasticsearch.index.analysis.CharFilterFactory;
|
||||
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
||||
import org.elasticsearch.index.analysis.TokenizerFactory;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.percolator.PercolatorFieldMapper;
|
||||
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
|
||||
import org.elasticsearch.index.percolator.ExtractQueryTermsService;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.IndicesModule;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.ContextIndexSearcher;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class PercolatorServiceTests extends ESTestCase {
|
||||
|
||||
private Directory directory;
|
||||
private IndexWriter indexWriter;
|
||||
private DirectoryReader directoryReader;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
directory = newDirectory();
|
||||
IndexWriterConfig config = new IndexWriterConfig(new WhitespaceAnalyzer());
|
||||
config.setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
indexWriter = new IndexWriter(directory, config);
|
||||
}
|
||||
|
||||
@After
|
||||
public void destroy() throws Exception {
|
||||
directoryReader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testCount() throws Exception {
|
||||
PercolateContext context = mock(PercolateContext.class);
|
||||
when(context.shardTarget()).thenReturn(new SearchShardTarget("_id", "_index", 0));
|
||||
when(context.percolatorTypeFilter()).thenReturn(new MatchAllDocsQuery());
|
||||
when(context.isOnlyCount()).thenReturn(true);
|
||||
|
||||
PercolatorQueriesRegistry registry = createRegistry();
|
||||
addPercolatorQuery("1", new TermQuery(new Term("field", "brown")), indexWriter, registry);
|
||||
addPercolatorQuery("2", new TermQuery(new Term("field", "fox")), indexWriter, registry);
|
||||
addPercolatorQuery("3", new TermQuery(new Term("field", "monkey")), indexWriter, registry);
|
||||
|
||||
indexWriter.close();
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
IndexSearcher shardSearcher = newSearcher(directoryReader);
|
||||
when(context.searcher()).thenReturn(new ContextIndexSearcher(new Engine.Searcher("test", shardSearcher), shardSearcher.getQueryCache(), shardSearcher.getQueryCachingPolicy()));
|
||||
|
||||
MemoryIndex memoryIndex = new MemoryIndex();
|
||||
memoryIndex.addField("field", "the quick brown fox jumps over the lazy dog", new WhitespaceAnalyzer());
|
||||
IndexSearcher percolateSearcher = memoryIndex.createSearcher();
|
||||
when(context.docSearcher()).thenReturn(percolateSearcher);
|
||||
|
||||
PercolateShardResponse response = PercolatorService.doPercolate(context, registry, null, null, null);
|
||||
assertThat(response.topDocs().totalHits, equalTo(2));
|
||||
}
|
||||
|
||||
public void testTopMatching() throws Exception {
|
||||
PercolateContext context = mock(PercolateContext.class);
|
||||
when(context.shardTarget()).thenReturn(new SearchShardTarget("_id", "_index", 0));
|
||||
when(context.percolatorTypeFilter()).thenReturn(new MatchAllDocsQuery());
|
||||
when(context.size()).thenReturn(10);
|
||||
|
||||
PercolatorQueriesRegistry registry = createRegistry();
|
||||
addPercolatorQuery("1", new TermQuery(new Term("field", "brown")), indexWriter, registry);
|
||||
addPercolatorQuery("2", new TermQuery(new Term("field", "monkey")), indexWriter, registry);
|
||||
addPercolatorQuery("3", new TermQuery(new Term("field", "fox")), indexWriter, registry);
|
||||
|
||||
indexWriter.close();
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
IndexSearcher shardSearcher = newSearcher(directoryReader);
|
||||
when(context.searcher()).thenReturn(new ContextIndexSearcher(new Engine.Searcher("test", shardSearcher), shardSearcher.getQueryCache(), shardSearcher.getQueryCachingPolicy()));
|
||||
|
||||
MemoryIndex memoryIndex = new MemoryIndex();
|
||||
memoryIndex.addField("field", "the quick brown fox jumps over the lazy dog", new WhitespaceAnalyzer());
|
||||
IndexSearcher percolateSearcher = memoryIndex.createSearcher();
|
||||
when(context.docSearcher()).thenReturn(percolateSearcher);
|
||||
|
||||
PercolateShardResponse response = PercolatorService.doPercolate(context, registry, null, null, null);
|
||||
TopDocs topDocs = response.topDocs();
|
||||
assertThat(topDocs.totalHits, equalTo(2));
|
||||
assertThat(topDocs.scoreDocs.length, equalTo(2));
|
||||
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
|
||||
assertThat(topDocs.scoreDocs[1].doc, equalTo(2));
|
||||
}
|
||||
|
||||
void addPercolatorQuery(String id, Query query, IndexWriter writer, PercolatorQueriesRegistry registry) throws IOException {
|
||||
registry.getPercolateQueries().put(new BytesRef(id), query);
|
||||
ParseContext.Document document = new ParseContext.Document();
|
||||
FieldType extractedQueryTermsFieldType = new FieldType();
|
||||
extractedQueryTermsFieldType.setTokenized(false);
|
||||
extractedQueryTermsFieldType.setIndexOptions(IndexOptions.DOCS);
|
||||
extractedQueryTermsFieldType.freeze();
|
||||
ExtractQueryTermsService.extractQueryTerms(query, document, PercolatorFieldMapper.EXTRACTED_TERMS_FULL_FIELD_NAME, PercolatorFieldMapper.UNKNOWN_QUERY_FULL_FIELD_NAME, extractedQueryTermsFieldType);
|
||||
document.add(new StoredField(UidFieldMapper.NAME, Uid.createUid(PercolatorService.TYPE_NAME, id)));
|
||||
writer.addDocument(document);
|
||||
}
|
||||
|
||||
PercolatorQueriesRegistry createRegistry() {
|
||||
Index index = new Index("_index");
|
||||
IndexSettings indexSettings = new IndexSettings(new IndexMetaData.Builder("_index").settings(
|
||||
Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.build(),
|
||||
Settings.EMPTY, Collections.emptyList()
|
||||
);
|
||||
return new PercolatorQueriesRegistry(
|
||||
new ShardId(index, 0),
|
||||
indexSettings,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
}
|
|
@ -136,7 +136,7 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
|
||||
DeleteIndexResponse actionGet = client().admin().indices().prepareDelete("test").get();
|
||||
assertThat(actionGet.isAcknowledged(), equalTo(true));
|
||||
client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).get();
|
||||
assertAcked(prepareCreate("test").addMapping("type1", "field1", "type=string").addMapping(PercolatorService.TYPE_NAME, "color", "type=string"));
|
||||
clusterHealth = client().admin().cluster().health(clusterHealthRequest().waitForYellowStatus().waitForActiveShards(1)).actionGet();
|
||||
logger.info("Done Cluster Health, status " + clusterHealth.getStatus());
|
||||
assertThat(clusterHealth.isTimedOut(), equalTo(false));
|
||||
|
@ -193,6 +193,7 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
.endObject())
|
||||
.get();
|
||||
}
|
||||
refresh();
|
||||
|
||||
logger.info("--> Percolate doc with field1=95");
|
||||
PercolateResponse response = client().preparePercolate()
|
||||
|
@ -249,6 +250,7 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
.setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject())
|
||||
.get();
|
||||
}
|
||||
refresh();
|
||||
|
||||
final String document = "{\"field\" : \"a\"}";
|
||||
client.prepareIndex("test", "type", "1")
|
||||
|
@ -269,7 +271,7 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
|
||||
for (int i = 0; i < numPercolateRequest; i++) {
|
||||
PercolateRequestBuilder percolateBuilder = client.preparePercolate()
|
||||
.setIndices("test").setDocumentType("type");
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries);
|
||||
if (randomBoolean()) {
|
||||
percolateBuilder.setGetRequest(Requests.getRequest("test").type("type").id("1"));
|
||||
} else {
|
||||
|
@ -289,7 +291,7 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
}
|
||||
} else {
|
||||
PercolateRequestBuilder percolateBuilder = client.preparePercolate()
|
||||
.setIndices("test").setDocumentType("type");
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries);
|
||||
if (randomBoolean()) {
|
||||
percolateBuilder.setPercolateDoc(docBuilder().setDoc(document));
|
||||
} else {
|
||||
|
|
|
@ -182,7 +182,7 @@ public class TTLPercolatorIT extends ESIntegTestCase {
|
|||
.endObject()
|
||||
.endObject()
|
||||
.endObject()
|
||||
).setTTL(randomIntBetween(1, 500)).execute().actionGet();
|
||||
).setTTL(randomIntBetween(1, 500)).setRefresh(true).execute().actionGet();
|
||||
} catch (MapperParsingException e) {
|
||||
logger.info("failed indexing {}", i, e);
|
||||
// if we are unlucky the TTL is so small that we see the expiry date is already in the past when
|
||||
|
|
Binary file not shown.
|
@ -578,3 +578,19 @@ balancing into account but don't assign the shard if the allocation deciders are
|
|||
in the case where shard copies can be found. Previously, a node not holding the shard copy was chosen if none of the nodes
|
||||
holding shard copies were satisfying the allocation deciders. Now, the shard will be assigned to a node having a shard copy,
|
||||
even if none of the nodes holding a shard copy satisfy the allocation deciders.
|
||||
|
||||
=== Percolator
|
||||
|
||||
Adding percolator queries and modifications to existing percolator queries are no longer visible in immediately
|
||||
to the percolator. A refresh is required to run before the changes are visible to the percolator.
|
||||
|
||||
The reason that this has changed is that on newly created indices the percolator automatically indexes the query terms
|
||||
and these query terms are used at percolate time to reduce the amount of queries the percolate API needs evaluate.
|
||||
This optimization didn't work in the percolate API mode where modifications to queries are immediately visible.
|
||||
|
||||
The percolator by defaults sets the `size` option to `10` whereas before this was set to unlimited.
|
||||
|
||||
The percolate api can no longer accept documents that have fields that don't exist in the mapping.
|
||||
|
||||
When percolating an existing document then specifying a document in the source of the percolate request is not allowed
|
||||
any more.
|
|
@ -1,6 +1,12 @@
|
|||
[[search-percolate]]
|
||||
== Percolator
|
||||
|
||||
added[3.0.0,Percolator queries modifications aren't visible immediately and a refresh is required]
|
||||
|
||||
added[3.0.0,Percolate api by defaults limits the number of matches to `10` whereas before this wasn't set]
|
||||
|
||||
added[3.0.0,For indices created on or after version 3.0.0 the percolator automatically indexes the query terms with the percolator queries this allows the percolator to percolate documents quicker. It is advisable to reindex any pre 3.0.0 indices to take advantage of this new optimization]
|
||||
|
||||
Traditionally you design documents based on your data, store them into an index, and then define queries via the search API
|
||||
in order to retrieve these documents. The percolator works in the opposite direction. First you store queries into an
|
||||
index and then, via the percolate API, you define documents in order to retrieve these queries.
|
||||
|
@ -10,9 +16,6 @@ JSON. This allows you to embed queries into documents via the index API. Elastic
|
|||
document and make it available to the percolate API. Since documents are also defined as JSON, you can define a document
|
||||
in a request to the percolate API.
|
||||
|
||||
The percolator and most of its features work in realtime, so once a percolate query is indexed it can immediately be used
|
||||
in the percolate API.
|
||||
|
||||
[IMPORTANT]
|
||||
=====================================
|
||||
|
||||
|
@ -219,7 +222,7 @@ filter will be included in the percolate execution. The filter option works in n
|
|||
occurred for the filter to included the latest percolate queries.
|
||||
* `query` - Same as the `filter` option, but also the score is computed. The computed scores can then be used by the
|
||||
`track_scores` and `sort` option.
|
||||
* `size` - Defines to maximum number of matches (percolate queries) to be returned. Defaults to unlimited.
|
||||
* `size` - Defines to maximum number of matches (percolate queries) to be returned. Defaults to 10.
|
||||
* `track_scores` - Whether the `_score` is included for each match. The `_score` is based on the query and represents
|
||||
how the query matched the *percolate query's metadata*, *not* how the document (that is being percolated) matched
|
||||
the query. The `query` option is required for this option. Defaults to `false`.
|
||||
|
@ -310,6 +313,10 @@ document.
|
|||
Internally the percolate API will issue a GET request for fetching the `_source` of the document to percolate.
|
||||
For this feature to work, the `_source` for documents to be percolated needs to be stored.
|
||||
|
||||
If percolating an existing document and the a document is also specified in the source of the percolate request then
|
||||
an error is thrown. Either the document to percolate should be specified in the source or be defined by specifying the
|
||||
index, type and id.
|
||||
|
||||
[float]
|
||||
==== Example
|
||||
|
||||
|
@ -379,13 +386,11 @@ requests.txt:
|
|||
{"percolate" : {"index" : "twitter", "type" : "tweet"}}
|
||||
{"doc" : {"message" : "some text"}}
|
||||
{"percolate" : {"index" : "twitter", "type" : "tweet", "id" : "1"}}
|
||||
{}
|
||||
{"percolate" : {"index" : "users", "type" : "user", "id" : "3", "percolate_index" : "users_2012" }}
|
||||
{"size" : 10}
|
||||
{"count" : {"index" : "twitter", "type" : "tweet"}}
|
||||
{"doc" : {"message" : "some other text"}}
|
||||
{"count" : {"index" : "twitter", "type" : "tweet", "id" : "1"}}
|
||||
{}
|
||||
--------------------------------------------------
|
||||
|
||||
For a percolate existing document item (headers with the `id` field), the response can be an empty JSON object.
|
||||
|
|
|
@ -128,7 +128,7 @@ public class TemplateQueryParserTests extends ESTestCase {
|
|||
ScriptService scriptService = injector.getInstance(ScriptService.class);
|
||||
SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap());
|
||||
MapperRegistry mapperRegistry = new IndicesModule().getMapperRegistry();
|
||||
MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry);
|
||||
MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, () -> context);
|
||||
IndexFieldDataService indexFieldDataService =new IndexFieldDataService(idxSettings, injector.getInstance(IndicesFieldDataCache.class), injector.getInstance(CircuitBreakerService.class), mapperService);
|
||||
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() {
|
||||
@Override
|
||||
|
|
|
@ -52,7 +52,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
|
|||
Collections.singletonMap(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser()),
|
||||
Collections.emptyMap());
|
||||
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
}
|
||||
|
||||
public void testDefaults() throws Exception {
|
||||
|
@ -128,7 +128,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
|
|||
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
|
||||
indexService = createIndex("test_bwc", settings);
|
||||
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties").startObject("field")
|
||||
.field("type", "murmur3")
|
||||
|
@ -144,7 +144,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
|
|||
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
|
||||
indexService = createIndex("test_bwc", settings);
|
||||
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties").startObject("field")
|
||||
.field("type", "murmur3")
|
||||
|
|
|
@ -58,7 +58,7 @@ public class SizeMappingTests extends ESSingleNodeTestCase {
|
|||
Map<String, MetadataFieldMapper.TypeParser> metadataMappers = new HashMap<>();
|
||||
IndicesModule indices = new IndicesModule();
|
||||
indices.registerMetadataMapper(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser());
|
||||
mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), indices.getMapperRegistry());
|
||||
mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), indices.getMapperRegistry(), indexService::getQueryShardContext);
|
||||
parser = mapperService.documentMapperParser();
|
||||
}
|
||||
|
||||
|
@ -90,7 +90,7 @@ public class SizeMappingTests extends ESSingleNodeTestCase {
|
|||
Collections.emptyMap(),
|
||||
Collections.singletonMap(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser()));
|
||||
parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService,
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry);
|
||||
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
|
||||
DocumentMapper docMapper = parser.parse("type", new CompressedXContent(mapping));
|
||||
|
||||
BytesReference source = XContentFactory.jsonBuilder()
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
query:
|
||||
match_all: {}
|
||||
|
||||
- do:
|
||||
indices.refresh: {}
|
||||
|
||||
- do:
|
||||
mpercolate:
|
||||
body:
|
||||
|
@ -33,8 +36,7 @@
|
|||
index: percolator_index
|
||||
type: my_type
|
||||
id: 1
|
||||
- doc:
|
||||
foo: bar
|
||||
- {}
|
||||
|
||||
- match: { responses.0.total: 1 }
|
||||
- match: { responses.1.error.root_cause.0.type: index_not_found_exception }
|
||||
|
|
|
@ -30,6 +30,7 @@ dependencies {
|
|||
compile "org.apache.httpcomponents:httpcore:${versions.httpcore}"
|
||||
compile "commons-logging:commons-logging:${versions.commonslogging}"
|
||||
compile "commons-codec:commons-codec:${versions.commonscodec}"
|
||||
compile 'org.elasticsearch:securemock:1.2'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << '-Xlint:-cast,-deprecation,-fallthrough,-overrides,-rawtypes,-serial,-try,-unchecked'
|
||||
|
|
|
@ -58,6 +58,7 @@ public class MapperTestUtils {
|
|||
return new MapperService(indexSettings,
|
||||
analysisService,
|
||||
similarityService,
|
||||
mapperRegistry);
|
||||
mapperRegistry,
|
||||
() -> null);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue