Query DSL: Allow to associate a custom cache key with a filter, closes #1142.
This commit is contained in:
parent
91e050f885
commit
fbd6e85eac
|
@ -49,7 +49,7 @@ public class ExistsFieldQueryExtension implements FieldQueryExtension {
|
|||
}
|
||||
|
||||
// we always cache this one, really does not change...
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
||||
|
|
|
@ -51,10 +51,10 @@ public class MissingFieldQueryExtension implements FieldQueryExtension {
|
|||
}
|
||||
|
||||
// we always cache this one, really does not change... (exists)
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
filter = new NotFilter(filter);
|
||||
// cache the not filter as well, so it will be faster
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
||||
|
|
|
@ -68,7 +68,7 @@ public class ResidentFilterCache extends AbstractConcurrentMapFilterCache implem
|
|||
super.close();
|
||||
}
|
||||
|
||||
@Override protected ConcurrentMap<Filter, DocSet> buildFilterMap() {
|
||||
@Override protected ConcurrentMap<Object, DocSet> buildFilterMap() {
|
||||
MapMaker mapMaker = new MapMaker();
|
||||
if (maxSize != -1) {
|
||||
mapMaker.maximumSize(maxSize);
|
||||
|
|
|
@ -68,7 +68,7 @@ public class SoftFilterCache extends AbstractConcurrentMapFilterCache implements
|
|||
super.close();
|
||||
}
|
||||
|
||||
@Override protected ConcurrentMap<Filter, DocSet> buildFilterMap() {
|
||||
@Override protected ConcurrentMap<Object, DocSet> buildFilterMap() {
|
||||
// DocSet are not really stored with strong reference only when searching on them...
|
||||
// Filter might be stored in query cache
|
||||
MapMaker mapMaker = new MapMaker().softValues();
|
||||
|
|
|
@ -47,7 +47,7 @@ import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.*;
|
|||
*/
|
||||
public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComponent implements FilterCache, IndexReader.ReaderFinishedListener {
|
||||
|
||||
final ConcurrentMap<Object, FilterCacheValue<ConcurrentMap<Filter, DocSet>>> cache;
|
||||
final ConcurrentMap<Object, FilterCacheValue<ConcurrentMap<Object, DocSet>>> cache;
|
||||
|
||||
final boolean labEnabled;
|
||||
final ByteSizeValue labMaxAlloc;
|
||||
|
@ -75,11 +75,11 @@ public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComp
|
|||
this.labChunkSizeBytes = (int) (labChunkSize.bytes() / RamUsage.NUM_BYTES_LONG);
|
||||
}
|
||||
|
||||
protected ConcurrentMap<Object, FilterCacheValue<ConcurrentMap<Filter, DocSet>>> buildCache() {
|
||||
return new ConcurrentHashMap<Object, FilterCacheValue<ConcurrentMap<Filter, DocSet>>>();
|
||||
protected ConcurrentMap<Object, FilterCacheValue<ConcurrentMap<Object, DocSet>>> buildCache() {
|
||||
return new ConcurrentHashMap<Object, FilterCacheValue<ConcurrentMap<Object, DocSet>>>();
|
||||
}
|
||||
|
||||
protected ConcurrentMap<Filter, DocSet> buildFilterMap() {
|
||||
protected ConcurrentMap<Object, DocSet> buildFilterMap() {
|
||||
return newConcurrentMap();
|
||||
}
|
||||
|
||||
|
@ -92,7 +92,7 @@ public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComp
|
|||
}
|
||||
|
||||
@Override public void finished(IndexReader reader) {
|
||||
FilterCacheValue<ConcurrentMap<Filter, DocSet>> readerValue = cache.remove(reader.getCoreCacheKey());
|
||||
FilterCacheValue<ConcurrentMap<Object, DocSet>> readerValue = cache.remove(reader.getCoreCacheKey());
|
||||
// help soft/weak handling GC
|
||||
if (readerValue != null) {
|
||||
readerValue.value().clear();
|
||||
|
@ -100,7 +100,7 @@ public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComp
|
|||
}
|
||||
|
||||
@Override public void clear(IndexReader reader) {
|
||||
FilterCacheValue<ConcurrentMap<Filter, DocSet>> readerValue = cache.remove(reader.getCoreCacheKey());
|
||||
FilterCacheValue<ConcurrentMap<Object, DocSet>> readerValue = cache.remove(reader.getCoreCacheKey());
|
||||
// help soft/weak handling GC
|
||||
if (readerValue != null) {
|
||||
readerValue.value().clear();
|
||||
|
@ -111,7 +111,7 @@ public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComp
|
|||
long sizeInBytes = 0;
|
||||
long totalCount = 0;
|
||||
int segmentsCount = 0;
|
||||
for (FilterCacheValue<ConcurrentMap<Filter, DocSet>> readerValue : cache.values()) {
|
||||
for (FilterCacheValue<ConcurrentMap<Object, DocSet>> readerValue : cache.values()) {
|
||||
segmentsCount++;
|
||||
for (DocSet docSet : readerValue.value().values()) {
|
||||
sizeInBytes += docSet.sizeInBytes();
|
||||
|
@ -151,27 +151,32 @@ public abstract class AbstractConcurrentMapFilterCache extends AbstractIndexComp
|
|||
}
|
||||
|
||||
@Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
|
||||
FilterCacheValue<ConcurrentMap<Filter, DocSet>> cacheValue = cache.cache.get(reader.getCoreCacheKey());
|
||||
FilterCacheValue<ConcurrentMap<Object, DocSet>> cacheValue = cache.cache.get(reader.getCoreCacheKey());
|
||||
if (cacheValue == null) {
|
||||
LongsLAB longsLAB = null;
|
||||
if (cache.labEnabled) {
|
||||
longsLAB = new LongsLAB(cache.labChunkSizeBytes, cache.labMaxAllocBytes);
|
||||
}
|
||||
cacheValue = new FilterCacheValue<ConcurrentMap<Filter, DocSet>>(cache.buildFilterMap(), longsLAB);
|
||||
FilterCacheValue<ConcurrentMap<Filter, DocSet>> prev = cache.cache.putIfAbsent(reader.getCoreCacheKey(), cacheValue);
|
||||
cacheValue = new FilterCacheValue<ConcurrentMap<Object, DocSet>>(cache.buildFilterMap(), longsLAB);
|
||||
FilterCacheValue<ConcurrentMap<Object, DocSet>> prev = cache.cache.putIfAbsent(reader.getCoreCacheKey(), cacheValue);
|
||||
if (prev != null) {
|
||||
cacheValue = prev;
|
||||
} else {
|
||||
reader.addReaderFinishedListener(cache);
|
||||
}
|
||||
}
|
||||
DocSet docSet = cacheValue.value().get(filter);
|
||||
Object key = filter;
|
||||
if (filter instanceof CacheKeyFilter) {
|
||||
key = ((CacheKeyFilter) filter).cacheKey();
|
||||
}
|
||||
|
||||
DocSet docSet = cacheValue.value().get(key);
|
||||
if (docSet != null) {
|
||||
return docSet;
|
||||
}
|
||||
DocIdSet docIdSet = filter.getDocIdSet(reader);
|
||||
docSet = FilterCacheValue.cacheable(reader, cacheValue.longsLAB(), docIdSet);
|
||||
DocSet prev = cacheValue.value().putIfAbsent(filter, docSet);
|
||||
DocSet prev = cacheValue.value().putIfAbsent(key, docSet);
|
||||
if (prev != null) {
|
||||
docSet = prev;
|
||||
}
|
||||
|
|
|
@ -164,7 +164,11 @@ public abstract class AbstractWeightedFilterCache extends AbstractIndexComponent
|
|||
}
|
||||
|
||||
@Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
|
||||
FilterCacheKey cacheKey = new FilterCacheKey(reader.getCoreCacheKey(), filter);
|
||||
Object filterKey = filter;
|
||||
if (filter instanceof CacheKeyFilter) {
|
||||
filterKey = ((CacheKeyFilter) filter).cacheKey();
|
||||
}
|
||||
FilterCacheKey cacheKey = new FilterCacheKey(reader.getCoreCacheKey(), filterKey);
|
||||
ConcurrentMap<FilterCacheKey, FilterCacheValue<DocSet>> innerCache = cache.cache();
|
||||
|
||||
FilterCacheValue<DocSet> cacheValue = innerCache.get(cacheKey);
|
||||
|
@ -223,30 +227,30 @@ public abstract class AbstractWeightedFilterCache extends AbstractIndexComponent
|
|||
|
||||
public static class FilterCacheKey {
|
||||
private final Object readerKey;
|
||||
private final Filter filter;
|
||||
private final Object filterKey;
|
||||
|
||||
public FilterCacheKey(Object readerKey, Filter filter) {
|
||||
public FilterCacheKey(Object readerKey, Object filterKey) {
|
||||
this.readerKey = readerKey;
|
||||
this.filter = filter;
|
||||
this.filterKey = filterKey;
|
||||
}
|
||||
|
||||
public Object readerKey() {
|
||||
return readerKey;
|
||||
}
|
||||
|
||||
public Filter filter() {
|
||||
return filter;
|
||||
public Object filterKey() {
|
||||
return filterKey;
|
||||
}
|
||||
|
||||
@Override public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
// if (o == null || getClass() != o.getClass()) return false;
|
||||
FilterCacheKey that = (FilterCacheKey) o;
|
||||
return (readerKey == that.readerKey && filter.equals(that.filter));
|
||||
return (readerKey == that.readerKey && filterKey.equals(that.filterKey));
|
||||
}
|
||||
|
||||
@Override public int hashCode() {
|
||||
return readerKey.hashCode() + 31 * filter().hashCode();
|
||||
return readerKey.hashCode() + 31 * filterKey.hashCode();
|
||||
}
|
||||
}
|
||||
}
|
103
modules/elasticsearch/src/main/java/org/elasticsearch/index/cache/filter/support/CacheKeyFilter.java
vendored
Normal file
103
modules/elasticsearch/src/main/java/org/elasticsearch/index/cache/filter/support/CacheKeyFilter.java
vendored
Normal file
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.cache.filter.support;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.elasticsearch.common.Unicode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
public interface CacheKeyFilter {
|
||||
|
||||
public static class Key {
|
||||
|
||||
private final byte[] bytes;
|
||||
|
||||
// we pre-compute the hashCode for better performance (especially in IdCache)
|
||||
private final int hashCode;
|
||||
|
||||
public Key(byte[] bytes) {
|
||||
this.bytes = bytes;
|
||||
this.hashCode = Arrays.hashCode(bytes);
|
||||
}
|
||||
|
||||
public Key(String str) {
|
||||
this(Unicode.fromStringAsBytes(str));
|
||||
}
|
||||
|
||||
public byte[] bytes() {
|
||||
return this.bytes;
|
||||
}
|
||||
|
||||
public String utf8ToString() {
|
||||
return Unicode.fromBytes(bytes);
|
||||
}
|
||||
|
||||
@Override public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o.getClass() != this.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Key bytesWrap = (Key) o;
|
||||
return Arrays.equals(bytes, bytesWrap.bytes);
|
||||
}
|
||||
|
||||
@Override public int hashCode() {
|
||||
return hashCode;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Wrapper extends Filter implements CacheKeyFilter {
|
||||
|
||||
private final Filter filter;
|
||||
|
||||
private final Key key;
|
||||
|
||||
public Wrapper(Filter filter, Key key) {
|
||||
this.filter = filter;
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
@Override public Key cacheKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
@Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
|
||||
return filter.getDocIdSet(reader);
|
||||
}
|
||||
|
||||
@Override public int hashCode() {
|
||||
return filter.hashCode();
|
||||
}
|
||||
|
||||
@Override public boolean equals(Object obj) {
|
||||
return filter.equals(obj);
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return filter.toString();
|
||||
}
|
||||
}
|
||||
|
||||
Key cacheKey();
|
||||
}
|
|
@ -68,7 +68,7 @@ public class WeakFilterCache extends AbstractConcurrentMapFilterCache implements
|
|||
super.close();
|
||||
}
|
||||
|
||||
@Override protected ConcurrentMap<Filter, DocSet> buildFilterMap() {
|
||||
@Override protected ConcurrentMap<Object, DocSet> buildFilterMap() {
|
||||
MapMaker mapMaker = new MapMaker().weakValues();
|
||||
if (maxSize != -1) {
|
||||
mapMaker.maximumSize(maxSize);
|
||||
|
|
|
@ -125,7 +125,7 @@ public class TypeFieldMapper extends AbstractFieldMapper<String> implements Inte
|
|||
}
|
||||
|
||||
@Override public Query fieldQuery(String value, QueryParseContext context) {
|
||||
return new DeletionAwareConstantScoreQuery(context.cacheFilter(fieldFilter(value)));
|
||||
return new DeletionAwareConstantScoreQuery(context.cacheFilter(fieldFilter(value), null));
|
||||
}
|
||||
|
||||
@Override public boolean useFieldQueryWithQueryString() {
|
||||
|
|
|
@ -35,6 +35,7 @@ public class AndFilterBuilder extends BaseFilterBuilder {
|
|||
private ArrayList<FilterBuilder> filters = Lists.newArrayList();
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -60,6 +61,11 @@ public class AndFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public AndFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the filter name for the filter that can be used when searching for matched_filters per hit.
|
||||
*/
|
||||
|
@ -78,6 +84,9 @@ public class AndFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
builder.field("_name", filterName);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.AndFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -49,6 +50,7 @@ public class AndFilterParser implements FilterParser {
|
|||
ArrayList<Filter> filters = newArrayList();
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
|
@ -76,6 +78,8 @@ public class AndFilterParser implements FilterParser {
|
|||
cache = parser.booleanValue();
|
||||
} else if ("_name".equals(currentFieldName)) {
|
||||
filterName = parser.text();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -88,7 +92,7 @@ public class AndFilterParser implements FilterParser {
|
|||
// no need to cache this one
|
||||
Filter filter = new AndFilter(filters);
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
|
|
|
@ -35,6 +35,7 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
|
|||
private ArrayList<Clause> clauses = new ArrayList<Clause>();
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -80,6 +81,11 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public BoolFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject("bool");
|
||||
for (Clause clause : clauses) {
|
||||
|
@ -100,6 +106,9 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.search.OpenFilterClause;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.XBooleanFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -51,6 +52,7 @@ public class BoolFilterParser implements FilterParser {
|
|||
List<OpenFilterClause> clauses = newArrayList();
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
|
@ -85,6 +87,8 @@ public class BoolFilterParser implements FilterParser {
|
|||
cache = parser.booleanValue();
|
||||
} else if ("_name".equals(currentFieldName)) {
|
||||
filterName = parser.text();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -95,7 +99,7 @@ public class BoolFilterParser implements FilterParser {
|
|||
}
|
||||
Filter filter = boolFilter;
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -50,6 +51,7 @@ public class ConstantScoreQueryParser implements QueryParser {
|
|||
Query query = null;
|
||||
float boost = 1.0f;
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token;
|
||||
|
@ -67,6 +69,8 @@ public class ConstantScoreQueryParser implements QueryParser {
|
|||
boost = parser.floatValue();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -77,7 +81,7 @@ public class ConstantScoreQueryParser implements QueryParser {
|
|||
if (filter != null) {
|
||||
// cache the filter if possible needed
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
Query query1 = new DeletionAwareConstantScoreQuery(filter);
|
||||
|
|
|
@ -79,7 +79,7 @@ public class ExistsFilterParser implements FilterParser {
|
|||
}
|
||||
|
||||
// we always cache this one, really does not change...
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -49,6 +50,7 @@ public class FQueryFilterParser implements FilterParser {
|
|||
|
||||
Query query = null;
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
|
@ -65,12 +67,14 @@ public class FQueryFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
Filter filter = new QueryWrapperFilter(query);
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -50,6 +51,7 @@ public class FilteredQueryParser implements QueryParser {
|
|||
Filter filter = null;
|
||||
float boost = 1.0f;
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token;
|
||||
|
@ -65,8 +67,10 @@ public class FilteredQueryParser implements QueryParser {
|
|||
} else if (token.isValue()) {
|
||||
if ("boost".equals(currentFieldName)) {
|
||||
boost = parser.floatValue();
|
||||
} else if ("cache".equals(currentFieldName)) {
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -79,7 +83,7 @@ public class FilteredQueryParser implements QueryParser {
|
|||
|
||||
// cache if required
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
// if its a match_all query, use constant_score
|
||||
|
|
|
@ -40,6 +40,7 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
|
|||
private String bottomRightGeohash;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -99,6 +100,11 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public GeoBoundingBoxFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(GeoBoundingBoxFilterParser.NAME);
|
||||
|
||||
|
@ -126,6 +132,9 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.index.query;
|
|||
import org.apache.lucene.search.Filter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.geo.GeoPointFieldDataType;
|
||||
|
@ -51,6 +52,7 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
GeoBoundingBoxFilter.Point topLeft = new GeoBoundingBoxFilter.Point();
|
||||
GeoBoundingBoxFilter.Point bottomRight = new GeoBoundingBoxFilter.Point();
|
||||
|
@ -140,6 +142,8 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -158,7 +162,7 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = new GeoBoundingBoxFilter(topLeft, bottomRight, fieldName, parseContext.indexCache().fieldData());
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
filter = wrapSmartNameFilter(filter, parseContext.smartFieldMappers(fieldName), parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -43,6 +43,7 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
|
|||
private GeoDistance geoDistance;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -102,6 +103,11 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public GeoDistanceFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(GeoDistanceFilterParser.NAME);
|
||||
if (geohash != null) {
|
||||
|
@ -119,6 +125,9 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.unit.DistanceUnit;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.geo.GeoPointFieldDataType;
|
||||
|
@ -62,6 +63,7 @@ public class GeoDistanceFilterParser implements FilterParser {
|
|||
XContentParser.Token token;
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
double lat = 0;
|
||||
|
@ -128,6 +130,8 @@ public class GeoDistanceFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
} else {
|
||||
// assume the value is the actual value
|
||||
String value = parser.text();
|
||||
|
@ -163,7 +167,7 @@ public class GeoDistanceFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = new GeoDistanceFilter(lat, lon, distance, geoDistance, fieldName, parseContext.indexCache().fieldData());
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
filter = wrapSmartNameFilter(filter, parseContext.smartFieldMappers(fieldName), parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -45,6 +45,7 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
|
|||
private GeoDistance geoDistance;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -138,6 +139,11 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public GeoDistanceRangeFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(GeoDistanceRangeFilterParser.NAME);
|
||||
if (geohash != null) {
|
||||
|
@ -158,6 +164,9 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.unit.DistanceUnit;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.geo.GeoPointFieldDataType;
|
||||
|
@ -62,6 +63,7 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
|
|||
XContentParser.Token token;
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
double lat = 0;
|
||||
|
@ -177,6 +179,8 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
} else {
|
||||
// assume the value is the actual value
|
||||
String value = parser.text();
|
||||
|
@ -219,7 +223,7 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = new GeoDistanceRangeFilter(lat, lon, from, to, includeLower, includeUpper, geoDistance, fieldName, parseContext.indexCache().fieldData());
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
filter = wrapSmartNameFilter(filter, parseContext.smartFieldMappers(fieldName), parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -37,6 +37,7 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
|
|||
private final List<GeoPolygonFilter.Point> points = Lists.newArrayList();
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -77,6 +78,11 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public GeoPolygonFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(GeoPolygonFilterParser.NAME);
|
||||
|
||||
|
@ -94,6 +100,9 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.collect.Lists;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.geo.GeoPointFieldDataType;
|
||||
|
@ -64,6 +65,7 @@ public class GeoPolygonFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
List<GeoPolygonFilter.Point> points = Lists.newArrayList();
|
||||
|
||||
|
@ -137,6 +139,8 @@ public class GeoPolygonFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -157,7 +161,7 @@ public class GeoPolygonFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = new GeoPolygonFilter(points.toArray(new GeoPolygonFilter.Point[points.size()]), fieldName, parseContext.indexCache().fieldData());
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
filter = wrapSmartNameFilter(filter, parseContext.smartFieldMappers(fieldName), parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -89,7 +89,7 @@ public class HasChildFilterParser implements FilterParser {
|
|||
String parentType = childDocMapper.parentFieldMapper().type();
|
||||
|
||||
// wrap the query with type query
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter()));
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter(), null));
|
||||
|
||||
SearchContext searchContext = SearchContext.current();
|
||||
|
||||
|
|
|
@ -90,7 +90,7 @@ public class HasChildQueryParser implements QueryParser {
|
|||
|
||||
query.setBoost(boost);
|
||||
// wrap the query with type query
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter()));
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter(), null));
|
||||
|
||||
SearchContext searchContext = SearchContext.current();
|
||||
HasChildFilter childFilter = new HasChildFilter(query, scope, childType, parentType, searchContext);
|
||||
|
|
|
@ -80,10 +80,10 @@ public class MissingFilterParser implements FilterParser {
|
|||
}
|
||||
|
||||
// we always cache this one, really does not change... (exists)
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
filter = new NotFilter(filter);
|
||||
// cache the not filter as well, so it will be faster
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -33,6 +33,7 @@ public class NestedFilterBuilder extends BaseFilterBuilder {
|
|||
private String scope;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
private String filterName;
|
||||
|
||||
public NestedFilterBuilder(String path, QueryBuilder queryBuilder) {
|
||||
|
@ -61,6 +62,11 @@ public class NestedFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public NestedFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the filter name for the filter that can be used when searching for matched_filters per hit.
|
||||
*/
|
||||
|
@ -88,6 +94,9 @@ public class NestedFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.search.QueryWrapperFilter;
|
|||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.object.ObjectMapper;
|
||||
import org.elasticsearch.index.search.nested.BlockJoinQuery;
|
||||
|
@ -55,6 +56,7 @@ public class NestedFilterParser implements FilterParser {
|
|||
String scope = null;
|
||||
String path = null;
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String filterName = null;
|
||||
|
||||
// we need a late binding filter so we can inject a parent nested filter inner nested queries
|
||||
|
@ -85,6 +87,8 @@ public class NestedFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -113,7 +117,7 @@ public class NestedFilterParser implements FilterParser {
|
|||
throw new QueryParsingException(parseContext.index(), "[nested] nested object under path [" + path + "] is not of nested type");
|
||||
}
|
||||
|
||||
Filter childFilter = parseContext.cacheFilter(objectMapper.nestedTypeFilter());
|
||||
Filter childFilter = parseContext.cacheFilter(objectMapper.nestedTypeFilter(), null);
|
||||
usAsParentFilter.filter = childFilter;
|
||||
// wrap the child query to only work on the nested path type
|
||||
query = new FilteredQuery(query, childFilter);
|
||||
|
@ -125,7 +129,7 @@ public class NestedFilterParser implements FilterParser {
|
|||
// filter based on the type...
|
||||
parentFilter = mapper.docMapper().typeFilter();
|
||||
}
|
||||
parentFilter = parseContext.cacheFilter(parentFilter);
|
||||
parentFilter = parseContext.cacheFilter(parentFilter, null);
|
||||
}
|
||||
|
||||
// restore the thread local one...
|
||||
|
@ -139,7 +143,7 @@ public class NestedFilterParser implements FilterParser {
|
|||
|
||||
Filter joinFilter = new QueryWrapperFilter(joinQuery);
|
||||
if (cache) {
|
||||
joinFilter = parseContext.cacheFilter(joinFilter);
|
||||
joinFilter = parseContext.cacheFilter(joinFilter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, joinFilter);
|
||||
|
|
|
@ -122,7 +122,7 @@ public class NestedQueryParser implements QueryParser {
|
|||
throw new QueryParsingException(parseContext.index(), "[nested] nested object under path [" + path + "] is not of nested type");
|
||||
}
|
||||
|
||||
Filter childFilter = parseContext.cacheFilter(objectMapper.nestedTypeFilter());
|
||||
Filter childFilter = parseContext.cacheFilter(objectMapper.nestedTypeFilter(), null);
|
||||
usAsParentFilter.filter = childFilter;
|
||||
// wrap the child query to only work on the nested path type
|
||||
query = new FilteredQuery(query, childFilter);
|
||||
|
@ -134,7 +134,7 @@ public class NestedQueryParser implements QueryParser {
|
|||
// filter based on the type...
|
||||
parentFilter = mapper.docMapper().typeFilter();
|
||||
}
|
||||
parentFilter = parseContext.cacheFilter(parentFilter);
|
||||
parentFilter = parseContext.cacheFilter(parentFilter, null);
|
||||
}
|
||||
|
||||
// restore the thread local one...
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.NotFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -45,6 +46,7 @@ public class NotFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = null;
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
|
@ -64,6 +66,8 @@ public class NotFilterParser implements FilterParser {
|
|||
cache = parser.booleanValue();
|
||||
} else if ("_name".equals(currentFieldName)) {
|
||||
filterName = parser.text();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -74,7 +78,7 @@ public class NotFilterParser implements FilterParser {
|
|||
|
||||
Filter notFilter = new NotFilter(filter);
|
||||
if (cache) {
|
||||
notFilter = parseContext.cacheFilter(notFilter);
|
||||
notFilter = parseContext.cacheFilter(notFilter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, notFilter);
|
||||
|
|
|
@ -43,6 +43,7 @@ public class NumericRangeFilterBuilder extends BaseFilterBuilder {
|
|||
private boolean includeUpper = true;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -347,6 +348,11 @@ public class NumericRangeFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public NumericRangeFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(NumericRangeFilterParser.NAME);
|
||||
|
||||
|
@ -363,6 +369,9 @@ public class NumericRangeFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.index.query;
|
|||
import org.apache.lucene.search.Filter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
|
||||
|
@ -48,6 +49,7 @@ public class NumericRangeFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = false; // default to false, since its using fielddata cache
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
String from = null;
|
||||
String to = null;
|
||||
|
@ -94,6 +96,8 @@ public class NumericRangeFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -111,7 +115,7 @@ public class NumericRangeFilterParser implements FilterParser {
|
|||
Filter filter = ((NumberFieldMapper) mapper).rangeFilter(parseContext.indexCache().fieldData(), from, to, includeLower, includeUpper);
|
||||
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
if (filterName != null) {
|
||||
|
|
|
@ -35,6 +35,7 @@ public class OrFilterBuilder extends BaseFilterBuilder {
|
|||
private ArrayList<FilterBuilder> filters = Lists.newArrayList();
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -60,6 +61,11 @@ public class OrFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public OrFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
public OrFilterBuilder filterName(String filterName) {
|
||||
this.filterName = filterName;
|
||||
return this;
|
||||
|
@ -75,6 +81,9 @@ public class OrFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
builder.field("_name", filterName);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.OrFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -49,6 +50,7 @@ public class OrFilterParser implements FilterParser {
|
|||
ArrayList<Filter> filters = newArrayList();
|
||||
|
||||
boolean cache = false;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
|
@ -76,6 +78,8 @@ public class OrFilterParser implements FilterParser {
|
|||
cache = parser.booleanValue();
|
||||
} else if ("_name".equals(currentFieldName)) {
|
||||
filterName = parser.text();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -88,7 +92,7 @@ public class OrFilterParser implements FilterParser {
|
|||
// no need to cache this one
|
||||
Filter filter = new OrFilter(filters);
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
|
|
|
@ -36,6 +36,7 @@ public class PrefixFilterBuilder extends BaseFilterBuilder {
|
|||
private final String prefix;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -67,6 +68,11 @@ public class PrefixFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public PrefixFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override public void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(PrefixFilterParser.NAME);
|
||||
builder.field(name, prefix);
|
||||
|
@ -76,6 +82,9 @@ public class PrefixFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.apache.lucene.search.PrefixFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -48,6 +49,7 @@ public class PrefixFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = true;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
String value = null;
|
||||
|
||||
|
@ -62,6 +64,8 @@ public class PrefixFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
} else {
|
||||
fieldName = currentFieldName;
|
||||
value = parser.text();
|
||||
|
@ -84,7 +88,7 @@ public class PrefixFilterParser implements FilterParser {
|
|||
Filter filter = new PrefixFilter(new Term(fieldName, value));
|
||||
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.cache.IndexCache;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.engine.IndexEngine;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.FieldMappers;
|
||||
|
@ -116,7 +117,10 @@ public class QueryParseContext {
|
|||
return multiFieldQueryParser;
|
||||
}
|
||||
|
||||
public Filter cacheFilter(Filter filter) {
|
||||
public Filter cacheFilter(Filter filter, @Nullable CacheKeyFilter.Key cacheKey) {
|
||||
if (cacheKey != null) {
|
||||
filter = new CacheKeyFilter.Wrapper(filter, cacheKey);
|
||||
}
|
||||
return indexQueryParser.indexCache.filter().cache(filter);
|
||||
}
|
||||
|
||||
|
|
|
@ -41,6 +41,7 @@ public class RangeFilterBuilder extends BaseFilterBuilder {
|
|||
private boolean includeUpper = true;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -354,6 +355,11 @@ public class RangeFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public RangeFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(RangeFilterParser.NAME);
|
||||
|
||||
|
@ -370,6 +376,9 @@ public class RangeFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.apache.lucene.search.TermRangeFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -47,6 +48,7 @@ public class RangeFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = true;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
String from = null;
|
||||
String to = null;
|
||||
|
@ -93,6 +95,8 @@ public class RangeFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -113,7 +117,7 @@ public class RangeFilterParser implements FilterParser {
|
|||
}
|
||||
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
|
|
@ -38,6 +38,7 @@ public class ScriptFilterBuilder extends BaseFilterBuilder {
|
|||
private String lang;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -86,6 +87,11 @@ public class ScriptFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public ScriptFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(ScriptFilterParser.NAME);
|
||||
builder.field("script", script);
|
||||
|
@ -101,6 +107,9 @@ public class ScriptFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.common.collect.Maps;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.docset.GetDocSet;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
@ -54,6 +55,7 @@ public class ScriptFilterParser implements FilterParser {
|
|||
XContentParser.Token token;
|
||||
|
||||
boolean cache = false; // no need to cache it by default, changes a lot?
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
// also, when caching, since its isCacheable is false, will result in loading all bit set...
|
||||
String script = null;
|
||||
String scriptLang = null;
|
||||
|
@ -77,6 +79,8 @@ public class ScriptFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -90,7 +94,7 @@ public class ScriptFilterParser implements FilterParser {
|
|||
|
||||
Filter filter = new ScriptFilter(scriptLang, script, params, parseContext.scriptService());
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
|
|
|
@ -35,6 +35,7 @@ public class TermFilterBuilder extends BaseFilterBuilder {
|
|||
private final Object value;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -115,6 +116,11 @@ public class TermFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public TermFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override public void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(TermFilterParser.NAME);
|
||||
builder.field(name, value);
|
||||
|
@ -124,6 +130,9 @@ public class TermFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.TermFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -48,6 +49,7 @@ public class TermFilterParser implements FilterParser {
|
|||
XContentParser parser = parseContext.parser();
|
||||
|
||||
boolean cache = true; // since usually term filter is on repeating terms, cache it by default
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
String fieldName = null;
|
||||
String value = null;
|
||||
|
||||
|
@ -62,6 +64,8 @@ public class TermFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
} else {
|
||||
fieldName = currentFieldName;
|
||||
value = parser.text();
|
||||
|
@ -89,7 +93,7 @@ public class TermFilterParser implements FilterParser {
|
|||
}
|
||||
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
|
|
@ -35,6 +35,7 @@ public class TermsFilterBuilder extends BaseFilterBuilder {
|
|||
private final Object[] values;
|
||||
|
||||
private Boolean cache;
|
||||
private String cacheKey;
|
||||
|
||||
private String filterName;
|
||||
|
||||
|
@ -131,6 +132,11 @@ public class TermsFilterBuilder extends BaseFilterBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public TermsFilterBuilder cacheKey(String cacheKey) {
|
||||
this.cacheKey = cacheKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override public void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(TermsFilterParser.NAME);
|
||||
builder.startArray(name);
|
||||
|
@ -145,6 +151,9 @@ public class TermsFilterBuilder extends BaseFilterBuilder {
|
|||
if (cache != null) {
|
||||
builder.field("_cache", cache);
|
||||
}
|
||||
if (cacheKey != null) {
|
||||
builder.field("_cache_key", cacheKey);
|
||||
}
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.apache.lucene.search.PublicTermsFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
|
@ -53,6 +54,7 @@ public class TermsFilterParser implements FilterParser {
|
|||
PublicTermsFilter termsFilter = new PublicTermsFilter();
|
||||
String filterName = null;
|
||||
String currentFieldName = null;
|
||||
CacheKeyFilter.Key cacheKey = null;
|
||||
XContentParser.Token token;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
|
@ -83,13 +85,15 @@ public class TermsFilterParser implements FilterParser {
|
|||
filterName = parser.text();
|
||||
} else if ("_cache".equals(currentFieldName)) {
|
||||
cache = parser.booleanValue();
|
||||
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
|
||||
cacheKey = new CacheKeyFilter.Key(parser.text());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Filter filter = termsFilter;
|
||||
if (cache) {
|
||||
filter = parseContext.cacheFilter(filter);
|
||||
filter = parseContext.cacheFilter(filter, cacheKey);
|
||||
}
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
|
|
|
@ -98,7 +98,7 @@ public class TopChildrenQueryParser implements QueryParser {
|
|||
|
||||
query.setBoost(boost);
|
||||
// wrap the query with type query
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter()));
|
||||
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter(), null));
|
||||
|
||||
SearchContext searchContext = SearchContext.current();
|
||||
TopChildrenQuery childQuery = new TopChildrenQuery(query, scope, childType, parentType, scoreType, factor, incrementalFactor);
|
||||
|
|
|
@ -66,6 +66,6 @@ public class TypeFilterParser implements FilterParser {
|
|||
} else {
|
||||
filter = documentMapper.typeFilter();
|
||||
}
|
||||
return parseContext.cacheFilter(filter);
|
||||
return parseContext.cacheFilter(filter, null);
|
||||
}
|
||||
}
|
|
@ -47,7 +47,7 @@ public final class QueryParsers {
|
|||
return query;
|
||||
}
|
||||
DocumentMapper docMapper = smartFieldMappers.docMapper();
|
||||
return new FilteredQuery(query, parseContext.cacheFilter(docMapper.typeFilter()));
|
||||
return new FilteredQuery(query, parseContext.cacheFilter(docMapper.typeFilter(), null));
|
||||
}
|
||||
|
||||
public static Filter wrapSmartNameFilter(Filter filter, @Nullable MapperService.SmartNameFieldMappers smartFieldMappers,
|
||||
|
@ -59,6 +59,6 @@ public final class QueryParsers {
|
|||
return filter;
|
||||
}
|
||||
DocumentMapper docMapper = smartFieldMappers.docMapper();
|
||||
return new AndFilter(ImmutableList.of(parseContext.cacheFilter(docMapper.typeFilter()), filter));
|
||||
return new AndFilter(ImmutableList.of(parseContext.cacheFilter(docMapper.typeFilter(), null), filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.testng.annotations.AfterClass;
|
|||
import org.testng.annotations.BeforeClass;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.*;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.*;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.*;
|
||||
|
@ -304,4 +306,26 @@ public class SimpleQueryTests extends AbstractNodesTests {
|
|||
|
||||
}
|
||||
|
||||
@Test public void testFiltersWithCustomCacheKey() throws Exception {
|
||||
client.admin().indices().prepareDelete().execute().actionGet();
|
||||
|
||||
client.prepareIndex("test", "type1", "1").setSource("field1", "value1").execute().actionGet();
|
||||
client.admin().indices().prepareRefresh().execute().actionGet();
|
||||
|
||||
SearchResponse searchResponse = client.prepareSearch("test").setQuery(constantScoreQuery(termsFilter("field1", "value1").cacheKey("test1"))).execute().actionGet();
|
||||
assertThat("Failures " + Arrays.toString(searchResponse.shardFailures()), searchResponse.shardFailures().length, equalTo(0));
|
||||
assertThat(searchResponse.hits().totalHits(), equalTo(1l));
|
||||
|
||||
searchResponse = client.prepareSearch("test").setQuery(constantScoreQuery(termsFilter("field1", "value1").cacheKey("test1"))).execute().actionGet();
|
||||
assertThat("Failures " + Arrays.toString(searchResponse.shardFailures()), searchResponse.shardFailures().length, equalTo(0));
|
||||
assertThat(searchResponse.hits().totalHits(), equalTo(1l));
|
||||
|
||||
searchResponse = client.prepareSearch("test").setQuery(constantScoreQuery(termsFilter("field1", "value1"))).execute().actionGet();
|
||||
assertThat("Failures " + Arrays.toString(searchResponse.shardFailures()), searchResponse.shardFailures().length, equalTo(0));
|
||||
assertThat(searchResponse.hits().totalHits(), equalTo(1l));
|
||||
|
||||
searchResponse = client.prepareSearch("test").setQuery(constantScoreQuery(termsFilter("field1", "value1"))).execute().actionGet();
|
||||
assertThat("Failures " + Arrays.toString(searchResponse.shardFailures()), searchResponse.shardFailures().length, equalTo(0));
|
||||
assertThat(searchResponse.hits().totalHits(), equalTo(1l));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue