Enable percolation for nested documents

closes #5082
This commit is contained in:
Britta Weber 2014-02-11 13:46:54 +01:00
parent de91ffd0ce
commit db3c6c2a8e
9 changed files with 538 additions and 80 deletions

View File

@ -437,7 +437,7 @@ the time the percolate api needs to run can be decreased.
=== Important notes
Because the percolator API is processing one document at a time, it doesn't support queries and filters that run
against child and nested documents such as `has_child`, `has_parent`, `top_children`, and `nested`.
against child documents such as `has_child`, `has_parent` and `top_children`.
The `wildcard` and `regexp` query natively use a lot of memory and because the percolator keeps the queries into memory
this can easily take up the available memory in the heap space. If possible try to use a `prefix` query or ngramming to

View File

@ -0,0 +1,113 @@
---
setup:
- do:
indices.create:
index: nestedindex
body:
mappings:
company:
properties:
companyname:
type: string
employee:
type: nested
properties:
name:
type: string
- do:
indices.refresh: {}
- do:
index:
index: nestedindex
type: ".percolator"
id: query
body: {"query":{"nested":{"path":"employee","score_mode":"avg","query":{"match":{"employee.name":{"query":"virginia potts","operator":"and"}}}}}}
- do:
indices.refresh: {}
---
"Basic percolation tests on nested doc":
- do:
percolate:
index: nestedindex
type: company
body: {"doc":{"companyname":"stark","employee":[{"name":"virginia stark"},{"name":"tony potts"}]}}
- match: {'total': 0}
- do:
percolate:
index: nestedindex
type: company
body: {"doc":{"companyname":"stark","employee":[{"name":"virginia potts"},{"name":"tony stark"}]}}
- match: {'total': 1}
---
"Percolate existing docs":
- do:
index:
index: nestedindex
type: company
id: notmatching
body: {"companyname":"stark","employee":[{"name":"virginia stark"},{"name":"tony potts"}]}
- do:
index:
index: nestedindex
type: company
id: matching
body: {"companyname":"stark","employee":[{"name":"virginia potts"},{"name":"tony stark"}]}
- do:
indices.refresh: {}
- do:
percolate:
index: nestedindex
type: company
id: notmatching
- match: {'total': 0}
- do:
percolate:
index: nestedindex
type: company
id: matching
- match: {'total': 1}
---
"Test multi percolate":
- do:
mpercolate:
body:
- percolate: {"index": "nestedindex", "type": "company"}
- doc: {"companyname":"stark","employee":[{"name":"virginia stark"},{"name":"tony potts"}]}
- percolate: {"index": "nestedindex", "type": "company"}
- doc: {"companyname":"stark","employee":[{"name":"virginia potts"},{"name":"tony stark"}]}
- match: {'responses.0.total': 0}
- match: {'responses.1.total': 1}
- do:
mpercolate:
body:
- percolate: {"index": "nestedindex", "type": "company"}
- doc: {"companyname":"stark","employee":[{"name":"virginia potts"},{"name":"tony stark"}]}
- percolate: {"index": "nestedindex", "type": "company"}
- doc: {"companyname":"stark","employee":[{"name":"virginia stark"},{"name":"tony potts"}]}
- match: {'responses.0.total': 1}
- match: {'responses.1.total': 0}

View File

@ -0,0 +1,118 @@
/*
* 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.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.index.*;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
class MultiDocumentPercolatorIndex implements PercolatorIndex {
public MultiDocumentPercolatorIndex() {
}
@Override
public void prepare(PercolateContext context, ParsedDocument parsedDocument) {
int docCounter = 0;
IndexReader[] memoryIndices = new IndexReader[parsedDocument.docs().size()];
for (ParseContext.Document d : parsedDocument.docs()) {
memoryIndices[docCounter] = indexDoc(d, parsedDocument.analyzer()).createSearcher().getIndexReader();
docCounter++;
}
MultiReader mReader = new MultiReader(memoryIndices, true);
try {
AtomicReader slowReader = SlowCompositeReaderWrapper.wrap(mReader);
DocSearcher docSearcher = new DocSearcher(new IndexSearcher(slowReader));
context.initialize(docSearcher, parsedDocument);
} catch (IOException e) {
throw new ElasticsearchException("Failed to create index for percolator with nested document ", e);
}
}
MemoryIndex indexDoc(ParseContext.Document d, Analyzer analyzer) {
MemoryIndex memoryIndex = new MemoryIndex(true);
for (IndexableField field : d.getFields()) {
if (!field.fieldType().indexed() && field.name().equals(UidFieldMapper.NAME)) {
continue;
}
try {
TokenStream tokenStream = field.tokenStream(analyzer);
if (tokenStream != null) {
memoryIndex.addField(field.name(), tokenStream, field.boost());
}
} catch (IOException e) {
throw new ElasticsearchException("Failed to create token stream", e);
}
}
return memoryIndex;
}
@Override
public void clean() {
// noop
}
private class DocSearcher implements Engine.Searcher {
private final IndexSearcher searcher;
private DocSearcher(IndexSearcher searcher) {
this.searcher = searcher;
}
@Override
public String source() {
return "percolate";
}
@Override
public IndexReader reader() {
return searcher.getIndexReader();
}
@Override
public IndexSearcher searcher() {
return searcher;
}
@Override
public boolean release() throws ElasticsearchException {
try {
searcher.getIndexReader().close();
} catch (IOException e) {
throw new ElasticsearchException("failed to close IndexReader in percolator with nested doc", e);
}
return true;
}
}
}

View File

@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
@ -73,7 +72,6 @@ import org.elasticsearch.search.rescore.RescoreSearchContext;
import org.elasticsearch.search.scan.ScanContext;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -100,7 +98,7 @@ public class PercolateContext extends SearchContext {
private final ConcurrentMap<HashedBytesRef, Query> percolateQueries;
private String[] types;
private Engine.Searcher docEngineSearcher;
private Engine.Searcher docSearcher;
private Engine.Searcher engineSearcher;
private ContextIndexSearcher searcher;
@ -134,38 +132,16 @@ public class PercolateContext extends SearchContext {
this.scriptService = scriptService;
}
public void initialize(final MemoryIndex memoryIndex, ParsedDocument parsedDocument) {
final IndexSearcher docSearcher = memoryIndex.createSearcher();
final IndexReader topLevelReader = docSearcher.getIndexReader();
AtomicReaderContext readerContext = topLevelReader.leaves().get(0);
docEngineSearcher = new Engine.Searcher() {
@Override
public String source() {
return "percolate";
}
public IndexSearcher docSearcher() {
return docSearcher.searcher();
}
@Override
public IndexReader reader() {
return topLevelReader;
}
public void initialize(Engine.Searcher docSearcher, ParsedDocument parsedDocument) {
this.docSearcher = docSearcher;
@Override
public IndexSearcher searcher() {
return docSearcher;
}
@Override
public boolean release() throws ElasticsearchException {
try {
docSearcher.getIndexReader().close();
memoryIndex.reset();
} catch (IOException e) {
throw new ElasticsearchException("failed to close percolator in-memory index", e);
}
return true;
}
};
lookup().setNextReader(readerContext);
IndexReader indexReader = docSearcher.reader();
AtomicReaderContext atomicReaderContext = indexReader.leaves().get(0);
lookup().setNextReader(atomicReaderContext);
lookup().setNextDocId(0);
lookup().source().setNextSource(parsedDocument.source());
@ -173,12 +149,10 @@ public class PercolateContext extends SearchContext {
for (IndexableField field : parsedDocument.rootDoc().getFields()) {
fields.put(field.name(), new InternalSearchHitField(field.name(), ImmutableList.of()));
}
hitContext = new FetchSubPhase.HitContext();
hitContext.reset(new InternalSearchHit(0, "unknown", new StringText(request.documentType()), fields), readerContext, 0, topLevelReader, 0, new JustSourceFieldsVisitor());
}
public IndexSearcher docSearcher() {
return docEngineSearcher.searcher();
hitContext().reset(
new InternalSearchHit(0, "unknown", new StringText(parsedDocument.type()), fields),
atomicReaderContext, 0, indexReader, 0, new JustSourceFieldsVisitor()
);
}
public IndexShard indexShard() {
@ -202,6 +176,9 @@ public class PercolateContext extends SearchContext {
}
public FetchSubPhase.HitContext hitContext() {
if (hitContext == null) {
hitContext = new FetchSubPhase.HitContext();
}
return hitContext;
}
@ -231,11 +208,11 @@ public class PercolateContext extends SearchContext {
@Override
public boolean release() throws ElasticsearchException {
try {
if (docEngineSearcher != null) {
IndexReader indexReader = docEngineSearcher.reader();
if (docSearcher != null) {
IndexReader indexReader = docSearcher.reader();
fieldDataService.clear(indexReader);
indexService.cache().clear(indexReader);
return docEngineSearcher.release();
return docSearcher.release();
} else {
return false;
}

View File

@ -0,0 +1,40 @@
/*
* 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.elasticsearch.index.mapper.ParsedDocument;
interface PercolatorIndex {
/**
* Indexes the document(s) and initializes the PercolateContext
*
* @param context Initialized with document related properties for fetch phase.
* @param document Document that is percolated. Can contain several documents.
* */
void prepare(PercolateContext context, ParsedDocument document);
/**
* Release resources
* */
void clean();
}

View File

@ -21,15 +21,10 @@ package org.elasticsearch.percolator;
import com.carrotsearch.hppc.ByteObjectOpenHashMap;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexableField;
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.*;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ElasticsearchParseException;
@ -52,8 +47,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.BytesText;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@ -66,7 +59,6 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.IdFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.service.IndexService;
@ -106,13 +98,15 @@ public class PercolatorService extends AbstractComponent {
public final static float NO_SCORE = Float.NEGATIVE_INFINITY;
public final static String TYPE_NAME = ".percolator";
private final CloseableThreadLocal<MemoryIndex> cache;
private final IndicesService indicesService;
private final ByteObjectOpenHashMap<PercolatorType> percolatorTypes;
private final CacheRecycler cacheRecycler;
private final PageCacheRecycler pageCacheRecycler;
private final ClusterService clusterService;
private final PercolatorIndex single;
private final PercolatorIndex multi;
private final FacetPhase facetPhase;
private final HighlightPhase highlightPhase;
private final AggregationPhase aggregationPhase;
@ -134,13 +128,8 @@ public class PercolatorService extends AbstractComponent {
this.scriptService = scriptService;
this.sortParseElement = new SortParseElement();
final long maxReuseBytes = settings.getAsBytesSize("indices.memory.memory_index.size_per_thread", new ByteSizeValue(1, ByteSizeUnit.MB)).bytes();
cache = new CloseableThreadLocal<MemoryIndex>() {
@Override
protected MemoryIndex initialValue() {
return new ExtendedMemoryIndex(true, maxReuseBytes);
}
};
single = new SingleDocumentPercolatorIndex(settings);
multi = new MultiDocumentPercolatorIndex();
percolatorTypes = new ByteObjectOpenHashMap<PercolatorType>(6);
percolatorTypes.put(countPercolator.id(), countPercolator);
@ -170,7 +159,6 @@ public class PercolatorService extends AbstractComponent {
request, searchShardTarget, indexShard, percolateIndexService, cacheRecycler, pageCacheRecycler, scriptService
);
try {
ParsedDocument parsedDocument = parseRequest(percolateIndexService, request, context);
if (context.percolateQueries().isEmpty()) {
return new PercolateShardResponse(context, request.index(), request.shardId());
@ -198,22 +186,12 @@ public class PercolatorService extends AbstractComponent {
context.size = 0;
}
// first, parse the source doc into a MemoryIndex
final MemoryIndex memoryIndex = cache.get();
// TODO: This means percolation does not support nested docs...
// So look into: ByteBufferDirectory
for (IndexableField field : parsedDocument.rootDoc().getFields()) {
if (!field.fieldType().indexed() && field.name().equals(UidFieldMapper.NAME)) {
continue;
}
try {
TokenStream tokenStream = field.tokenStream(parsedDocument.analyzer());
if (tokenStream != null) {
memoryIndex.addField(field.name(), tokenStream, field.boost());
}
} catch (IOException e) {
throw new ElasticsearchException("Failed to create token stream", e);
}
// parse the source either into one MemoryIndex, if it is a single document or index multiple docs if nested
PercolatorIndex percolatorIndex;
if (indexShard.mapperService().documentMapper(request.documentType()).hasNestedObjects()) {
percolatorIndex = multi;
} else {
percolatorIndex = single;
}
PercolatorType action;
@ -230,7 +208,8 @@ public class PercolatorService extends AbstractComponent {
}
context.percolatorTypeId = action.id();
context.initialize(memoryIndex, parsedDocument);
percolatorIndex.prepare(context, parsedDocument);
indexShard.readAllowed();
return action.doPercolate(request, context);
} finally {
@ -413,7 +392,8 @@ public class PercolatorService extends AbstractComponent {
}
public void close() {
cache.close();
single.clean();
multi.clean();
}
interface PercolatorType {

View File

@ -0,0 +1,114 @@
/*
* 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.TokenStream;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.memory.ExtendedMemoryIndex;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import java.io.IOException;
class SingleDocumentPercolatorIndex implements PercolatorIndex {
private final CloseableThreadLocal<MemoryIndex> cache;
public SingleDocumentPercolatorIndex(Settings settings) {
final long maxReuseBytes = settings.getAsBytesSize("indices.memory.memory_index.size_per_thread", new ByteSizeValue(1, ByteSizeUnit.MB)).bytes();
cache = new CloseableThreadLocal<MemoryIndex>() {
@Override
protected MemoryIndex initialValue() {
return new ExtendedMemoryIndex(true, maxReuseBytes);
}
};
}
@Override
public void prepare(PercolateContext context, ParsedDocument parsedDocument) {
MemoryIndex memoryIndex = cache.get();
for (IndexableField field : parsedDocument.rootDoc().getFields()) {
if (!field.fieldType().indexed() && field.name().equals(UidFieldMapper.NAME)) {
continue;
}
try {
TokenStream tokenStream = field.tokenStream(parsedDocument.analyzer());
if (tokenStream != null) {
memoryIndex.addField(field.name(), tokenStream, field.boost());
}
} catch (IOException e) {
throw new ElasticsearchException("Failed to create token stream", e);
}
}
context.initialize(new DocEngineSearcher(memoryIndex), parsedDocument);
}
@Override
public void clean() {
cache.close();
}
private class DocEngineSearcher implements Engine.Searcher {
private final IndexSearcher searcher;
private final MemoryIndex memoryIndex;
public DocEngineSearcher(MemoryIndex memoryIndex) {
this.searcher = memoryIndex.createSearcher();
this.memoryIndex = memoryIndex;
}
@Override
public String source() {
return "percolate";
}
@Override
public IndexReader reader() {
return searcher.getIndexReader();
}
@Override
public IndexSearcher searcher() {
return searcher;
}
@Override
public boolean release() throws ElasticsearchException {
try {
searcher.getIndexReader().close();
memoryIndex.reset();
} catch (IOException e) {
throw new ElasticsearchException("failed to close percolator in-memory index", e);
}
return true;
}
}
}

View File

@ -19,17 +19,23 @@
package org.elasticsearch.percolator;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.percolate.MultiPercolateRequestBuilder;
import org.elasticsearch.action.percolate.MultiPercolateResponse;
import org.elasticsearch.action.percolate.*;
import org.elasticsearch.client.Requests;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.MatchQueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.io.IOException;
import static org.elasticsearch.action.percolate.PercolateSourceBuilder.docBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.*;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.percolator.PercolatorTests.convertFromTextArray;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertMatchCount;
import static org.hamcrest.Matchers.*;
@ -261,4 +267,52 @@ public class MultiPercolatorTests extends ElasticsearchIntegrationTest {
assertThat(response.items()[numPercolateRequest].getResponse().getMatches().length, equalTo(numQueries));
}
@Test
public void testNestedMultiPercolation() throws IOException {
initNestedIndexAndPercolation();
MultiPercolateRequestBuilder mpercolate= client().prepareMultiPercolate();
mpercolate.add(client().preparePercolate().setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(getNotMatchingNestedDoc())).setIndices("nestedindex").setDocumentType("company"));
mpercolate.add(client().preparePercolate().setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(getMatchingNestedDoc())).setIndices("nestedindex").setDocumentType("company"));
MultiPercolateResponse response = mpercolate.get();
assertEquals(response.getItems()[0].getResponse().getMatches().length, 0);
assertEquals(response.getItems()[1].getResponse().getMatches().length, 1);
assertEquals(response.getItems()[1].getResponse().getMatches()[0].getId().string(), "Q");
}
void initNestedIndexAndPercolation() throws IOException {
XContentBuilder mapping = XContentFactory.jsonBuilder();
mapping.startObject().startObject("properties").startObject("companyname").field("type", "string").endObject()
.startObject("employee").field("type", "nested").startObject("properties")
.startObject("name").field("type", "string").endObject().endObject().endObject().endObject()
.endObject();
assertAcked(client().admin().indices().prepareCreate("nestedindex").addMapping("company", mapping));
ensureGreen("nestedindex");
client().prepareIndex("nestedindex", PercolatorService.TYPE_NAME, "Q").setSource(jsonBuilder().startObject()
.field("query", QueryBuilders.nestedQuery("employee", QueryBuilders.matchQuery("employee.name", "virginia potts").operator(MatchQueryBuilder.Operator.AND)).scoreMode("avg")).endObject()).get();
refresh();
}
XContentBuilder getMatchingNestedDoc() throws IOException {
XContentBuilder doc = XContentFactory.jsonBuilder();
doc.startObject().field("companyname", "stark").startArray("employee")
.startObject().field("name", "virginia potts").endObject()
.startObject().field("name", "tony stark").endObject()
.endArray().endObject();
return doc;
}
XContentBuilder getNotMatchingNestedDoc() throws IOException {
XContentBuilder doc = XContentFactory.jsonBuilder();
doc.startObject().field("companyname", "notstark").startArray("employee")
.startObject().field("name", "virginia stark").endObject()
.startObject().field("name", "tony potts").endObject()
.endArray().endObject();
return doc;
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.get.GetRequestBuilder;
import org.elasticsearch.action.percolate.PercolateResponse;
import org.elasticsearch.action.percolate.PercolateSourceBuilder;
import org.elasticsearch.action.search.SearchResponse;
@ -35,11 +36,13 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.ImmutableSettings.Builder;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContent;
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.query.FilterBuilders;
import org.elasticsearch.index.query.MatchQueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.functionscore.factor.FactorBuilder;
import org.elasticsearch.rest.RestStatus;
@ -48,10 +51,12 @@ import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.io.IOException;
import java.util.*;
import static org.elasticsearch.action.percolate.PercolateSourceBuilder.docBuilder;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
import static org.elasticsearch.common.xcontent.XContentFactory.*;
import static org.elasticsearch.index.query.FilterBuilders.termFilter;
import static org.elasticsearch.index.query.QueryBuilders.*;
@ -1642,4 +1647,61 @@ public class PercolatorTests extends ElasticsearchIntegrationTest {
assertMatchCount(percolate, 0l);
}
@Test
public void testNestedPercolation() throws IOException {
initNestedIndexAndPercolation();
PercolateResponse response = client().preparePercolate().setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(getNotMatchingNestedDoc())).setIndices("nestedindex").setDocumentType("company").get();
assertEquals(response.getMatches().length, 0);
response = client().preparePercolate().setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(getMatchingNestedDoc())).setIndices("nestedindex").setDocumentType("company").get();
assertEquals(response.getMatches().length, 1);
assertEquals(response.getMatches()[0].getId().string(), "Q");
}
@Test
public void testNestedPercolationOnExistingDoc() throws IOException {
initNestedIndexAndPercolation();
client().prepareIndex("nestedindex", "company", "notmatching").setSource(getNotMatchingNestedDoc()).get();
client().prepareIndex("nestedindex", "company", "matching").setSource(getMatchingNestedDoc()).get();
refresh();
PercolateResponse response = client().preparePercolate().setGetRequest(Requests.getRequest("nestedindex").type("company").id("notmatching")).setDocumentType("company").setIndices("nestedindex").get();
assertEquals(response.getMatches().length, 0);
response = client().preparePercolate().setGetRequest(Requests.getRequest("nestedindex").type("company").id("matching")).setDocumentType("company").setIndices("nestedindex").get();
assertEquals(response.getMatches().length, 1);
assertEquals(response.getMatches()[0].getId().string(), "Q");
}
void initNestedIndexAndPercolation() throws IOException {
XContentBuilder mapping = XContentFactory.jsonBuilder();
mapping.startObject().startObject("properties").startObject("companyname").field("type", "string").endObject()
.startObject("employee").field("type", "nested").startObject("properties")
.startObject("name").field("type", "string").endObject().endObject().endObject().endObject()
.endObject();
assertAcked(client().admin().indices().prepareCreate("nestedindex").addMapping("company", mapping));
ensureGreen("nestedindex");
client().prepareIndex("nestedindex", PercolatorService.TYPE_NAME, "Q").setSource(jsonBuilder().startObject()
.field("query", QueryBuilders.nestedQuery("employee", QueryBuilders.matchQuery("employee.name", "virginia potts").operator(MatchQueryBuilder.Operator.AND)).scoreMode("avg")).endObject()).get();
refresh();
}
XContentBuilder getMatchingNestedDoc() throws IOException {
XContentBuilder doc = XContentFactory.jsonBuilder();
doc.startObject().field("companyname", "stark").startArray("employee")
.startObject().field("name", "virginia potts").endObject()
.startObject().field("name", "tony stark").endObject()
.endArray().endObject();
return doc;
}
XContentBuilder getNotMatchingNestedDoc() throws IOException {
XContentBuilder doc = XContentFactory.jsonBuilder();
doc.startObject().field("companyname", "notstark").startArray("employee")
.startObject().field("name", "virginia stark").endObject()
.startObject().field("name", "tony potts").endObject()
.endArray().endObject();
return doc;
}
}