parent
de91ffd0ce
commit
db3c6c2a8e
|
@ -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
|
||||
|
|
|
@ -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}
|
||||
|
||||
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue