Core: Cut over to the Lucene filter cache.

This removes Elasticsearch's filter cache and uses Lucene's instead. It has some
implications:
 - custom cache keys (`_cache_key`) are unsupported
 - decisions are made internally and can't be overridden by users ('_cache`)
 - not only filters can be cached but also all queries that do not need scores
 - parent/child queries can now be cached, however cached entries are only
   valid for the current top-level reader so in practice it will likely only
   be used on read-only indices
 - the cache deduplicates filters, which plays nicer with large keys (eg. `terms`)
 - better stats: we already had ram usage and evictions, but now also hit count,
   miss count, lookup count, number of cached doc id sets and current number of
   doc id sets in the cache
 - dynamically changing the filter cache size is not supported anymore

Internally, an important change is that it removes the NoCacheFilter infrastructure
in favour of making Query.rewrite specializing the query for the current reader so
that it will only be cached on this reader (look for IndexCacheableQuery).

Note that consuming filters with the query API (createWeight/scorer) instead of
the filter API (getDocIdSet) is important for parent/child queries because
otherwise a QueryWrapperFilter(ParentQuery) would run the wrapped query per
segment while relations might be cross segments.
This commit is contained in:
Adrien Grand 2015-04-24 20:59:22 +02:00
parent df1914cb21
commit b72f27a410
200 changed files with 1734 additions and 2970 deletions

View File

@ -39,9 +39,6 @@ org.apache.lucene.index.IndexReader#decRef()
org.apache.lucene.index.IndexReader#incRef()
org.apache.lucene.index.IndexReader#tryIncRef()
@defaultMessage QueryWrapperFilter is cacheable by default - use Queries#wrap instead
org.apache.lucene.search.QueryWrapperFilter#<init>(org.apache.lucene.search.Query)
@defaultMessage Pass the precision step from the mappings explicitly instead
org.apache.lucene.search.NumericRangeQuery#newDoubleRange(java.lang.String,java.lang.Double,java.lang.Double,boolean,boolean)
org.apache.lucene.search.NumericRangeQuery#newFloatRange(java.lang.String,java.lang.Float,java.lang.Float,boolean,boolean)

View File

@ -153,9 +153,6 @@ due to forced awareness or allocation filtering.
`indices.cache.filter.size`::
See <<index-modules-cache>>
`indices.cache.filter.expire` (time)::
See <<index-modules-cache>>
[float]
==== TTL interval

View File

@ -418,6 +418,12 @@ favour or `bool`.
The `execution` option of the `terms` filter is now deprecated and ignored if
provided.
The `_cache` and `_cache_key` parameters of filters are deprecated in the REST
layer and removed in the Java API. In case they are specified they will be
ignored. Instead filters are always used as their own cache key and elasticsearch
makes decisions by itself about whether it should cache filters based on how
often they are used.
=== Snapshot and Restore
The obsolete parameters `expand_wildcards_open` and `expand_wildcards_close` are no longer

View File

@ -10,85 +10,14 @@ As a general rule, filters should be used instead of queries:
[[caching]]
=== Filters and Caching
Filters can be a great candidate for caching. Caching the result of a
filter does not require a lot of memory, and will cause other queries
executing against the same filter (same parameters) to be blazingly
fast.
Filters can be a great candidate for caching. Caching the document set that
a filter matches does not require much memory and can help improve
execution speed of queries.
However the cost of caching is not the same for all filters. For
instance some filters are already fast out of the box while caching could
add significant overhead, and some filters produce results that are already
cacheable so caching them is just a matter of putting the result in the
cache.
The default caching policy, `_cache: auto`, tracks the 1000 most recently
used filters on a per-index basis and makes decisions based on their
frequency.
[float]
==== Filters that read directly the index structure
Some filters can directly read the index structure and potentially jump
over large sequences of documents that are not worth evaluating (for
instance when these documents do not match the query). Caching these
filters introduces overhead given that all documents that the filter
matches need to be consumed in order to be loaded into the cache.
These filters, which include the <<query-dsl-term-filter,term>> and
<<query-dsl-term-query,query>> filters, are only cached after they
appear 5 times or more in the history of the 1000 most recently used
filters.
[float]
==== Filters that produce results that are already cacheable
Some filters produce results that are already cacheable, and the difference
between caching and not caching them is the act of placing the result in
the cache or not. These filters, which include the
<<query-dsl-terms-filter,terms>>,
<<query-dsl-prefix-filter,prefix>>, and
<<query-dsl-range-filter,range>> filters, are by default cached after they
appear twice or more in the history of the most 1000 recently used filters.
[float]
==== Computational filters
Some filters need to run some computation in order to figure out whether
a given document matches a filter. These filters, which include the geo and
<<query-dsl-script-filter,script>> filters, but also the
<<query-dsl-terms-filter,terms>> and <<query-dsl-range-filter,range>>
filters when using the `fielddata` execution mode are never cached by default,
as it would require to evaluate the filter on all documents in your indices
while they can otherwise be only evaluated on documents that match the query.
[float]
==== Compound filters
The last type of filters are those working with other filters, and includes
the <<query-dsl-bool-filter,bool>>,
<<query-dsl-and-filter,and>>,
<<query-dsl-not-filter,not>> and
<<query-dsl-or-filter,or>> filters.
There is no general rule about these filters. Depending on the filters that
they wrap, they will sometimes return a filter that dynamically evaluates the
sub filters and sometimes evaluate the sub filters eagerly in order to return
a result that is already cacheable, so depending on the case, these filters
will be cached after they appear 2+ or 5+ times in the history of the most
1000 recently used filters.
[float]
==== Overriding the default behaviour
All filters allow to set `_cache` element on them to explicitly control
caching. It accepts 3 values: `true` in order to cache the filter, `false`
to make sure that the filter will not be cached, and `auto`, which is the
default and will decide on whether to cache the filter based on the cost
to cache it and how often it has been used as explained above.
Filters also allow to set `_cache_key` which will be used as the
caching key for that filter. This can be handy when using very large
filters (like a terms filter with many elements in it).
Elasticsearch decides to cache filters based on how often they are used. For
this reason you might occasionally see better performance by splitting
complex filters into a static part that Elasticsearch will cache and a dynamic
part which is least costly than the original filter.
include::filters/and-filter.asciidoc[]

View File

@ -32,40 +32,3 @@ filters. Can be placed within queries that accept a filter.
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is only cached by default if there is evidence of
reuse. It is possible to opt-in explicitely for caching by setting `_cache`
to `true`. Since the `_cache` element requires to be set on the `and` filter
itself, the structure then changes a bit to have the filters provided within a
`filters` element:
[source,js]
--------------------------------------------------
{
"filtered" : {
"query" : {
"term" : { "name.first" : "shay" }
},
"filter" : {
"and" : {
"filters": [
{
"range" : {
"postDate" : {
"from" : "2010-03-01",
"to" : "2010-04-01"
}
}
},
{
"prefix" : { "name.second" : "ba" }
}
],
"_cache" : true
}
}
}
}
--------------------------------------------------

View File

@ -230,11 +230,3 @@ are not supported. Here is an example:
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is not cached by default. The `_cache` can be
set to `true` to cache the *result* of the filter. This is handy when
the same bounding box parameters are used on several (many) other
queries. Note, the process of caching the first execution is higher when
caching (since it needs to satisfy different queries).

View File

@ -172,11 +172,3 @@ The `geo_distance` filter can work with multiple locations / points per
document. Once a single location / point matches the filter, the
document will be included in the filter.
[float]
==== Caching
The result of the filter is not cached by default. The `_cache` can be
set to `true` to cache the *result* of the filter. This is handy when
the same point and distance parameters are used on several (many) other
queries. Note, the process of caching the first execution is higher when
caching (since it needs to satisfy different queries).

View File

@ -116,11 +116,3 @@ The filter *requires* the
<<mapping-geo-point-type,geo_point>> type to be
set on the relevant field.
[float]
==== Caching
The result of the filter is not cached by default. The `_cache` can be
set to `true` to cache the *result* of the filter. This is handy when
the same points parameters are used on several (many) other queries.
Note, the process of caching the first execution is higher when caching
(since it needs to satisfy different queries).

View File

@ -110,12 +110,3 @@ shape:
}
--------------------------------------------------
[float]
==== Caching
The result of the Filter is not cached by default. Setting `_cache` to
`true` will mean the results of the Filter will be cached. Since shapes
can contain 10s-100s of coordinates and any one differing means a new
shape, it may make sense to only using caching when you are sure that
the shapes will remain reasonably static.

View File

@ -61,10 +61,3 @@ next to the given cell.
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is not cached by default. The
`_cache` parameter can be set to `true` to turn caching on.
By default the filter uses the resulting geohash cells as a cache key.
This can be changed by using the `_cache_key` option.

View File

@ -88,9 +88,3 @@ APIS, eg:
curl -XGET "http://localhost:9200/_stats/id_cache?pretty&human"
--------------------------------------------------
[float]
==== Caching
The `has_child` filter cannot be cached in the filter cache. The `_cache`
and `_cache_key` options are a no-op in this filter. Also any filter that
wraps the `has_child` filter either directly or indirectly will not be cached.

View File

@ -63,9 +63,3 @@ APIS, eg:
curl -XGET "http://localhost:9200/_stats/id_cache?pretty&human"
--------------------------------------------------
[float]
==== Caching
The `has_parent` filter cannot be cached in the filter cache. The `_cache`
and `_cache_key` options are a no-op in this filter. Also any filter that
wraps the `has_parent` filter either directly or indirectly will not be cached.

View File

@ -2,10 +2,7 @@
=== Nested Filter
A `nested` filter works in a similar fashion to the
<<query-dsl-nested-query,nested>> query, except it's
used as a filter. It follows exactly the same structure, but also allows
to cache the results (set `_cache` to `true`), and have it named (set
the `_name` value). For example:
<<query-dsl-nested-query,nested>> query. For example:
[source,js]
--------------------------------------------------
@ -26,8 +23,7 @@ the `_name` value). For example:
}
]
}
},
"_cache" : true
}
}
}
}

View File

@ -50,33 +50,3 @@ Or, in a longer form with a `filter` element:
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is only cached if there is evidence of reuse.
The `_cache` can be set to `true` in order to cache it (though usually
not needed). Here is an example:
[source,js]
--------------------------------------------------
{
"filtered" : {
"query" : {
"term" : { "name.first" : "shay" }
},
"filter" : {
"not" : {
"filter" : {
"range" : {
"postDate" : {
"from" : "2010-03-01",
"to" : "2010-04-01"
}
}
},
"_cache" : true
}
}
}
}
--------------------------------------------------

View File

@ -27,36 +27,3 @@ filters. Can be placed within queries that accept a filter.
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is only cached by default if there is evidence
of reuse. The `_cache` can be
set to `true` in order to cache it (though usually not needed). Since
the `_cache` element requires to be set on the `or` filter itself, the
structure then changes a bit to have the filters provided within a
`filters` element:
[source,js]
--------------------------------------------------
{
"filtered" : {
"query" : {
"term" : { "name.first" : "shay" }
},
"filter" : {
"or" : {
"filters" : [
{
"term" : { "name.second" : "banon" }
},
{
"term" : { "name.nick" : "kimchy" }
}
],
"_cache" : true
}
}
}
}
--------------------------------------------------

View File

@ -16,22 +16,3 @@ a filter. Can be placed within queries that accept a filter.
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is cached by default if there is evidence of reuse.
The `_cache` can be set to `true` in order to cache it. Here is an example:
[source,js]
--------------------------------------------------
{
"constant_score" : {
"filter" : {
"prefix" : {
"user" : "ki",
"_cache" : true
}
}
}
}
--------------------------------------------------

View File

@ -19,34 +19,3 @@ that accept a filter.
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is only cached by default if there is evidence of reuse.
The `_cache` can be
set to `true` to cache the *result* of the filter. This is handy when
the same query is used on several (many) other queries. Note, the
process of caching the first execution is higher when not caching (since
it needs to satisfy different queries).
Setting the `_cache` element requires a different format for the
`query`:
[source,js]
--------------------------------------------------
{
"constantScore" : {
"filter" : {
"fquery" : {
"query" : {
"query_string" : {
"query" : "this AND that OR thus"
}
},
"_cache" : true
}
}
}
}
--------------------------------------------------

View File

@ -95,11 +95,3 @@ requires more memory, so make sure you have sufficient memory on your nodes in
order to use this execution mode. It usually makes sense to use it on fields
you're already aggregating or sorting by.
[float]
==== Caching
The result of the filter is only cached by default if there is evidence of reuse. The
`_cache` can be set to `false` to turn it off.
Having the `now` expression used without rounding will make the filter unlikely to be
cached since reuse is very unlikely.

View File

@ -51,9 +51,7 @@ You have to enable caching explicitly in order to have the
"flags" : "INTERSECTION|COMPLEMENT|EMPTY",
"max_determinized_states": 20000
},
"_name":"test",
"_cache" : true,
"_cache_key" : "key"
"_name":"test"
}
}
}

View File

@ -43,11 +43,3 @@ to use the ability to pass parameters to the script itself, for example:
}
----------------------------------------------
[float]
==== Caching
The result of the filter is not cached by default. The `_cache` can be
set to `true` to cache the *result* of the filter. This is handy when
the same script and parameters are used on several (many) other queries.
Note, the process of caching the first execution is higher when caching
(since it needs to satisfy different queries).

View File

@ -17,22 +17,3 @@ accept a filter, for example:
}
--------------------------------------------------
[float]
==== Caching
The result of the filter is only cached by default if there is evidence of reuse.
The `_cache` can be set to `false` to turn it off. Here is an example:
[source,js]
--------------------------------------------------
{
"constant_score" : {
"filter" : {
"term" : {
"user" : "kimchy",
"_cache" : false
}
}
}
}
--------------------------------------------------

View File

@ -18,13 +18,6 @@ Filters documents that have fields that match any of the provided terms
The `terms` filter is also aliased with `in` as the filter name for
simpler usage.
[float]
==== Caching
The result of the filter is cached if there is evidence of reuse. It is
possible to enable caching explicitely by setting `_cache` to `true` and
to disable caching by setting `_cache` to `false`.
[float]
==== Terms lookup mechanism
@ -93,8 +86,7 @@ curl -XGET localhost:9200/tweets/_search -d '{
"type" : "user",
"id" : "2",
"path" : "followers"
},
"_cache_key" : "user_2_friends"
}
}
}
}
@ -102,10 +94,6 @@ curl -XGET localhost:9200/tweets/_search -d '{
}'
--------------------------------------------------
If there are lots of matching values, then `_cache_key` is recommended to be set,
so that the filter cache will not store a reference to the potentially heavy
terms filter.
The structure of the external terms document can also include array of
inner objects, for example:

View File

@ -32,10 +32,6 @@
"type" : "boolean",
"description" : "Clear filter caches"
},
"filter_keys": {
"type" : "boolean",
"description" : "A comma-separated list of keys to clear when using the `filter_cache` parameter (default: all)"
},
"id": {
"type" : "boolean",
"description" : "Clear ID caches for parent/child"

View File

@ -18,9 +18,9 @@
package org.apache.lucene.search.postingshighlight;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Strings;
@ -91,8 +91,7 @@ public final class CustomPostingsHighlighter extends XPostingsHighlighter {
/*
Our own api to highlight a single document field, passing in the query terms, and get back our own Snippet object
*/
public Snippet[] highlightDoc(String field, BytesRef[] terms, IndexSearcher searcher, int docId, int maxPassages) throws IOException {
IndexReader reader = searcher.getIndexReader();
public Snippet[] highlightDoc(String field, BytesRef[] terms, IndexReader reader, int docId, int maxPassages) throws IOException {
IndexReaderContext readerContext = reader.getContext();
List<LeafReaderContext> leaves = readerContext.leaves();

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.admin.cluster.stats;
import com.carrotsearch.hppc.ObjectObjectOpenHashMap;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;

View File

@ -37,7 +37,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
private boolean recycler = false;
private boolean queryCache = false;
private String[] fields = null;
private String[] filterKeys = null;
ClearIndicesCacheRequest() {
@ -83,15 +82,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
return this.fields;
}
public ClearIndicesCacheRequest filterKeys(String... filterKeys) {
this.filterKeys = filterKeys;
return this;
}
public String[] filterKeys() {
return this.filterKeys;
}
public boolean idCache() {
return this.idCache;
}
@ -118,7 +108,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
idCache = in.readBoolean();
recycler = in.readBoolean();
fields = in.readStringArray();
filterKeys = in.readStringArray();
queryCache = in.readBoolean();
}
@ -130,7 +119,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
out.writeBoolean(idCache);
out.writeBoolean(recycler);
out.writeStringArrayNullable(fields);
out.writeStringArrayNullable(filterKeys);
out.writeBoolean(queryCache);
}
}

View File

@ -53,11 +53,6 @@ public class ClearIndicesCacheRequestBuilder extends BroadcastOperationRequestBu
return this;
}
public ClearIndicesCacheRequestBuilder setFilterKeys(String... filterKeys) {
request.filterKeys(filterKeys);
return this;
}
public ClearIndicesCacheRequestBuilder setIdCache(boolean idCache) {
request.idCache(idCache);
return this;

View File

@ -39,7 +39,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
private boolean queryCache = false;
private String[] fields = null;
private String[] filterKeys = null;
ShardClearIndicesCacheRequest() {
}
@ -50,7 +49,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
fieldDataCache = request.fieldDataCache();
idCache = request.idCache();
fields = request.fields();
filterKeys = request.filterKeys();
recycler = request.recycler();
queryCache = request.queryCache();
}
@ -79,10 +77,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
return this.fields;
}
public String[] filterKeys() {
return this.filterKeys;
}
public ShardClearIndicesCacheRequest waitForOperations(boolean waitForOperations) {
this.filterCache = waitForOperations;
return this;
@ -96,7 +90,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
idCache = in.readBoolean();
recycler = in.readBoolean();
fields = in.readStringArray();
filterKeys = in.readStringArray();
queryCache = in.readBoolean();
}
@ -108,7 +101,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
out.writeBoolean(idCache);
out.writeBoolean(recycler);
out.writeStringArrayNullable(fields);
out.writeStringArrayNullable(filterKeys);
out.writeBoolean(queryCache);
}
}

View File

@ -106,10 +106,6 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastOperatio
clearedAtLeastOne = true;
service.cache().filter().clear("api");
}
if (request.filterKeys() != null && request.filterKeys().length > 0) {
clearedAtLeastOne = true;
service.cache().filter().clear("api", request.filterKeys());
}
if (request.fieldDataCache()) {
clearedAtLeastOne = true;
if (request.fields() == null || request.fields().length == 0) {

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService;
@ -62,9 +61,6 @@ public class ClusterDynamicSettingsModule extends AbstractModule {
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP + "*");
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP + "*");
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP + "*");
clusterDynamicSettings.addDynamicSetting(IndicesFilterCache.INDICES_CACHE_FILTER_SIZE);
clusterDynamicSettings.addDynamicSetting(IndicesFilterCache.INDICES_CACHE_FILTER_EXPIRE, Validator.TIME);
clusterDynamicSettings.addDynamicSetting(IndicesFilterCache.INDICES_CACHE_FILTER_CONCURRENCY_LEVEL, Validator.POSITIVE_INTEGER);
clusterDynamicSettings.addDynamicSetting(IndicesStore.INDICES_STORE_THROTTLE_TYPE);
clusterDynamicSettings.addDynamicSetting(IndicesStore.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
clusterDynamicSettings.addDynamicSetting(IndicesTTLService.INDICES_TTL_INTERVAL, Validator.TIME);

View File

@ -0,0 +1,74 @@
/*
* 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.common.lucene;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import java.io.IOException;
import java.util.Objects;
/**
* Base implementation for a query which is cacheable at the index level but
* not the segment level as usually expected.
*/
public abstract class IndexCacheableQuery extends Query {
private Object readerCacheKey;
@Override
public Query rewrite(IndexReader reader) throws IOException {
if (reader.getCoreCacheKey() != this.readerCacheKey) {
IndexCacheableQuery rewritten = (IndexCacheableQuery) clone();
rewritten.readerCacheKey = reader.getCoreCacheKey();
return rewritten;
}
return super.rewrite(reader);
}
@Override
public boolean equals(Object obj) {
return super.equals(obj)
&& readerCacheKey == ((IndexCacheableQuery) obj).readerCacheKey;
}
@Override
public int hashCode() {
return 31 * super.hashCode() + Objects.hashCode(readerCacheKey);
}
@Override
public final Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
if (readerCacheKey == null) {
throw new IllegalStateException("Rewrite first");
}
if (readerCacheKey != searcher.getIndexReader().getCoreCacheKey()) {
throw new IllegalStateException("Must create weight on the same reader which has been used for rewriting");
}
return doCreateWeight(searcher, needsScores);
}
/** Create a {@link Weight} for this query.
* @see Query#createWeight(IndexSearcher, boolean)
*/
public abstract Weight doCreateWeight(IndexSearcher searcher, boolean needsScores) throws IOException;
}

View File

@ -0,0 +1,109 @@
/*
* 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.common.lucene;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Multimap;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils;
import java.io.IOException;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.Set;
/**
* A map between segment core cache keys and the shard that these segments
* belong to. This allows to get the shard that a segment belongs to or to get
* the entire set of live core cache keys for a given index. In order to work
* this class needs to be notified about new segments. It modifies the current
* mappings as segments that were not known before are added and prevents the
* structure from growing indefinitely by registering close listeners on these
* segments so that at any time it only tracks live segments.
*
* NOTE: This is heavy. Avoid using this class unless absolutely required.
*/
public final class ShardCoreKeyMap {
private final Map<Object, ShardId> coreKeyToShard;
private final Multimap<String, Object> indexToCoreKey;
public ShardCoreKeyMap() {
coreKeyToShard = new IdentityHashMap<>();
indexToCoreKey = HashMultimap.create();
}
/**
* Register a {@link LeafReader}. This is necessary so that the core cache
* key of this reader can be found later using {@link #getCoreCacheKeys(ShardId)}.
*/
public void add(LeafReader reader) {
final ShardId shardId = ShardUtils.extractShardId(reader);
if (shardId == null) {
throw new IllegalArgumentException("Could not extract shard id from " + reader);
}
final Object coreKey = reader.getCoreCacheKey();
final String index = shardId.getIndex();
synchronized (this) {
if (coreKeyToShard.put(coreKey, shardId) == null) {
final boolean added = indexToCoreKey.put(index, coreKey);
assert added;
reader.addCoreClosedListener(new CoreClosedListener() {
@Override
public void onClose(Object ownerCoreCacheKey) throws IOException {
assert coreKey == ownerCoreCacheKey;
synchronized (ShardCoreKeyMap.this) {
coreKeyToShard.remove(ownerCoreCacheKey);
indexToCoreKey.remove(index, coreKey);
}
}
});
}
}
}
/**
* Return the {@link ShardId} that holds the given segment, or {@code null}
* if this segment is not tracked.
*/
public synchronized ShardId getShardId(Object coreKey) {
return coreKeyToShard.get(coreKey);
}
/**
* Get the set of core cache keys associated with the given index.
*/
public synchronized Set<Object> getCoreKeysForIndex(String index) {
return ImmutableSet.copyOf(indexToCoreKey.get(index));
}
/**
* Return the number of tracked segments.
*/
public synchronized int size() {
assert indexToCoreKey.size() == coreKeyToShard.size();
return coreKeyToShard.size();
}
}

View File

@ -22,6 +22,8 @@ package org.elasticsearch.common.lucene.docset;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.TwoPhaseIterator;
import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
@ -104,32 +106,41 @@ public class DocIdSets {
}
/**
* Given a {@link DocIdSet}, return a {@link Bits} instance that will match
* Given a {@link Scorer}, return a {@link Bits} instance that will match
* all documents contained in the set. Note that the returned {@link Bits}
* instance should only be consumed once and in order.
* instance MUST be consumed in order.
*/
public static Bits asSequentialAccessBits(final int maxDoc, @Nullable DocIdSet set) throws IOException {
if (set == null) {
public static Bits asSequentialAccessBits(final int maxDoc, @Nullable Scorer scorer) throws IOException {
if (scorer == null) {
return new Bits.MatchNoBits(maxDoc);
}
Bits bits = set.bits();
if (bits != null) {
return bits;
}
final DocIdSetIterator iterator = set.iterator();
if (iterator == null) {
return new Bits.MatchNoBits(maxDoc);
final TwoPhaseIterator twoPhase = scorer.asTwoPhaseIterator();
final DocIdSetIterator iterator;
if (twoPhase == null) {
iterator = scorer;
} else {
iterator = twoPhase.approximation();
}
return new Bits() {
int previous = 0;
int previous = -1;
boolean previousMatched = false;
@Override
public boolean get(int index) {
if (index < 0 || index >= maxDoc) {
throw new IndexOutOfBoundsException(index + " is out of bounds: [" + 0 + "-" + maxDoc + "[");
}
if (index < previous) {
throw new IllegalArgumentException("This Bits instance can only be consumed in order. "
+ "Got called on [" + index + "] while previously called on [" + previous + "]");
}
if (index == previous) {
// we cache whether it matched because it is illegal to call
// twoPhase.matches() twice
return previousMatched;
}
previous = index;
int doc = iterator.docID();
@ -140,7 +151,14 @@ public class DocIdSets {
throw new IllegalStateException("Cannot advance iterator", e);
}
}
return index == doc;
if (index == doc) {
try {
return previousMatched = twoPhase == null || twoPhase.matches();
} catch (IOException e) {
throw new IllegalStateException("Cannot validate match", e);
}
}
return previousMatched = false;
}
@Override

View File

@ -1,32 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.lucene.search;
import org.apache.lucene.search.Filter;
/**
* A marker indicating that this is a cached filter.
*/
public abstract class CachedFilter extends Filter {
public static boolean isCached(Filter filter) {
return filter instanceof CachedFilter;
}
}

View File

@ -31,18 +31,18 @@ import java.io.IOException;
public class FilteredCollector implements Collector {
private final Collector collector;
private final Filter filter;
private final Weight filter;
public FilteredCollector(Collector collector, Filter filter) {
public FilteredCollector(Collector collector, Weight filter) {
this.collector = collector;
this.filter = filter;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final DocIdSet set = filter.getDocIdSet(context, null);
final Scorer filterScorer = filter.scorer(context, null);
final LeafCollector in = collector.getLeafCollector(context);
final Bits bits = DocIdSets.asSequentialAccessBits(context.reader().maxDoc(), set);
final Bits bits = DocIdSets.asSequentialAccessBits(context.reader().maxDoc(), filterScorer);
return new FilterLeafCollector(in) {
@Override

View File

@ -1,79 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.lucene.search;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits;
import java.io.IOException;
/**
* A marker interface for {@link org.apache.lucene.search.Filter} denoting the filter
* as one that should not be cached, ever.
*/
public abstract class NoCacheFilter extends Filter {
private static final class NoCacheFilterWrapper extends NoCacheFilter {
private final Filter delegate;
private NoCacheFilterWrapper(Filter delegate) {
this.delegate = delegate;
}
@Override
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
return delegate.getDocIdSet(context, acceptDocs);
}
@Override
public int hashCode() {
return delegate.hashCode();
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj instanceof NoCacheFilterWrapper) {
return delegate.equals(((NoCacheFilterWrapper)obj).delegate);
}
return false;
}
@Override
public String toString(String field) {
return "no_cache(" + delegate + ")";
}
}
/**
* Wraps a filter in a NoCacheFilter or returns it if it already is a NoCacheFilter.
*/
public static Filter wrap(Filter filter) {
if (filter instanceof NoCacheFilter) {
return filter;
}
return new NoCacheFilterWrapper(filter);
}
}

View File

@ -1,36 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.lucene.search;
import org.apache.lucene.search.Query;
/**
* Queries are never cached directly, but a query can be wrapped in a filter that may end being cached.
* Filters that wrap this query either directly or indirectly will never be cached.
*/
public abstract class NoCacheQuery extends Query {
@Override
public final String toString(String s) {
return "no_cache(" + innerToString(s) + ")";
}
public abstract String innerToString(String s);
}

View File

@ -31,10 +31,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.search.child.CustomQueryWrappingFilter;
import java.util.List;
import java.util.regex.Pattern;
@ -54,19 +51,19 @@ public class Queries {
}
public static Filter newMatchAllFilter() {
return wrap(newMatchAllQuery());
return new QueryWrapperFilter(newMatchAllQuery());
}
public static Filter newMatchNoDocsFilter() {
return wrap(newMatchNoDocsQuery());
return new QueryWrapperFilter(newMatchNoDocsQuery());
}
public static Filter newNestedFilter() {
return wrap(new PrefixQuery(new Term(TypeFieldMapper.NAME, new BytesRef("__"))));
return new QueryWrapperFilter(new PrefixQuery(new Term(TypeFieldMapper.NAME, new BytesRef("__"))));
}
public static Filter newNonNestedFilter() {
return wrap(not(newNestedFilter()));
return new QueryWrapperFilter(not(newNestedFilter()));
}
/** Return a query that matches all documents but those that match the given query. */
@ -169,24 +166,4 @@ public class Queries {
optionalClauseCount : (result < 0 ? 0 : result));
}
/**
* Wraps a query in a filter.
*
* If a filter has an anti per segment execution / caching nature then @{@link CustomQueryWrappingFilter} is returned
* otherwise the standard {@link org.apache.lucene.search.QueryWrapperFilter} is returned.
*/
@SuppressForbidden(reason = "QueryWrapperFilter cachability")
public static Filter wrap(Query query, QueryParseContext context) {
if ((context != null && context.requireCustomQueryWrappingFilter()) || CustomQueryWrappingFilter.shouldUseCustomQueryWrappingFilter(query)) {
return new CustomQueryWrappingFilter(query);
} else {
return new QueryWrapperFilter(query);
}
}
/** Wrap as a {@link Filter}. */
public static Filter wrap(Query query) {
return wrap(query, null);
}
}

View File

@ -19,9 +19,11 @@
package org.elasticsearch.common.lucene.search;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import java.io.IOException;
@ -46,4 +48,13 @@ public abstract class ResolvableFilter extends Filter {
return null;
}
}
@Override
public Query rewrite(IndexReader reader) throws IOException {
final Filter resolved = resolve();
if (resolved != null) {
return resolved;
}
return super.rewrite(reader);
}
}

View File

@ -119,16 +119,22 @@ public class FiltersFunctionScoreQuery extends Query {
// TODO: needsScores
// if we dont need scores, just return the underlying Weight?
Weight subQueryWeight = subQuery.createWeight(searcher, needsScores);
return new CustomBoostFactorWeight(this, subQueryWeight);
Weight[] filterWeights = new Weight[filterFunctions.length];
for (int i = 0; i < filterFunctions.length; ++i) {
filterWeights[i] = searcher.createNormalizedWeight(filterFunctions[i].filter, false);
}
return new CustomBoostFactorWeight(this, subQueryWeight, filterWeights);
}
class CustomBoostFactorWeight extends Weight {
final Weight subQueryWeight;
final Weight[] filterWeights;
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight) throws IOException {
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight, Weight[] filterWeights) throws IOException {
super(parent);
this.subQueryWeight = subQueryWeight;
this.filterWeights = filterWeights;
}
@Override
@ -162,7 +168,8 @@ public class FiltersFunctionScoreQuery extends Query {
for (int i = 0; i < filterFunctions.length; i++) {
FilterFunction filterFunction = filterFunctions[i];
functions[i] = filterFunction.function.getLeafScoreFunction(context);
docSets[i] = DocIdSets.asSequentialAccessBits(context.reader().maxDoc(), filterFunction.filter.getDocIdSet(context, acceptDocs));
Scorer filterScorer = filterWeights[i].scorer(context, null); // no need to apply accepted docs
docSets[i] = DocIdSets.asSequentialAccessBits(context.reader().maxDoc(), filterScorer);
}
return new FiltersFunctionFactorScorer(this, subQueryScorer, scoreMode, filterFunctions, maxBoost, functions, docSets, combineFunction, minScore);
}
@ -177,7 +184,8 @@ public class FiltersFunctionScoreQuery extends Query {
// First: Gather explanations for all filters
List<Explanation> filterExplanations = new ArrayList<>();
float weightSum = 0;
for (FilterFunction filterFunction : filterFunctions) {
for (int i = 0; i < filterFunctions.length; ++i) {
FilterFunction filterFunction = filterFunctions[i];
if (filterFunction.function instanceof WeightFactorFunction) {
weightSum += ((WeightFactorFunction) filterFunction.function).getWeight();
@ -186,7 +194,7 @@ public class FiltersFunctionScoreQuery extends Query {
}
Bits docSet = DocIdSets.asSequentialAccessBits(context.reader().maxDoc(),
filterFunction.filter.getDocIdSet(context, context.reader().getLiveDocs()));
filterWeights[i].scorer(context, null));
if (docSet.get(doc)) {
Explanation functionExplanation = filterFunction.function.getLeafScoreFunction(context).explainScore(doc, subQueryExpl);
double factor = functionExplanation.getValue();

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterators;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -148,7 +149,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
this.indicesLifecycle = (InternalIndicesLifecycle) injector.getInstance(IndicesLifecycle.class);
// inject workarounds for cyclic dep
indexCache.filter().setIndexService(this);
indexFieldData.setIndexService(this);
bitSetFilterCache.setIndexService(this);
this.nodeEnv = nodeEnv;

View File

@ -22,10 +22,10 @@ package org.elasticsearch.index.aliases;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -109,7 +109,7 @@ public class IndexAliasesService extends AbstractIndexComponent implements Itera
return null;
}
}
return Queries.wrap(combined);
return new QueryWrapperFilter(combined);
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.cache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -37,12 +38,14 @@ import java.io.IOException;
public class IndexCache extends AbstractIndexComponent implements Closeable {
private final FilterCache filterCache;
private final QueryCachingPolicy filterCachingPolicy;
private final BitsetFilterCache bitsetFilterCache;
@Inject
public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, BitsetFilterCache bitsetFilterCache) {
public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, QueryCachingPolicy filterCachingPolicy, BitsetFilterCache bitsetFilterCache) {
super(index, indexSettings);
this.filterCache = filterCache;
this.filterCachingPolicy = filterCachingPolicy;
this.bitsetFilterCache = bitsetFilterCache;
}
@ -50,6 +53,10 @@ public class IndexCache extends AbstractIndexComponent implements Closeable {
return filterCache;
}
public QueryCachingPolicy filterPolicy() {
return filterCachingPolicy;
}
/**
* Return the {@link BitsetFilterCache} for this index.
*/

View File

@ -36,7 +36,6 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.NoCacheFilter;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@ -105,7 +104,6 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
public BitDocIdSetFilter getBitDocIdSetFilter(Filter filter) {
assert filter != null;
assert !(filter instanceof NoCacheFilter);
return new BitDocIdSetFilterWrapper(filter);
}

View File

@ -19,19 +19,14 @@
package org.elasticsearch.index.cache.filter;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.index.IndexComponent;
import org.elasticsearch.index.IndexService;
import java.io.Closeable;
/**
*
*/
public interface FilterCache extends IndexComponent, Closeable {
public interface FilterCache extends IndexComponent, Closeable, org.apache.lucene.search.QueryCache {
static class EntriesStats {
public final long sizeInBytes;
@ -43,16 +38,5 @@ public interface FilterCache extends IndexComponent, Closeable {
}
}
// we need to "inject" the index service to not create cyclic dep
void setIndexService(IndexService indexService);
String type();
Filter cache(Filter filterToCache, @Nullable HashedBytesRef cacheKey, QueryCachingPolicy policy);
void clear(Object reader);
void clear(String reason);
void clear(String reason, String[] keys);
}

View File

@ -24,7 +24,7 @@ import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.filter.weighted.WeightedFilterCache;
import org.elasticsearch.index.cache.filter.index.IndexFilterCache;
/**
*
@ -46,7 +46,7 @@ public class FilterCacheModule extends AbstractModule {
@Override
protected void configure() {
bind(FilterCache.class)
.to(settings.getAsClass(FilterCacheSettings.FILTER_CACHE_TYPE, WeightedFilterCache.class, "org.elasticsearch.index.cache.filter.", "FilterCache"))
.to(settings.getAsClass(FilterCacheSettings.FILTER_CACHE_TYPE, IndexFilterCache.class, "org.elasticsearch.index.cache.filter.", "FilterCache"))
.in(Scopes.SINGLETON);
// the filter cache is a node-level thing, however we want the most popular filters
// to be computed on a per-index basis, that is why we don't use the SINGLETON

View File

@ -33,32 +33,79 @@ import java.io.IOException;
*/
public class FilterCacheStats implements Streamable, ToXContent {
long memorySize;
long evictions;
long ramBytesUsed;
long hitCount;
long missCount;
long cacheCount;
long cacheSize;
public FilterCacheStats() {
}
public FilterCacheStats(long memorySize, long evictions) {
this.memorySize = memorySize;
this.evictions = evictions;
public FilterCacheStats(long ramBytesUsed, long hitCount, long missCount, long cacheCount, long cacheSize) {
this.ramBytesUsed = ramBytesUsed;
this.hitCount = hitCount;
this.missCount = missCount;
this.cacheCount = cacheCount;
this.cacheSize = cacheSize;
}
public void add(FilterCacheStats stats) {
this.memorySize += stats.memorySize;
this.evictions += stats.evictions;
ramBytesUsed += stats.ramBytesUsed;
hitCount += stats.hitCount;
missCount += stats.missCount;
cacheCount += stats.cacheCount;
cacheSize += stats.cacheSize;
}
public long getMemorySizeInBytes() {
return this.memorySize;
return ramBytesUsed;
}
public ByteSizeValue getMemorySize() {
return new ByteSizeValue(memorySize);
return new ByteSizeValue(ramBytesUsed);
}
/**
* The total number of lookups in the cache.
*/
public long getTotalCount() {
return hitCount + missCount;
}
/**
* The number of successful lookups in the cache.
*/
public long getHitCount() {
return hitCount;
}
/**
* The number of lookups in the cache that failed to retrieve a {@link DocIdSet}.
*/
public long getMissCount() {
return missCount;
}
/**
* The number of {@link DocIdSet}s that have been cached.
*/
public long getCacheCount() {
return cacheCount;
}
/**
* The number of {@link DocIdSet}s that are in the cache.
*/
public long getCacheSize() {
return cacheSize;
}
/**
* The number of {@link DocIdSet}s that have been evicted from the cache.
*/
public long getEvictions() {
return this.evictions;
return cacheCount - cacheSize;
}
public static FilterCacheStats readFilterCacheStats(StreamInput in) throws IOException {
@ -67,22 +114,34 @@ public class FilterCacheStats implements Streamable, ToXContent {
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
memorySize = in.readVLong();
evictions = in.readVLong();
ramBytesUsed = in.readLong();
hitCount = in.readLong();
missCount = in.readLong();
cacheCount = in.readLong();
cacheSize = in.readLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(memorySize);
out.writeVLong(evictions);
out.writeLong(ramBytesUsed);
out.writeLong(hitCount);
out.writeLong(missCount);
out.writeLong(cacheCount);
out.writeLong(cacheSize);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startObject(Fields.FILTER_CACHE);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, memorySize);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, ramBytesUsed);
builder.field(Fields.TOTAL_COUNT, getTotalCount());
builder.field(Fields.HIT_COUNT, getHitCount());
builder.field(Fields.MISS_COUNT, getMissCount());
builder.field(Fields.CACHE_SIZE, getCacheSize());
builder.field(Fields.CACHE_COUNT, getCacheCount());
builder.field(Fields.EVICTIONS, getEvictions());
builder.endObject();
return builder;
@ -92,6 +151,12 @@ public class FilterCacheStats implements Streamable, ToXContent {
static final XContentBuilderString FILTER_CACHE = new XContentBuilderString("filter_cache");
static final XContentBuilderString MEMORY_SIZE = new XContentBuilderString("memory_size");
static final XContentBuilderString MEMORY_SIZE_IN_BYTES = new XContentBuilderString("memory_size_in_bytes");
static final XContentBuilderString TOTAL_COUNT = new XContentBuilderString("total_count");
static final XContentBuilderString HIT_COUNT = new XContentBuilderString("hit_count");
static final XContentBuilderString MISS_COUNT = new XContentBuilderString("miss_count");
static final XContentBuilderString CACHE_SIZE = new XContentBuilderString("cache_size");
static final XContentBuilderString CACHE_COUNT = new XContentBuilderString("cache_count");
static final XContentBuilderString EVICTIONS = new XContentBuilderString("evictions");
}
}

View File

@ -19,45 +19,35 @@
package org.elasticsearch.index.cache.filter;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.apache.lucene.search.DocIdSet;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.filter.weighted.WeightedFilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
import java.io.Closeable;
import java.io.IOException;
/**
*/
public class ShardFilterCache extends AbstractIndexShardComponent implements RemovalListener<WeightedFilterCache.FilterCacheKey, DocIdSet> {
public class ShardFilterCache extends AbstractIndexShardComponent implements Closeable {
final CounterMetric evictionsMetric = new CounterMetric();
final CounterMetric totalMetric = new CounterMetric();
final IndicesFilterCache cache;
@Inject
public ShardFilterCache(ShardId shardId, @IndexSettings Settings indexSettings) {
public ShardFilterCache(ShardId shardId, @IndexSettings Settings indexSettings, IndicesFilterCache cache) {
super(shardId, indexSettings);
this.cache = cache;
}
public FilterCacheStats stats() {
return new FilterCacheStats(totalMetric.count(), evictionsMetric.count());
}
public void onCached(long sizeInBytes) {
totalMetric.inc(sizeInBytes);
return cache.getStats(shardId);
}
@Override
public void onRemoval(RemovalNotification<WeightedFilterCache.FilterCacheKey, DocIdSet> removalNotification) {
if (removalNotification.wasEvicted()) {
evictionsMetric.inc();
}
if (removalNotification.getValue() != null) {
totalMetric.dec(DocIdSets.sizeInBytes(removalNotification.getValue()));
}
public void close() throws IOException {
cache.onClose(shardId);
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.cache.filter.index;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.Weight;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.filter.FilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
/**
* The index-level filter cache. This class mostly delegates to the node-level
* filter cache: {@link IndicesFilterCache}.
*/
public class IndexFilterCache extends AbstractIndexComponent implements FilterCache {
final IndicesFilterCache indicesFilterCache;
@Inject
public IndexFilterCache(Index index, @IndexSettings Settings indexSettings, IndicesFilterCache indicesFilterCache) {
super(index, indexSettings);
this.indicesFilterCache = indicesFilterCache;
}
@Override
public void close() throws ElasticsearchException {
clear("close");
}
@Override
public void clear(String reason) {
logger.debug("full cache clear, reason [{}]", reason);
indicesFilterCache.clearIndex(index.getName());
}
@Override
public Weight doCache(Weight weight, QueryCachingPolicy policy) {
return indicesFilterCache.doCache(weight, policy);
}
}

View File

@ -19,15 +19,12 @@
package org.elasticsearch.index.cache.filter.none;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.Nullable;
import org.apache.lucene.search.Weight;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.cache.filter.FilterCache;
import org.elasticsearch.index.settings.IndexSettings;
@ -42,38 +39,18 @@ public class NoneFilterCache extends AbstractIndexComponent implements FilterCac
logger.debug("Using no filter cache");
}
@Override
public void setIndexService(IndexService indexService) {
// nothing to do here...
}
@Override
public String type() {
return "none";
}
@Override
public void close() {
// nothing to do here
}
@Override
public Filter cache(Filter filterToCache, @Nullable HashedBytesRef cacheKey, QueryCachingPolicy policy) {
return filterToCache;
public Weight doCache(Weight weight, QueryCachingPolicy policy) {
return weight;
}
@Override
public void clear(String reason) {
// nothing to do here
}
@Override
public void clear(String reason, String[] keys) {
// nothing to do there
}
@Override
public void clear(Object reader) {
// nothing to do here
}
}

View File

@ -1,277 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.cache.filter.weighted;
import com.google.common.cache.Cache;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.Weigher;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.util.Bits;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.common.lucene.search.CachedFilter;
import org.elasticsearch.common.lucene.search.NoCacheFilter;
import org.elasticsearch.common.lucene.search.ResolvableFilter;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.cache.filter.FilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils;
import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
import java.io.IOException;
import java.util.concurrent.ConcurrentMap;
public class WeightedFilterCache extends AbstractIndexComponent implements FilterCache, SegmentReader.CoreClosedListener, IndexReader.ReaderClosedListener {
final IndicesFilterCache indicesFilterCache;
IndexService indexService;
final ConcurrentMap<Object, Boolean> seenReaders = ConcurrentCollections.newConcurrentMap();
@Inject
public WeightedFilterCache(Index index, @IndexSettings Settings indexSettings, IndicesFilterCache indicesFilterCache) {
super(index, indexSettings);
this.indicesFilterCache = indicesFilterCache;
}
@Override
public void setIndexService(IndexService indexService) {
this.indexService = indexService;
}
@Override
public String type() {
return "weighted";
}
@Override
public void close() {
clear("close");
}
@Override
public void onClose(IndexReader reader) {
clear(reader.getCoreCacheKey());
}
@Override
public void clear(String reason) {
logger.debug("full cache clear, reason [{}]", reason);
for (Object readerKey : seenReaders.keySet()) {
Boolean removed = seenReaders.remove(readerKey);
if (removed == null) {
return;
}
indicesFilterCache.addReaderKeyToClean(readerKey);
}
}
@Override
public void clear(String reason, String[] keys) {
logger.debug("clear keys [], reason [{}]", reason, keys);
for (String key : keys) {
final HashedBytesRef keyBytes = new HashedBytesRef(key);
for (Object readerKey : seenReaders.keySet()) {
indicesFilterCache.cache().invalidate(new FilterCacheKey(readerKey, keyBytes));
}
}
}
@Override
public void onClose(Object coreKey) {
clear(coreKey);
}
@Override
public void clear(Object coreCacheKey) {
// we add the seen reader before we add the first cache entry for this reader
// so, if we don't see it here, its won't be in the cache
Boolean removed = seenReaders.remove(coreCacheKey);
if (removed == null) {
return;
}
indicesFilterCache.addReaderKeyToClean(coreCacheKey);
}
@Override
public Filter cache(Filter filterToCache, @Nullable HashedBytesRef cacheKey, QueryCachingPolicy cachePolicy) {
if (filterToCache == null) {
return null;
}
if (filterToCache instanceof NoCacheFilter) {
return filterToCache;
}
if (CachedFilter.isCached(filterToCache)) {
return filterToCache;
}
if (filterToCache instanceof ResolvableFilter) {
throw new IllegalArgumentException("Cannot cache instances of ResolvableFilter: " + filterToCache);
}
return new FilterCacheFilterWrapper(filterToCache, cacheKey, cachePolicy, this);
}
static class FilterCacheFilterWrapper extends CachedFilter {
private final Filter filter;
private final Object filterCacheKey;
private final QueryCachingPolicy cachePolicy;
private final WeightedFilterCache cache;
FilterCacheFilterWrapper(Filter filter, Object cacheKey, QueryCachingPolicy cachePolicy, WeightedFilterCache cache) {
this.filter = filter;
this.filterCacheKey = cacheKey != null ? cacheKey : filter;
this.cachePolicy = cachePolicy;
this.cache = cache;
}
@Override
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
if (context.ord == 0) {
cachePolicy.onUse(filter);
}
FilterCacheKey cacheKey = new FilterCacheKey(context.reader().getCoreCacheKey(), filterCacheKey);
Cache<FilterCacheKey, DocIdSet> innerCache = cache.indicesFilterCache.cache();
DocIdSet cacheValue = innerCache.getIfPresent(cacheKey);
final DocIdSet ret;
if (cacheValue != null) {
ret = cacheValue;
} else {
final DocIdSet uncached = filter.getDocIdSet(context, null);
if (cachePolicy.shouldCache(filter, context)) {
if (!cache.seenReaders.containsKey(context.reader().getCoreCacheKey())) {
Boolean previous = cache.seenReaders.putIfAbsent(context.reader().getCoreCacheKey(), Boolean.TRUE);
if (previous == null) {
// we add a core closed listener only, for non core IndexReaders we rely on clear being called (percolator for example)
context.reader().addCoreClosedListener(cache);
}
}
// we can't pass down acceptedDocs provided, because we are caching the result, and acceptedDocs
// might be specific to a query. We don't pass the live docs either because a cache built for a specific
// generation of a segment might be reused by an older generation which has fewer deleted documents
cacheValue = DocIdSets.toCacheable(context.reader(), uncached);
// we might put the same one concurrently, that's fine, it will be replaced and the removal
// will be called
ShardId shardId = ShardUtils.extractShardId(context.reader());
if (shardId != null) {
IndexShard shard = cache.indexService.shard(shardId.id());
if (shard != null) {
cacheKey.removalListener = shard.filterCache();
shard.filterCache().onCached(DocIdSets.sizeInBytes(cacheValue));
}
}
innerCache.put(cacheKey, cacheValue);
ret = cacheValue;
} else {
// uncached
ret = uncached;
}
}
return BitsFilteredDocIdSet.wrap(DocIdSets.isEmpty(ret) ? null : ret, acceptDocs);
}
@Override
public String toString(String field) {
return "cache(" + filter + ")";
}
@Override
public boolean equals(Object o) {
if (super.equals(o) == false) return false;
return this.filter.equals(((FilterCacheFilterWrapper) o).filter);
}
@Override
public int hashCode() {
return 31 * super.hashCode() + filter.hashCode();
}
}
/** A weigher for the Guava filter cache that uses a minimum entry size */
public static class FilterCacheValueWeigher implements Weigher<WeightedFilterCache.FilterCacheKey, DocIdSet> {
private final int minimumEntrySize;
public FilterCacheValueWeigher(int minimumEntrySize) {
this.minimumEntrySize = minimumEntrySize;
}
@Override
public int weigh(FilterCacheKey key, DocIdSet value) {
int weight = (int) Math.min(DocIdSets.sizeInBytes(value), Integer.MAX_VALUE);
return Math.max(weight, this.minimumEntrySize);
}
}
public static class FilterCacheKey {
private final Object readerKey;
private final Object filterKey;
// if we know, we will try and set the removal listener (for statistics)
// its ok that its not volatile because we make sure we only set it when the object is created before its shared between threads
@Nullable
public RemovalListener<WeightedFilterCache.FilterCacheKey, DocIdSet> removalListener;
public FilterCacheKey(Object readerKey, Object filterKey) {
this.readerKey = readerKey;
this.filterKey = filterKey;
}
public Object readerKey() {
return readerKey;
}
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().equals(that.readerKey()) && filterKey.equals(that.filterKey));
}
@Override
public int hashCode() {
return readerKey().hashCode() + 31 * filterKey.hashCode();
}
}
}

View File

@ -21,14 +21,10 @@ package org.elasticsearch.index.cache.query;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.apache.lucene.search.DocIdSet;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.docset.DocIdSets;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.filter.FilterCacheStats;
import org.elasticsearch.index.cache.filter.weighted.WeightedFilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;

View File

@ -21,6 +21,8 @@ package org.elasticsearch.index.engine;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
@ -74,6 +76,8 @@ public final class EngineConfig {
private final Similarity similarity;
private final CodecService codecService;
private final Engine.FailedEngineListener failedEngineListener;
private final QueryCache filterCache;
private final QueryCachingPolicy filterCachingPolicy;
/**
* Index setting for index concurrency / number of threadstates in the indexwriter.
@ -130,7 +134,11 @@ public final class EngineConfig {
/**
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
*/
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService, IndexSettingsService indexSettingsService, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog, MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler, Analyzer analyzer, Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener, TranslogRecoveryPerformer translogRecoveryPerformer) {
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService,
IndexSettingsService indexSettingsService, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
Translog translog, MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache filterCache, QueryCachingPolicy filterCachingPolicy) {
this.shardId = shardId;
this.threadPool = threadPool;
this.indexingService = indexingService;
@ -155,6 +163,8 @@ public final class EngineConfig {
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer;
this.filterCache = filterCache;
this.filterCachingPolicy = filterCachingPolicy;
}
/** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */
@ -396,4 +406,18 @@ public final class EngineConfig {
public TranslogRecoveryPerformer getTranslogRecoveryPerformer() {
return translogRecoveryPerformer;
}
/**
* Return the cache to use for filters.
*/
public QueryCache getFilterCache() {
return filterCache;
}
/**
* Return the policy to use when caching filters.
*/
public QueryCachingPolicy getFilterCachingPolicy() {
return filterCachingPolicy;
}
}

View File

@ -40,7 +40,9 @@ public class EngineSearcherFactory extends SearcherFactory {
@Override
public IndexSearcher newSearcher(IndexReader reader, IndexReader previousReader) throws IOException {
IndexSearcher searcher = new IndexSearcher(reader);
IndexSearcher searcher = super.newSearcher(reader, previousReader);
searcher.setQueryCache(engineConfig.getFilterCache());
searcher.setQueryCachingPolicy(engineConfig.getFilterCachingPolicy());
searcher.setSimilarity(engineConfig.getSimilarity());
return searcher;
}

View File

@ -1044,7 +1044,7 @@ public class InternalEngine extends Engine {
try {
assert isMergedSegment(reader);
if (warmer != null) {
final Engine.Searcher searcher = new Searcher("warmer", new IndexSearcher(reader));
final Engine.Searcher searcher = new Searcher("warmer", searcherFactory.newSearcher(reader, null));
final IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, searcher);
warmer.warmNewReaders(context);
}
@ -1077,8 +1077,7 @@ public class InternalEngine extends Engine {
@Override
public IndexSearcher newSearcher(IndexReader reader, IndexReader previousReader) throws IOException {
IndexSearcher searcher = new IndexSearcher(reader);
searcher.setSimilarity(engineConfig.getSimilarity());
IndexSearcher searcher = super.newSearcher(reader, previousReader);
if (warmer != null) {
// we need to pass a custom searcher that does not release anything on Engine.Search Release,
// we will release explicitly
@ -1110,7 +1109,8 @@ public class InternalEngine extends Engine {
}
if (!readers.isEmpty()) {
// we don't want to close the inner readers, just increase ref on them
newSearcher = new IndexSearcher(new MultiReader(readers.toArray(new IndexReader[readers.size()]), false));
IndexReader newReader = new MultiReader(readers.toArray(new IndexReader[readers.size()]), false);
newSearcher = super.newSearcher(newReader, null);
closeNewSearcher = true;
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index.mapper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.DocIdSet;
@ -43,7 +44,19 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.mapper.Mapping.SourceTransform;
import org.elasticsearch.index.mapper.internal.*;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import org.elasticsearch.index.mapper.internal.FieldNamesFieldMapper;
import org.elasticsearch.index.mapper.internal.IdFieldMapper;
import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
import org.elasticsearch.index.mapper.internal.SizeFieldMapper;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.internal.VersionFieldMapper;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
import org.elasticsearch.script.ExecutableScript;
@ -54,7 +67,12 @@ import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.*;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
/**
@ -343,7 +361,7 @@ public class DocumentMapper implements ToXContent {
continue;
}
Filter filter = sc.filterCache().cache(objectMapper.nestedTypeFilter(), null, sc.queryParserService().autoFilterCachePolicy());
Filter filter = objectMapper.nestedTypeFilter();
if (filter == null) {
continue;
}

View File

@ -34,6 +34,7 @@ import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchGenerationException;
@ -371,11 +372,11 @@ public class MapperService extends AbstractIndexComponent {
BooleanQuery bq = new BooleanQuery();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(Queries.newNonNestedFilter(), Occur.MUST);
return Queries.wrap(bq);
return new QueryWrapperFilter(bq);
} else if (hasNested) {
return Queries.newNonNestedFilter();
} else if (filterPercolateType) {
return Queries.wrap(Queries.not(percolatorType));
return new QueryWrapperFilter(Queries.not(percolatorType));
} else {
return null;
}
@ -384,12 +385,12 @@ public class MapperService extends AbstractIndexComponent {
// since they have different types (starting with __)
if (types.length == 1) {
DocumentMapper docMapper = documentMapper(types[0]);
Filter filter = docMapper != null ? docMapper.typeFilter() : Queries.wrap(new TermQuery(new Term(TypeFieldMapper.NAME, types[0])));
Filter filter = docMapper != null ? docMapper.typeFilter() : new QueryWrapperFilter(new TermQuery(new Term(TypeFieldMapper.NAME, types[0])));
if (filterPercolateType) {
BooleanQuery bq = new BooleanQuery();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(filter, Occur.MUST);
return Queries.wrap(bq);
return new QueryWrapperFilter(bq);
} else {
return filter;
}
@ -419,9 +420,9 @@ public class MapperService extends AbstractIndexComponent {
BooleanQuery bq = new BooleanQuery();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(termsFilter, Occur.MUST);
return Queries.wrap(bq);
return new QueryWrapperFilter(bq);
} else {
return Queries.wrap(termsFilter);
return new QueryWrapperFilter(termsFilter);
}
} else {
// Current bool filter requires that at least one should clause matches, even with a must clause.
@ -441,7 +442,7 @@ public class MapperService extends AbstractIndexComponent {
bool.add(Queries.newNonNestedFilter(), BooleanClause.Occur.MUST);
}
return Queries.wrap(bool);
return new QueryWrapperFilter(bool);
}
}

View File

@ -24,6 +24,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.base.Objects;
import com.google.common.collect.ImmutableList;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -35,6 +36,7 @@ import org.apache.lucene.search.FuzzyQuery;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeQuery;
@ -480,7 +482,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
@Override
public Filter termFilter(Object value, @Nullable QueryParseContext context) {
return Queries.wrap(new TermQuery(names().createIndexNameTerm(indexedValueForSearch(value))));
return new QueryWrapperFilter(new TermQuery(names().createIndexNameTerm(indexedValueForSearch(value))));
}
@Override
@ -499,7 +501,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
for (int i = 0; i < bytesRefs.length; i++) {
bytesRefs[i] = indexedValueForSearch(values.get(i));
}
return Queries.wrap(new TermsQuery(names.indexName(), bytesRefs));
return new QueryWrapperFilter(new TermsQuery(names.indexName(), bytesRefs));
}
}
@ -529,7 +531,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(new TermRangeQuery(names.indexName(),
return new QueryWrapperFilter(new TermRangeQuery(names.indexName(),
lowerTerm == null ? null : indexedValueForSearch(lowerTerm),
upperTerm == null ? null : indexedValueForSearch(upperTerm),
includeLower, includeUpper));
@ -551,7 +553,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
@Override
public Filter prefixFilter(Object value, @Nullable QueryParseContext context) {
return Queries.wrap(new PrefixQuery(names().createIndexNameTerm(indexedValueForSearch(value))));
return new QueryWrapperFilter(new PrefixQuery(names().createIndexNameTerm(indexedValueForSearch(value))));
}
@Override
@ -565,7 +567,7 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T> {
@Override
public Filter regexpFilter(Object value, int flags, int maxDeterminizedStates, @Nullable QueryParseContext parseContext) {
return Queries.wrap(new RegexpQuery(names().createIndexNameTerm(indexedValueForSearch(value)), flags, maxDeterminizedStates));
return new QueryWrapperFilter(new RegexpQuery(names().createIndexNameTerm(indexedValueForSearch(value)), flags, maxDeterminizedStates));
}
@Override

View File

@ -24,13 +24,13 @@ import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -205,7 +205,7 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
if (nullValue == null) {
return null;
}
return Queries.wrap(new TermQuery(names().createIndexNameTerm(nullValue ? Values.TRUE : Values.FALSE)));
return new QueryWrapperFilter(new TermQuery(names().createIndexNameTerm(nullValue ? Values.TRUE : Values.FALSE)));
}
@Override

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -34,7 +35,6 @@ import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -212,7 +212,7 @@ public class ByteFieldMapper extends NumberFieldMapper<Byte> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper,
@Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseValueAsInt(lowerTerm),
upperTerm == null ? null : parseValueAsInt(upperTerm),
includeLower, includeUpper));
@ -231,7 +231,7 @@ public class ByteFieldMapper extends NumberFieldMapper<Byte> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
nullValue.intValue(),
nullValue.intValue(),
true, true));

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -39,8 +40,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.joda.DateMathParser;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.lucene.search.NoCacheQuery;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.lucene.search.ResolvableFilter;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
@ -392,7 +391,7 @@ public class DateFieldMapper extends NumberFieldMapper<Long> {
if (fieldData != null) {
filter = NumericRangeFieldDataFilter.newLongRange(fieldData, lowerVal,upperVal, includeLower, includeUpper);
} else {
filter = Queries.wrap(NumericRangeQuery.newLongRange(
filter = new QueryWrapperFilter(NumericRangeQuery.newLongRange(
names.indexName(), precisionStep, lowerVal, upperVal, includeLower, includeUpper
));
}
@ -406,7 +405,7 @@ public class DateFieldMapper extends NumberFieldMapper<Long> {
return null;
}
long value = parseStringValue(nullValue);
return Queries.wrap(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
value,
value,
true, true));
@ -588,7 +587,7 @@ public class DateFieldMapper extends NumberFieldMapper<Long> {
}
}
public final class LateParsingQuery extends NoCacheQuery {
public final class LateParsingQuery extends Query {
final Object lowerTerm;
final Object upperTerm;
@ -613,7 +612,7 @@ public class DateFieldMapper extends NumberFieldMapper<Long> {
}
@Override
public String innerToString(String s) {
public String toString(String s) {
final StringBuilder sb = new StringBuilder();
return sb.append(names.indexName()).append(':')
.append(includeLower ? '[' : '{')

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -38,7 +39,6 @@ import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.util.ByteUtils;
@ -202,14 +202,14 @@ public class DoubleFieldMapper extends NumberFieldMapper<Double> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseDoubleValue(lowerTerm),
upperTerm == null ? null : parseDoubleValue(upperTerm),
includeLower, includeUpper));
}
public Filter rangeFilter(Double lowerTerm, Double upperTerm, boolean includeLower, boolean includeUpper) {
return Queries.wrap(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep, lowerTerm, upperTerm, includeLower, includeUpper));
return new QueryWrapperFilter(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep, lowerTerm, upperTerm, includeLower, includeUpper));
}
@Override
@ -225,7 +225,7 @@ public class DoubleFieldMapper extends NumberFieldMapper<Double> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep,
nullValue,
nullValue,
true, true));

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -39,7 +40,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.util.ByteUtils;
@ -212,7 +212,7 @@ public class FloatFieldMapper extends NumberFieldMapper<Float> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper));
@ -231,7 +231,7 @@ public class FloatFieldMapper extends NumberFieldMapper<Float> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
nullValue,
nullValue,
true, true));

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -36,7 +37,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -206,7 +206,7 @@ public class IntegerFieldMapper extends NumberFieldMapper<Integer> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper));
@ -225,7 +225,7 @@ public class IntegerFieldMapper extends NumberFieldMapper<Integer> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
nullValue,
nullValue,
true, true));

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -36,7 +37,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -196,7 +196,7 @@ public class LongFieldMapper extends NumberFieldMapper<Long> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseLongValue(lowerTerm),
upperTerm == null ? null : parseLongValue(upperTerm),
includeLower, includeUpper));
@ -215,7 +215,7 @@ public class LongFieldMapper extends NumberFieldMapper<Long> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
nullValue,
nullValue,
true, true));

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -36,7 +37,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -212,7 +212,7 @@ public class ShortFieldMapper extends NumberFieldMapper<Short> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseValueAsInt(lowerTerm),
upperTerm == null ? null : parseValueAsInt(upperTerm),
includeLower, includeUpper));
@ -220,7 +220,7 @@ public class ShortFieldMapper extends NumberFieldMapper<Short> {
@Override
public Filter rangeFilter(QueryParseContext parseContext, Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeFieldDataFilter.newShortRange((IndexNumericFieldData) parseContext.getForField(this),
return new QueryWrapperFilter(NumericRangeFieldDataFilter.newShortRange((IndexNumericFieldData) parseContext.getForField(this),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper));
@ -231,7 +231,7 @@ public class ShortFieldMapper extends NumberFieldMapper<Short> {
if (nullValue == null) {
return null;
}
return Queries.wrap(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
nullValue.intValue(),
nullValue.intValue(),
true, true));

View File

@ -34,6 +34,7 @@ import org.apache.lucene.search.Filter;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
@ -41,7 +42,6 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -200,7 +200,7 @@ public class IdFieldMapper extends AbstractFieldMapper<String> implements Intern
if (fieldType.indexOptions() != IndexOptions.NONE || context == null) {
return super.termFilter(value, context);
}
return Queries.wrap(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(context.queryTypes(), value)));
return new QueryWrapperFilter(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(context.queryTypes(), value)));
}
@Override
@ -208,7 +208,7 @@ public class IdFieldMapper extends AbstractFieldMapper<String> implements Intern
if (fieldType.indexOptions() != IndexOptions.NONE || context == null) {
return super.termsFilter(values, context);
}
return Queries.wrap(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(context.queryTypes(), values)));
return new QueryWrapperFilter(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(context.queryTypes(), values)));
}
@Override
@ -238,7 +238,7 @@ public class IdFieldMapper extends AbstractFieldMapper<String> implements Intern
for (String queryType : queryTypes) {
filter.add(new PrefixQuery(new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(queryType, BytesRefs.toBytesRef(value)))), BooleanClause.Occur.SHOULD);
}
return Queries.wrap(filter);
return new QueryWrapperFilter(filter);
}
@Override
@ -277,7 +277,7 @@ public class IdFieldMapper extends AbstractFieldMapper<String> implements Intern
filter.add(new RegexpQuery(new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(queryType, BytesRefs.toBytesRef(value))),
flags, maxDeterminizedStates), BooleanClause.Occur.SHOULD);
}
return Queries.wrap(filter);
return new QueryWrapperFilter(filter);
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
@ -275,7 +276,7 @@ public class ParentFieldMapper extends AbstractFieldMapper<Uid> implements Inter
}
BytesRef bValue = BytesRefs.toBytesRef(value);
if (Uid.hasDelimiter(bValue)) {
return Queries.wrap(new TermQuery(new Term(names.indexName(), bValue)));
return new QueryWrapperFilter(new TermQuery(new Term(names.indexName(), bValue)));
}
List<String> types = new ArrayList<>(context.mapperService().types().size());
@ -288,14 +289,14 @@ public class ParentFieldMapper extends AbstractFieldMapper<Uid> implements Inter
if (types.isEmpty()) {
return Queries.newMatchNoDocsFilter();
} else if (types.size() == 1) {
return Queries.wrap(new TermQuery(new Term(names.indexName(), Uid.createUidAsBytes(types.get(0), bValue))));
return new QueryWrapperFilter(new TermQuery(new Term(names.indexName(), Uid.createUidAsBytes(types.get(0), bValue))));
} else {
// we use all non child types, cause we don't know if its exact or not...
List<BytesRef> typesValues = new ArrayList<>(types.size());
for (String type : context.mapperService().types()) {
typesValues.add(Uid.createUidAsBytes(type, bValue));
}
return Queries.wrap(new TermsQuery(names.indexName(), typesValues));
return new QueryWrapperFilter(new TermsQuery(names.indexName(), typesValues));
}
}
@ -328,7 +329,7 @@ public class ParentFieldMapper extends AbstractFieldMapper<Uid> implements Inter
}
}
}
return Queries.wrap(new TermsQuery(names.indexName(), bValues));
return new QueryWrapperFilter(new TermsQuery(names.indexName(), bValues));
}
/**

View File

@ -28,13 +28,13 @@ import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
@ -133,15 +133,15 @@ public class TypeFieldMapper extends AbstractFieldMapper<String> implements Inte
@Override
public Query termQuery(Object value, @Nullable QueryParseContext context) {
return new ConstantScoreQuery(context.cacheFilter(termFilter(value, context), null, context.autoFilterCachePolicy()));
return new ConstantScoreQuery(termFilter(value, context));
}
@Override
public Filter termFilter(Object value, @Nullable QueryParseContext context) {
if (fieldType.indexOptions() == IndexOptions.NONE) {
return Queries.wrap(new PrefixQuery(new Term(UidFieldMapper.NAME, Uid.typePrefixAsBytes(BytesRefs.toBytesRef(value)))));
return new QueryWrapperFilter(new PrefixQuery(new Term(UidFieldMapper.NAME, Uid.typePrefixAsBytes(BytesRefs.toBytesRef(value)))));
}
return Queries.wrap(new TermQuery(names().createIndexNameTerm(BytesRefs.toBytesRef(value))));
return new QueryWrapperFilter(new TermQuery(names().createIndexNameTerm(BytesRefs.toBytesRef(value))));
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
@ -35,7 +36,6 @@ import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -253,7 +253,7 @@ public class IpFieldMapper extends NumberFieldMapper<Long> {
@Override
public Filter rangeFilter(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return Queries.wrap(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper));
@ -273,7 +273,7 @@ public class IpFieldMapper extends NumberFieldMapper<Long> {
return null;
}
final long value = ipToLong(nullValue);
return Queries.wrap(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
return new QueryWrapperFilter(NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
value,
value,
true, true));

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index.mapper.object;
import com.google.common.collect.Iterables;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
@ -29,7 +30,6 @@ import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -367,7 +367,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll, Clonea
}
this.nestedTypePathAsString = "__" + fullPath;
this.nestedTypePathAsBytes = new BytesRef(nestedTypePathAsString);
this.nestedTypeFilter = Queries.wrap(new TermQuery(new Term(TypeFieldMapper.NAME, nestedTypePathAsBytes)));
this.nestedTypeFilter = new QueryWrapperFilter(new TermQuery(new Term(TypeFieldMapper.NAME, nestedTypePathAsBytes)));
}
@Override

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.percolator;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
@ -28,7 +27,6 @@ import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -93,7 +91,7 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
private CloseableThreadLocal<QueryParseContext> cache = new CloseableThreadLocal<QueryParseContext>() {
@Override
protected QueryParseContext initialValue() {
return new QueryParseContext(shardId.index(), queryParserService, true);
return new QueryParseContext(shardId.index(), queryParserService);
}
};
@ -280,13 +278,7 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
shard.refresh("percolator_load_queries");
// Maybe add a mode load? This isn't really a write. We need write b/c state=post_recovery
try (Engine.Searcher searcher = shard.acquireSearcher("percolator_load_queries", true)) {
Query query = new ConstantScoreQuery(
indexCache.filter().cache(
Queries.wrap(new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME))),
null,
queryParserService.autoFilterCachePolicy()
)
);
Query query = new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME));
QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger, mapperService, indexFieldDataService);
searcher.searcher().search(query, queryCollector);
Map<BytesRef, Query> queries = queryCollector.queries();

View File

@ -34,9 +34,6 @@ public class AndFilterBuilder extends BaseFilterBuilder {
private ArrayList<FilterBuilder> filters = Lists.newArrayList();
private Boolean cache;
private String cacheKey;
private String filterName;
public AndFilterBuilder(FilterBuilder... filters) {
@ -53,19 +50,6 @@ public class AndFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public AndFilterBuilder cache(boolean cache) {
this.cache = cache;
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.
*/
@ -82,12 +66,6 @@ public class AndFilterBuilder extends BaseFilterBuilder {
filter.toXContent(builder, params);
}
builder.endArray();
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
if (filterName != null) {
builder.field("_name", filterName);
}

View File

@ -22,10 +22,8 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -56,9 +54,6 @@ public class AndFilterParser implements FilterParser {
ArrayList<Filter> filters = newArrayList();
boolean filtersFound = false;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String currentFieldName = null;
XContentParser.Token token = parser.currentToken();
@ -74,6 +69,8 @@ public class AndFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_ARRAY) {
if ("filters".equals(currentFieldName)) {
filtersFound = true;
@ -93,12 +90,8 @@ public class AndFilterParser implements FilterParser {
}
}
} else if (token.isValue()) {
if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_name".equals(currentFieldName)) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[and] filter does not support [" + currentFieldName + "]");
}
@ -120,10 +113,7 @@ public class AndFilterParser implements FilterParser {
for (Filter filter : filters) {
boolQuery.add(filter, Occur.MUST);
}
Filter filter = Queries.wrap(boolQuery);
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
Filter filter = new QueryWrapperFilter(boolQuery);
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -36,9 +36,6 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
private ArrayList<FilterBuilder> shouldClauses = new ArrayList<>();
private Boolean cache;
private String cacheKey;
private String filterName;
/**
@ -114,19 +111,6 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public BoolFilterBuilder cache(boolean cache) {
this.cache = cache;
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");
@ -137,12 +121,6 @@ public class BoolFilterBuilder extends BaseFilterBuilder {
if (filterName != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
builder.endObject();
}

View File

@ -22,10 +22,8 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -52,9 +50,6 @@ public class BoolFilterParser implements FilterParser {
BooleanQuery boolFilter = new BooleanQuery();
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String currentFieldName = null;
XContentParser.Token token;
@ -64,6 +59,8 @@ public class BoolFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
if ("must".equals(currentFieldName)) {
hasAnyFilter = true;
@ -117,12 +114,8 @@ public class BoolFilterParser implements FilterParser {
throw new QueryParsingException(parseContext, "[bool] filter does not support [" + currentFieldName + "]");
}
} else if (token.isValue()) {
if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_name".equals(currentFieldName)) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[bool] filter does not support [" + currentFieldName + "]");
}
@ -138,10 +131,7 @@ public class BoolFilterParser implements FilterParser {
return null;
}
Filter filter = Queries.wrap(boolFilter);
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
Filter filter = new QueryWrapperFilter(boolFilter);
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -22,10 +22,8 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -55,14 +53,14 @@ public class ConstantScoreQueryParser implements QueryParser {
Query query = null;
boolean queryFound = false;
float boost = 1.0f;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String currentFieldName = null;
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
if ("filter".equals(currentFieldName)) {
filter = parseContext.parseInnerFilter();
@ -76,10 +74,6 @@ public class ConstantScoreQueryParser implements QueryParser {
} else if (token.isValue()) {
if ("boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[constant_score] query does not support [" + currentFieldName + "]");
}
@ -94,11 +88,6 @@ public class ConstantScoreQueryParser implements QueryParser {
}
if (filter != null) {
// cache the filter if possible needed
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
Query query1 = new ConstantScoreQuery(filter);
query1.setBoost(boost);
return query1;

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermRangeQuery;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
@ -120,11 +121,7 @@ public class ExistsFilterParser implements FilterParser {
boolFilter.add(filter, BooleanClause.Occur.SHOULD);
}
Filter filter = Queries.wrap(boolFilter);
// we always cache this one, really does not change... (exists)
// its ok to cache under the fieldName cacheKey, since its per segment and the mapping applies to this data on this segment...
filter = parseContext.cacheFilter(filter, new HashedBytesRef("$exists$" + fieldPattern), parseContext.autoFilterCachePolicy());
Filter filter = new QueryWrapperFilter(boolFilter);
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -21,10 +21,8 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -52,8 +50,6 @@ public class FQueryFilterParser implements FilterParser {
Query query = null;
boolean queryFound = false;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String currentFieldName = null;
@ -61,6 +57,8 @@ public class FQueryFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
if ("query".equals(currentFieldName)) {
queryFound = true;
@ -71,10 +69,6 @@ public class FQueryFilterParser implements FilterParser {
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.autoFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[fquery] filter does not support [" + currentFieldName + "]");
}
@ -86,10 +80,7 @@ public class FQueryFilterParser implements FilterParser {
if (query == null) {
return null;
}
Filter filter = Queries.wrap(query, parseContext);
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
Filter filter = new QueryWrapperFilter(query);
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.query;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentType;

View File

@ -23,9 +23,7 @@ import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
@ -55,8 +53,6 @@ public class FilteredQueryParser implements QueryParser {
Filter filter = null;
boolean filterFound = false;
float boost = 1.0f;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String queryName = null;
String currentFieldName = null;
@ -66,6 +62,8 @@ public class FilteredQueryParser implements QueryParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
if ("query".equals(currentFieldName)) {
query = parseContext.parseInnerQuery();
@ -99,10 +97,6 @@ public class FilteredQueryParser implements QueryParser {
queryName = parser.text();
} else if ("boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[filtered] query does not support [" + currentFieldName + "]");
}
@ -129,11 +123,6 @@ public class FilteredQueryParser implements QueryParser {
return query;
}
// cache if required
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
// if its a match_all query, use constant_score
if (Queries.isConstantMatchAllQuery(query)) {
Query q = new ConstantScoreQuery(filter);

View File

@ -42,9 +42,6 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
private double[] box = {Double.NaN, Double.NaN, Double.NaN, Double.NaN};
private Boolean cache;
private String cacheKey;
private String filterName;
private String type;
@ -140,19 +137,6 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public GeoBoundingBoxFilterBuilder cache(boolean cache) {
this.cache = cache;
return this;
}
public GeoBoundingBoxFilterBuilder cacheKey(String cacheKey) {
this.cacheKey = cacheKey;
return this;
}
/**
* Sets the type of executing of the geo bounding box. Can be either `memory` or `indexed`. Defaults
* to `memory`.
@ -185,12 +169,6 @@ public class GeoBoundingBoxFilterBuilder extends BaseFilterBuilder {
if (filterName != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
if (type != null) {
builder.field("type", type);
}

View File

@ -20,12 +20,10 @@
package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
@ -72,8 +70,6 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
public Filter parse(QueryParseContext parseContext) throws IOException, QueryParsingException {
XContentParser parser = parseContext.parser();
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String fieldName = null;
double top = Double.NaN;
@ -100,7 +96,9 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
token = parser.nextToken();
if (FIELD.equals(currentFieldName)) {
if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (FIELD.equals(currentFieldName)) {
fieldName = parser.text();
} else if (TOP.equals(currentFieldName)) {
top = parser.doubleValue();
@ -138,10 +136,6 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else if ("normalize".equals(currentFieldName)) {
normalize = parser.booleanValue();
} else if ("type".equals(currentFieldName)) {
@ -188,9 +182,6 @@ public class GeoBoundingBoxFilterParser implements FilterParser {
+ "] not supported, either 'indexed' or 'memory' are allowed");
}
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -45,9 +45,6 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
private String optimizeBbox;
private Boolean cache;
private String cacheKey;
private String filterName;
public GeoDistanceFilterBuilder(String name) {
@ -103,19 +100,6 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public GeoDistanceFilterBuilder cache(boolean cache) {
this.cache = cache;
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);
@ -134,12 +118,6 @@ public class GeoDistanceFilterBuilder extends BaseFilterBuilder {
if (filterName != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
builder.endObject();
}
}

View File

@ -20,13 +20,11 @@
package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -64,8 +62,6 @@ public class GeoDistanceFilterParser implements FilterParser {
XContentParser.Token token;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String currentFieldName = null;
GeoPoint point = new GeoPoint();
@ -80,6 +76,8 @@ public class GeoDistanceFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_ARRAY) {
fieldName = currentFieldName;
GeoUtils.parseGeoPoint(parser, point);
@ -125,10 +123,6 @@ public class GeoDistanceFilterParser implements FilterParser {
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.GEOHASH_SUFFIX.length());
} else if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else if ("optimize_bbox".equals(currentFieldName) || "optimizeBbox".equals(currentFieldName)) {
optimizeBbox = parser.textOrNull();
} else if ("normalize".equals(currentFieldName)) {
@ -167,9 +161,6 @@ public class GeoDistanceFilterParser implements FilterParser {
IndexGeoPointFieldData indexFieldData = parseContext.getForField(mapper);
Filter filter = new GeoDistanceFilter(point.lat(), point.lon(), distance, geoDistance, indexFieldData, geoMapper, optimizeBbox);
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -45,9 +45,6 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
private GeoDistance geoDistance;
private Boolean cache;
private String cacheKey;
private String filterName;
private String optimizeBbox;
@ -139,19 +136,6 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public GeoDistanceRangeFilterBuilder cache(boolean cache) {
this.cache = cache;
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);
@ -173,12 +157,6 @@ public class GeoDistanceRangeFilterBuilder extends BaseFilterBuilder {
if (filterName != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
builder.endObject();
}
}

View File

@ -20,13 +20,11 @@
package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -64,8 +62,6 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
XContentParser.Token token;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String currentFieldName = null;
GeoPoint point = new GeoPoint();
@ -82,6 +78,8 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_ARRAY) {
GeoUtils.parseGeoPoint(parser, point);
fieldName = currentFieldName;
@ -155,10 +153,6 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.GEOHASH_SUFFIX.length());
} else if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else if ("optimize_bbox".equals(currentFieldName) || "optimizeBbox".equals(currentFieldName)) {
optimizeBbox = parser.textOrNull();
} else if ("normalize".equals(currentFieldName)) {
@ -206,9 +200,6 @@ public class GeoDistanceRangeFilterParser implements FilterParser {
IndexGeoPointFieldData indexFieldData = parseContext.getForField(mapper);
Filter filter = new GeoDistanceRangeFilter(point, from, to, includeLower, includeUpper, geoDistance, geoMapper, indexFieldData, optimizeBbox);
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.query;
import com.google.common.collect.Lists;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -38,9 +39,6 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
private final List<GeoPoint> shell = Lists.newArrayList();
private Boolean cache;
private String cacheKey;
private String filterName;
public GeoPolygonFilterBuilder(String name) {
@ -75,19 +73,6 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public GeoPolygonFilterBuilder cache(boolean cache) {
this.cache = cache;
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);
@ -103,12 +88,6 @@ public class GeoPolygonFilterBuilder extends BaseFilterBuilder {
if (filterName != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
builder.endObject();
}

View File

@ -22,11 +22,9 @@ package org.elasticsearch.index.query;
import com.google.common.collect.Lists;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -68,8 +66,6 @@ public class GeoPolygonFilterParser implements FilterParser {
public Filter parse(QueryParseContext parseContext) throws IOException, QueryParsingException {
XContentParser parser = parseContext.parser();
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String fieldName = null;
List<GeoPoint> shell = Lists.newArrayList();
@ -84,6 +80,8 @@ public class GeoPolygonFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
fieldName = currentFieldName;
@ -107,10 +105,6 @@ public class GeoPolygonFilterParser implements FilterParser {
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else if ("normalize".equals(currentFieldName)) {
normalizeLat = parser.booleanValue();
normalizeLon = parser.booleanValue();
@ -154,9 +148,6 @@ public class GeoPolygonFilterParser implements FilterParser {
IndexGeoPointFieldData indexFieldData = parseContext.getForField(mapper);
Filter filter = new GeoPolygonFilter(indexFieldData, shell.toArray(new GeoPoint[shell.size()]));
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -37,9 +37,6 @@ public class GeoShapeFilterBuilder extends BaseFilterBuilder {
private SpatialStrategy strategy = null;
private Boolean cache;
private String cacheKey;
private String filterName;
private final String indexedShapeId;
@ -93,28 +90,6 @@ public class GeoShapeFilterBuilder extends BaseFilterBuilder {
this.indexedShapeType = indexedShapeType;
}
/**
* Sets whether the filter will be cached.
*
* @param cache Whether filter will be cached
* @return this
*/
public GeoShapeFilterBuilder cache(boolean cache) {
this.cache = cache;
return this;
}
/**
* Sets the key used for the filter if it is cached
*
* @param cacheKey Key for the Filter if cached
* @return this
*/
public GeoShapeFilterBuilder cacheKey(String cacheKey) {
this.cacheKey = cacheKey;
return this;
}
/**
* Sets the name of the filter
*
@ -205,12 +180,6 @@ public class GeoShapeFilterBuilder extends BaseFilterBuilder {
if (name != null) {
builder.field("_name", filterName);
}
if (cache != null) {
builder.field("_cache", cache);
}
if (cacheKey != null) {
builder.field("_cache_key", cacheKey);
}
builder.endObject();
}

View File

@ -24,15 +24,13 @@ import com.spatial4j.core.shape.Shape;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
import org.elasticsearch.common.geo.ShapeRelation;
import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
@ -85,8 +83,6 @@ public class GeoShapeFilterParser implements FilterParser {
ShapeRelation shapeRelation = ShapeRelation.INTERSECTS;
String strategyName = null;
ShapeBuilder shape = null;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
String filterName = null;
String id = null;
@ -100,6 +96,8 @@ public class GeoShapeFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
fieldName = currentFieldName;
@ -147,10 +145,6 @@ public class GeoShapeFilterParser implements FilterParser {
} else if (token.isValue()) {
if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
cache = parseContext.parseFilterCachePolicy();
} else if ("_cache_key".equals(currentFieldName)) {
cacheKey = new HashedBytesRef(parser.text());
} else {
throw new QueryParsingException(parseContext, "[geo_shape] filter does not support [" + currentFieldName + "]");
}
@ -189,15 +183,11 @@ public class GeoShapeFilterParser implements FilterParser {
Filter intersects = strategy.makeFilter(GeoShapeQueryParser.getArgs(shape, ShapeRelation.INTERSECTS));
bool.add(exists, BooleanClause.Occur.MUST);
bool.add(intersects, BooleanClause.Occur.MUST_NOT);
filter = Queries.wrap(bool);
filter = new QueryWrapperFilter(bool);
} else {
filter = strategy.makeFilter(GeoShapeQueryParser.getArgs(shape, shapeRelation));
}
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryCachingPolicy;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
@ -28,7 +27,6 @@ import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -62,8 +60,6 @@ public class GeohashCellFilter {
public static final String NAME = "geohash_cell";
public static final String NEIGHBORS = "neighbors";
public static final String PRECISION = "precision";
public static final String CACHE = "_cache";
public static final String CACHE_KEY = "_cache_key";
/**
* Create a new geohash filter for a given set of geohashes. In general this method
@ -103,8 +99,6 @@ public class GeohashCellFilter {
private String geohash;
private int levels = -1;
private boolean neighbors;
private Boolean cache;
private String cacheKey;
public Builder(String field) {
@ -161,19 +155,6 @@ public class GeohashCellFilter {
return this;
}
/**
* Should the filter be cached or not. Defaults to <tt>false</tt>.
*/
public Builder cache(boolean cache) {
this.cache = cache;
return this;
}
public Builder cacheKey(String cacheKey) {
this.cacheKey = cacheKey;
return this;
}
@Override
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
@ -183,12 +164,6 @@ public class GeohashCellFilter {
if(levels > 0) {
builder.field(PRECISION, levels);
}
if (cache != null) {
builder.field(CACHE, cache);
}
if (cacheKey != null) {
builder.field(CACHE_KEY, cacheKey);
}
builder.field(field, geohash);
builder.endObject();
@ -214,8 +189,6 @@ public class GeohashCellFilter {
String geohash = null;
int levels = -1;
boolean neighbors = false;
QueryCachingPolicy cache = parseContext.autoFilterCachePolicy();
HashedBytesRef cacheKey = null;
XContentParser.Token token;
@ -227,7 +200,9 @@ public class GeohashCellFilter {
if (token == Token.FIELD_NAME) {
String field = parser.text();
if (PRECISION.equals(field)) {
if (parseContext.isDeprecatedSetting(field)) {
// skip
} else if (PRECISION.equals(field)) {
token = parser.nextToken();
if(token == Token.VALUE_NUMBER) {
levels = parser.intValue();
@ -238,12 +213,6 @@ public class GeohashCellFilter {
} else if (NEIGHBORS.equals(field)) {
parser.nextToken();
neighbors = parser.booleanValue();
} else if (CACHE.equals(field)) {
parser.nextToken();
cache = parseContext.parseFilterCachePolicy();
} else if (CACHE_KEY.equals(field)) {
parser.nextToken();
cacheKey = new HashedBytesRef(parser.text());
} else {
fieldName = field;
token = parser.nextToken();
@ -296,10 +265,6 @@ public class GeohashCellFilter {
filter = create(parseContext, geoMapper, geohash, null);
}
if (cache != null) {
filter = parseContext.cacheFilter(filter, cacheKey, cache);
}
return filter;
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.join.BitDocIdSetFilter;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
@ -34,7 +35,6 @@ import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.XContentStructure;
import org.elasticsearch.index.search.child.ChildrenConstantScoreQuery;
import org.elasticsearch.index.search.child.ChildrenQuery;
import org.elasticsearch.index.search.child.CustomQueryWrappingFilter;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SubSearchContext;
@ -80,6 +80,8 @@ public class HasChildFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
// Usually, the query would be parsed here, but the child
// type may not have been extracted yet, so use the
@ -101,10 +103,6 @@ public class HasChildFilterParser implements FilterParser {
childType = parser.text();
} else if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
// noop to be backwards compatible
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
// noop to be backwards compatible
} else if ("short_circuit_cutoff".equals(currentFieldName)) {
shortCircuitParentDocSet = parser.intValue();
} else if ("min_children".equals(currentFieldName) || "minChildren".equals(currentFieldName)) {
@ -150,7 +148,7 @@ public class HasChildFilterParser implements FilterParser {
String parentType = parentFieldMapper.type();
// wrap the query with type query
query = new FilteredQuery(query, parseContext.cacheFilter(childDocMapper.typeFilter(), null, parseContext.autoFilterCachePolicy()));
query = new FilteredQuery(query, childDocMapper.typeFilter());
DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType);
if (parentDocMapper == null) {
@ -167,7 +165,7 @@ public class HasChildFilterParser implements FilterParser {
nonNestedDocsFilter = parseContext.bitsetFilter(Queries.newNonNestedFilter());
}
Filter parentFilter = parseContext.cacheFilter(parentDocMapper.typeFilter(), null, parseContext.autoFilterCachePolicy());
Filter parentFilter = parentDocMapper.typeFilter();
ParentChildIndexFieldData parentChildIndexFieldData = parseContext.getForField(parentFieldMapper);
Query childrenQuery;
@ -178,9 +176,9 @@ public class HasChildFilterParser implements FilterParser {
shortCircuitParentDocSet, nonNestedDocsFilter);
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, new CustomQueryWrappingFilter(childrenQuery));
parseContext.addNamedFilter(filterName, new QueryWrapperFilter(childrenQuery));
}
return new CustomQueryWrappingFilter(childrenQuery);
return new QueryWrapperFilter(childrenQuery);
}
}

View File

@ -35,7 +35,6 @@ import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.XContentStructure;
import org.elasticsearch.index.search.child.ChildrenConstantScoreQuery;
import org.elasticsearch.index.search.child.ChildrenQuery;
import org.elasticsearch.index.search.child.CustomQueryWrappingFilter;
import org.elasticsearch.index.search.child.ScoreType;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SubSearchContext;
@ -166,10 +165,10 @@ public class HasChildQueryParser implements QueryParser {
}
// wrap the query with type query
innerQuery = new FilteredQuery(innerQuery, parseContext.cacheFilter(childDocMapper.typeFilter(), null, parseContext.autoFilterCachePolicy()));
innerQuery = new FilteredQuery(innerQuery, childDocMapper.typeFilter());
Query query;
Filter parentFilter = parseContext.cacheFilter(parentDocMapper.typeFilter(), null, parseContext.autoFilterCachePolicy());
Filter parentFilter = parentDocMapper.typeFilter();
ParentChildIndexFieldData parentChildIndexFieldData = parseContext.getForField(parentFieldMapper);
if (minChildren > 1 || maxChildren > 0 || scoreType != ScoreType.NONE) {
query = new ChildrenQuery(parentChildIndexFieldData, parentType, childType, parentFilter, innerQuery, scoreType, minChildren,
@ -179,7 +178,7 @@ public class HasChildQueryParser implements QueryParser {
shortCircuitParentDocSet, nonNestedDocsFilter);
}
if (queryName != null) {
parseContext.addNamedFilter(queryName, new CustomQueryWrappingFilter(query));
parseContext.addNamedQuery(queryName, query);
}
query.setBoost(boost);
return query;

View File

@ -62,20 +62,6 @@ public class HasParentFilterBuilder extends BaseFilterBuilder {
return this;
}
/**
* This is a noop since has_parent can't be cached.
*/
public HasParentFilterBuilder cache(boolean cache) {
return this;
}
/**
* This is a noop since has_parent can't be cached.
*/
public HasParentFilterBuilder cacheKey(String cacheKey) {
return this;
}
/**
* Sets inner hit definition in the scope of this filter and reusing the defined type and query.
*/

View File

@ -20,13 +20,13 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.XContentStructure;
import org.elasticsearch.index.search.child.CustomQueryWrappingFilter;
import org.elasticsearch.search.internal.SubSearchContext;
import java.io.IOException;
@ -69,6 +69,8 @@ public class HasParentFilterParser implements FilterParser {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
// Usually, the query would be parsed here, but the child
// type may not have been extracted yet, so use the
@ -90,10 +92,6 @@ public class HasParentFilterParser implements FilterParser {
parentType = parser.text();
} else if ("_name".equals(currentFieldName)) {
filterName = parser.text();
} else if ("_cache".equals(currentFieldName)) {
// noop to be backwards compatible
} else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) {
// noop to be backwards compatible
} else {
throw new QueryParsingException(parseContext, "[has_parent] filter does not support [" + currentFieldName + "]");
}
@ -122,9 +120,9 @@ public class HasParentFilterParser implements FilterParser {
return null;
}
if (filterName != null) {
parseContext.addNamedFilter(filterName, new CustomQueryWrappingFilter(parentQuery));
parseContext.addNamedFilter(filterName, new QueryWrapperFilter(parentQuery));
}
return new CustomQueryWrappingFilter(parentQuery);
return new QueryWrapperFilter(parentQuery);
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
@ -33,7 +34,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
import org.elasticsearch.index.query.support.XContentStructure;
import org.elasticsearch.index.search.child.CustomQueryWrappingFilter;
import org.elasticsearch.index.search.child.ParentConstantScoreQuery;
import org.elasticsearch.index.search.child.ParentQuery;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
@ -136,7 +136,7 @@ public class HasParentQueryParser implements QueryParser {
query.setBoost(boost);
if (queryName != null) {
parseContext.addNamedFilter(queryName, new CustomQueryWrappingFilter(query));
parseContext.addNamedQuery(queryName, query);
}
return query;
}
@ -186,7 +186,7 @@ public class HasParentQueryParser implements QueryParser {
parentsFilter.add(documentMapper.typeFilter(), BooleanClause.Occur.SHOULD);
}
}
parentFilter = Queries.wrap(parentsFilter);
parentFilter = new QueryWrapperFilter(parentsFilter);
}
if (parentFilter == null) {
@ -194,8 +194,8 @@ public class HasParentQueryParser implements QueryParser {
}
// wrap the query with type query
innerQuery = new FilteredQuery(innerQuery, parseContext.cacheFilter(parentDocMapper.typeFilter(), null, parseContext.autoFilterCachePolicy()));
Filter childrenFilter = parseContext.cacheFilter(Queries.wrap(Queries.not(parentFilter)), null, parseContext.autoFilterCachePolicy());
innerQuery = new FilteredQuery(innerQuery, parentDocMapper.typeFilter());
Filter childrenFilter = new QueryWrapperFilter(Queries.not(parentFilter));
if (score) {
return new ParentQuery(parentChildIndexFieldData, innerQuery, parentDocMapper.type(), childrenFilter);
} else {

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
@ -109,7 +110,7 @@ public class IdsFilterParser implements FilterParser {
types = parseContext.mapperService().types();
}
Filter filter = Queries.wrap(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(types, ids)));
Filter filter = new QueryWrapperFilter(new TermsQuery(UidFieldMapper.NAME, Uid.createTypeUids(types, ids)));
if (filterName != null) {
parseContext.addNamedFilter(filterName, filter);
}

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
@ -94,8 +93,6 @@ public class IndexQueryParserService extends AbstractIndexComponent {
final BitsetFilterCache bitsetFilterCache;
final QueryCachingPolicy autoFilterCachePolicy;
private final Map<String, QueryParser> queryParsers;
private final Map<String, FilterParser> filterParsers;
@ -111,7 +108,6 @@ public class IndexQueryParserService extends AbstractIndexComponent {
ScriptService scriptService, AnalysisService analysisService,
MapperService mapperService, IndexCache indexCache, IndexFieldDataService fieldDataService,
BitsetFilterCache bitsetFilterCache,
QueryCachingPolicy autoFilterCachePolicy,
@Nullable SimilarityService similarityService,
@Nullable Map<String, QueryParserFactory> namedQueryParsers,
@Nullable Map<String, FilterParserFactory> namedFilterParsers) {
@ -123,7 +119,6 @@ public class IndexQueryParserService extends AbstractIndexComponent {
this.indexCache = indexCache;
this.fieldDataService = fieldDataService;
this.bitsetFilterCache = bitsetFilterCache;
this.autoFilterCachePolicy = autoFilterCachePolicy;
this.defaultField = indexSettings.get(DEFAULT_FIELD, AllFieldMapper.NAME);
this.queryStringLenient = indexSettings.getAsBoolean(QUERY_STRING_LENIENT, false);
@ -185,10 +180,6 @@ public class IndexQueryParserService extends AbstractIndexComponent {
return this.defaultField;
}
public QueryCachingPolicy autoFilterCachePolicy() {
return autoFilterCachePolicy;
}
public boolean queryStringLenient() {
return this.queryStringLenient;
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.TermRangeQuery;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.HashedBytesRef;
@ -143,13 +144,8 @@ public class MissingFilterParser implements FilterParser {
boolFilter.add(filter, BooleanClause.Occur.SHOULD);
}
// we always cache this one, really does not change... (exists)
// its ok to cache under the fieldName cacheKey, since its per segment and the mapping applies to this data on this segment...
existenceFilter = Queries.wrap(boolFilter);
existenceFilter = parseContext.cacheFilter(existenceFilter, new HashedBytesRef("$exists$" + fieldPattern), parseContext.autoFilterCachePolicy());
existenceFilter = Queries.wrap(Queries.not(existenceFilter));
// cache the not filter as well, so it will be faster
existenceFilter = parseContext.cacheFilter(existenceFilter, new HashedBytesRef("$missing$" + fieldPattern), parseContext.autoFilterCachePolicy());
existenceFilter = new QueryWrapperFilter(boolFilter);
existenceFilter = new QueryWrapperFilter(Queries.not(existenceFilter));;
}
if (nullValue) {
@ -157,10 +153,6 @@ public class MissingFilterParser implements FilterParser {
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(field);
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
nullFilter = smartNameFieldMappers.mapper().nullValueFilter();
if (nullFilter != null) {
// cache the not filter as well, so it will be faster
nullFilter = parseContext.cacheFilter(nullFilter, new HashedBytesRef("$null$" + fieldPattern), parseContext.autoFilterCachePolicy());
}
}
}
}
@ -172,7 +164,7 @@ public class MissingFilterParser implements FilterParser {
combined.add(existenceFilter, BooleanClause.Occur.SHOULD);
combined.add(nullFilter, BooleanClause.Occur.SHOULD);
// cache the not filter as well, so it will be faster
filter = parseContext.cacheFilter(Queries.wrap(combined), null, parseContext.autoFilterCachePolicy());
filter = new QueryWrapperFilter(combined);
} else {
filter = nullFilter;
}

Some files were not shown because too many files have changed in this diff Show More