INDEX_CODEC_SETTING = new Setting<>("index.codec", "default", s -> {
+ switch (s) {
case "default":
case "best_compression":
case "lucene_default":
return s;
default:
if (Codec.availableCodecs().contains(s) == false) { // we don't error message the not officially supported ones
- throw new IllegalArgumentException("unknown value for [index.codec] must be one of [default, best_compression] but was: " + s);
+ throw new IllegalArgumentException(
+ "unknown value for [index.codec] must be one of [default, best_compression] but was: " + s);
}
return s;
}
- }, Property.IndexScope);
+ }, Property.IndexScope, Property.NodeScope);
/** if set to true the engine will start even if the translog id in the commit point can not be found */
public static final String INDEX_FORCE_NEW_TRANSLOG = "index.engine.force_new_translog";
@@ -97,7 +102,8 @@ public final class EngineConfig {
IndexSettings indexSettings, Engine.Warmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy,Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
- TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) {
+ TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
+ TranslogConfig translogConfig, TimeValue flushMergesAfter) {
this.shardId = shardId;
final Settings settings = indexSettings.getSettings();
this.indexSettings = indexSettings;
@@ -138,7 +144,8 @@ public final class EngineConfig {
}
/**
- * Returns the initial index buffer size. This setting is only read on startup and otherwise controlled by {@link IndexingMemoryController}
+ * Returns the initial index buffer size. This setting is only read on startup and otherwise controlled
+ * by {@link IndexingMemoryController}
*/
public ByteSizeValue getIndexingBufferSize() {
return indexingBufferSize;
@@ -146,11 +153,12 @@ public final class EngineConfig {
/**
* Returns true
iff delete garbage collection in the engine should be enabled. This setting is updateable
- * in realtime and forces a volatile read. Consumers can safely read this value directly go fetch it's latest value. The default is true
+ * in realtime and forces a volatile read. Consumers can safely read this value directly go fetch it's latest value.
+ * The default is true
*
* Engine GC deletion if enabled collects deleted documents from in-memory realtime data structures after a certain amount of
- * time ({@link IndexSettings#getGcDeletesInMillis()} if enabled. Before deletes are GCed they will cause re-adding the document that was deleted
- * to fail.
+ * time ({@link IndexSettings#getGcDeletesInMillis()} if enabled. Before deletes are GCed they will cause re-adding the document
+ * that was deleted to fail.
*
*/
public boolean isEnableGcDeletes() {
@@ -168,7 +176,8 @@ public final class EngineConfig {
}
/**
- * Returns a thread-pool mainly used to get estimated time stamps from {@link org.elasticsearch.threadpool.ThreadPool#estimatedTimeInMillis()} and to schedule
+ * Returns a thread-pool mainly used to get estimated time stamps from
+ * {@link org.elasticsearch.threadpool.ThreadPool#estimatedTimeInMillis()} and to schedule
* async force merge calls on the {@link org.elasticsearch.threadpool.ThreadPool.Names#FORCE_MERGE} thread-pool
*/
public ThreadPool getThreadPool() {
@@ -183,8 +192,9 @@ public final class EngineConfig {
}
/**
- * Returns the {@link org.elasticsearch.index.store.Store} instance that provides access to the {@link org.apache.lucene.store.Directory}
- * used for the engines {@link org.apache.lucene.index.IndexWriter} to write it's index files to.
+ * Returns the {@link org.elasticsearch.index.store.Store} instance that provides access to the
+ * {@link org.apache.lucene.store.Directory} used for the engines {@link org.apache.lucene.index.IndexWriter} to write it's index files
+ * to.
*
* Note: In order to use this instance the consumer needs to increment the stores reference before it's used the first time and hold
* it's reference until it's not needed anymore.
diff --git a/core/src/main/java/org/elasticsearch/index/fieldstats/FieldStatsProvider.java b/core/src/main/java/org/elasticsearch/index/fieldstats/FieldStatsProvider.java
new file mode 100644
index 00000000000..066d1207e3f
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/index/fieldstats/FieldStatsProvider.java
@@ -0,0 +1,181 @@
+/*
+ * 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.fieldstats;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.action.fieldstats.FieldStats;
+import org.elasticsearch.action.fieldstats.IndexConstraint;
+import org.elasticsearch.action.fieldstats.IndexConstraint.Comparison;
+import org.elasticsearch.action.fieldstats.IndexConstraint.Property;
+import org.elasticsearch.common.joda.DateMathParser;
+import org.elasticsearch.index.engine.Engine.Searcher;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.MapperService;
+import org.elasticsearch.index.mapper.core.DateFieldMapper.DateFieldType;
+import org.elasticsearch.index.mapper.ip.IpFieldMapper.IpFieldType;
+import org.joda.time.DateTimeZone;
+
+import java.io.IOException;
+
+/**
+ * Provides a service for gettings the {@link FieldStats} for a given field from
+ * the index.
+ */
+public class FieldStatsProvider {
+
+ private final Searcher searcher;
+ private final MapperService mapperService;
+
+ /**
+ * @param searcher
+ * the {@link Searcher}to use when retrieving the
+ * {@link FieldStats}
+ * @param mapperService
+ * the {@link MapperService}
+ */
+ public FieldStatsProvider(Searcher searcher, MapperService mapperService) {
+ this.searcher = searcher;
+ this.mapperService = mapperService;
+ }
+
+ /**
+ * @param field
+ * the name of the field to return {@link FieldStats} for.
+ * @return a {@link FieldStats} object for the given field
+ * @throws IOException
+ * if the field statistics cannot be read
+ */
+ public > FieldStats get(String field) throws IOException {
+ MappedFieldType mappedFieldType = mapperService.fullName(field);
+ if (mappedFieldType != null) {
+ IndexReader reader = searcher.reader();
+ Terms terms = MultiFields.getTerms(reader, field);
+ if (terms != null) {
+ return mappedFieldType.stats(terms, reader.maxDoc());
+ }
+ }
+ return null;
+ }
+
+ /**
+ * @param fieldName
+ * the fieldName to check
+ * @param from
+ * the minimum value for the query
+ * @param to
+ * the maximum value for the query
+ * @param includeLower
+ * whether the from value is inclusive
+ * @param includeUpper
+ * whether the to value is inclusive
+ * @param timeZone
+ * the timeZone to use for date fields
+ * @param dateMathParser
+ * the {@link DateMathParser} to use for date fields
+ * @return A {@link Relation} indicating the overlap of the range of terms
+ * for the field with the query range. This method will return:
+ *
+ * - {@link Relation#WITHIN} if the range of terms for the field
+ * in the shard is completely within the query range
+ * - {@link Relation#DISJOINT} if the range of terms for the field
+ * in the shard is completely outside the query range
+ * - {@link Relation#INTERSECTS} if the range of terms for the
+ * field in the shard intersects with the query range
+ *
+ * @throws IOException
+ * if the index cannot be read
+ */
+ public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper,
+ DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException {
+ MappedFieldType mappedFieldType = mapperService.fullName(fieldName);
+ FieldStats> fieldStats = get(fieldName);
+ if (fieldStats == null) {
+ // No fieldStats for the field so the field doesn't exist on
+ // this shard, so relation is DISJOINT
+ return Relation.DISJOINT;
+ } else {
+ // Convert the from and to values to Strings so they can be used
+ // in the IndexConstraints. Since DateTime is represented as a
+ // Long field in Lucene we need to use the millisecond value of
+ // the DateTime in that case
+ String fromString = null;
+ if (from != null) {
+ if (mappedFieldType instanceof DateFieldType) {
+ long millis = ((DateFieldType) mappedFieldType).parseToMilliseconds(from, !includeLower, timeZone, dateMathParser);
+ fromString = fieldStats.stringValueOf(millis, null);
+ } else if (mappedFieldType instanceof IpFieldType) {
+ if (from instanceof BytesRef) {
+ from = ((BytesRef) from).utf8ToString();
+ }
+ long ipAsLong = ((IpFieldType) mappedFieldType).value(from);
+ fromString = fieldStats.stringValueOf(ipAsLong, null);
+ } else {
+ fromString = fieldStats.stringValueOf(from, null);
+ }
+ }
+ String toString = null;
+ if (to != null) {
+ if (mappedFieldType instanceof DateFieldType) {
+ long millis = ((DateFieldType) mappedFieldType).parseToMilliseconds(to, includeUpper, timeZone, dateMathParser);
+ toString = fieldStats.stringValueOf(millis, null);
+ } else if (mappedFieldType instanceof IpFieldType) {
+ if (to instanceof BytesRef) {
+ to = ((BytesRef) to).utf8ToString();
+ }
+ long ipAsLong = ((IpFieldType) mappedFieldType).value(to);
+ toString = fieldStats.stringValueOf(ipAsLong, null);
+ } else {
+ toString = fieldStats.stringValueOf(to, null);
+ }
+ }
+ if ((from == null || fieldStats
+ .match(new IndexConstraint(fieldName, Property.MIN, includeLower ? Comparison.GTE : Comparison.GT, fromString)))
+ && (to == null || fieldStats.match(
+ new IndexConstraint(fieldName, Property.MAX, includeUpper ? Comparison.LTE : Comparison.LT, toString)))) {
+ // If the min and max terms for the field are both within
+ // the query range then all documents will match so relation is
+ // WITHIN
+ return Relation.WITHIN;
+ } else if ((to != null && fieldStats
+ .match(new IndexConstraint(fieldName, Property.MIN, includeUpper ? Comparison.GT : Comparison.GTE, toString)))
+ || (from != null && fieldStats.match(
+ new IndexConstraint(fieldName, Property.MAX, includeLower ? Comparison.LT : Comparison.LTE, fromString)))) {
+ // If the min and max terms are both outside the query range
+ // then no document will match so relation is DISJOINT (N.B.
+ // since from <= to we only need
+ // to check one bould for each side of the query range)
+ return Relation.DISJOINT;
+ }
+ }
+ // Range of terms doesn't match any of the constraints so must INTERSECT
+ return Relation.INTERSECTS;
+ }
+
+ /**
+ * An enum used to describe the relation between the range of terms in a
+ * shard when compared with a query range
+ */
+ public static enum Relation {
+ WITHIN, INTERSECTS, DISJOINT;
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
index 414ea0f7e9c..eaf897e7fbd 100755
--- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
@@ -33,12 +33,12 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.object.ObjectMapper;
+import org.elasticsearch.index.percolator.PercolatorFieldMapper;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.InvalidTypeNameException;
import org.elasticsearch.indices.TypeMissingException;
import org.elasticsearch.indices.mapper.MapperRegistry;
-import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.script.ScriptService;
import java.io.Closeable;
@@ -333,7 +333,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
private boolean typeNameStartsWithIllegalDot(DocumentMapper mapper) {
- return mapper.type().startsWith(".") && !PercolatorService.TYPE_NAME.equals(mapper.type());
+ return mapper.type().startsWith(".") && !PercolatorFieldMapper.TYPE_NAME.equals(mapper.type());
}
private boolean assertSerialization(DocumentMapper mapper) {
@@ -405,7 +405,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
public DocumentMapper parse(String mappingType, CompressedXContent mappingSource, boolean applyDefault) throws MapperParsingException {
String defaultMappingSource;
- if (PercolatorService.TYPE_NAME.equals(mappingType)) {
+ if (PercolatorFieldMapper.TYPE_NAME.equals(mappingType)) {
defaultMappingSource = this.defaultPercolatorMappingSource;
} else {
defaultMappingSource = this.defaultMappingSource;
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java
index 048c58297cb..724c37fcfcd 100644
--- a/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java
@@ -417,10 +417,15 @@ public class DateFieldMapper extends NumberFieldMapper {
}
public long parseToMilliseconds(Object value, boolean inclusive, @Nullable DateTimeZone zone, @Nullable DateMathParser forcedDateParser) {
+ if (value instanceof Long) {
+ return ((Long) value).longValue();
+ }
+
DateMathParser dateParser = dateMathParser();
if (forcedDateParser != null) {
dateParser = forcedDateParser;
}
+
String strValue;
if (value instanceof BytesRef) {
strValue = ((BytesRef) value).utf8ToString();
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java
index 9a4cf70782b..2ffb5d4ecf5 100644
--- a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java
@@ -22,12 +22,14 @@ package org.elasticsearch.index.mapper.ip;
import org.apache.lucene.analysis.LegacyNumericTokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
+import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
@@ -262,6 +264,13 @@ public class IpFieldMapper extends NumberFieldMapper {
iValue + iSim,
true, true);
}
+
+ @Override
+ public FieldStats stats(Terms terms, int maxDoc) throws IOException {
+ long minValue = LegacyNumericUtils.getMinLong(terms);
+ long maxValue = LegacyNumericUtils.getMaxLong(terms);
+ return new FieldStats.Ip(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue);
+ }
}
protected IpFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolateStats.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolateStats.java
deleted file mode 100644
index f4c899dff9a..00000000000
--- a/core/src/main/java/org/elasticsearch/index/percolator/PercolateStats.java
+++ /dev/null
@@ -1,164 +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.percolator;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.io.stream.Streamable;
-import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentBuilderString;
-
-import java.io.IOException;
-
-/**
- * Exposes percolator related statistics.
- */
-public class PercolateStats implements Streamable, ToXContent {
-
- private long percolateCount;
- private long percolateTimeInMillis;
- private long current;
- private long memorySizeInBytes = -1;
- private long numQueries;
-
- /**
- * Noop constructor for serialization purposes.
- */
- public PercolateStats() {
- }
-
- PercolateStats(long percolateCount, long percolateTimeInMillis, long current, long memorySizeInBytes, long numQueries) {
- this.percolateCount = percolateCount;
- this.percolateTimeInMillis = percolateTimeInMillis;
- this.current = current;
- this.memorySizeInBytes = memorySizeInBytes;
- this.numQueries = numQueries;
- }
-
- /**
- * @return The number of times the percolate api has been invoked.
- */
- public long getCount() {
- return percolateCount;
- }
-
- /**
- * @return The total amount of time spend in the percolate api
- */
- public long getTimeInMillis() {
- return percolateTimeInMillis;
- }
-
- /**
- * @return The total amount of time spend in the percolate api
- */
- public TimeValue getTime() {
- return new TimeValue(getTimeInMillis());
- }
-
- /**
- * @return The total amount of active percolate api invocations.
- */
- public long getCurrent() {
- return current;
- }
-
- /**
- * @return The total number of loaded percolate queries.
- */
- public long getNumQueries() {
- return numQueries;
- }
-
- /**
- * @return Temporarily returns -1
, but this used to return the total size the loaded queries take in
- * memory, but this is disabled now because the size estimation was too expensive cpu wise. This will be enabled
- * again when a cheaper size estimation can be found.
- */
- public long getMemorySizeInBytes() {
- return memorySizeInBytes;
- }
-
- /**
- * @return The total size the loaded queries take in memory.
- */
- public ByteSizeValue getMemorySize() {
- return new ByteSizeValue(memorySizeInBytes);
- }
-
- @Override
- public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
- builder.startObject(Fields.PERCOLATE);
- builder.field(Fields.TOTAL, percolateCount);
- builder.timeValueField(Fields.TIME_IN_MILLIS, Fields.TIME, percolateTimeInMillis);
- builder.field(Fields.CURRENT, current);
- builder.field(Fields.MEMORY_SIZE_IN_BYTES, memorySizeInBytes);
- builder.field(Fields.MEMORY_SIZE, getMemorySize());
- builder.field(Fields.QUERIES, getNumQueries());
- builder.endObject();
- return builder;
- }
-
- public void add(PercolateStats percolate) {
- if (percolate == null) {
- return;
- }
-
- percolateCount += percolate.getCount();
- percolateTimeInMillis += percolate.getTimeInMillis();
- current += percolate.getCurrent();
- numQueries += percolate.getNumQueries();
- }
-
- static final class Fields {
- static final XContentBuilderString PERCOLATE = new XContentBuilderString("percolate");
- static final XContentBuilderString TOTAL = new XContentBuilderString("total");
- static final XContentBuilderString TIME = new XContentBuilderString("time");
- static final XContentBuilderString TIME_IN_MILLIS = new XContentBuilderString("time_in_millis");
- static final XContentBuilderString CURRENT = new XContentBuilderString("current");
- static final XContentBuilderString MEMORY_SIZE_IN_BYTES = new XContentBuilderString("memory_size_in_bytes");
- static final XContentBuilderString MEMORY_SIZE = new XContentBuilderString("memory_size");
- static final XContentBuilderString QUERIES = new XContentBuilderString("queries");
- }
-
- public static PercolateStats readPercolateStats(StreamInput in) throws IOException {
- PercolateStats stats = new PercolateStats();
- stats.readFrom(in);
- return stats;
- }
-
- @Override
- public void readFrom(StreamInput in) throws IOException {
- percolateCount = in.readVLong();
- percolateTimeInMillis = in.readVLong();
- current = in.readVLong();
- numQueries = in.readVLong();
- }
-
- @Override
- public void writeTo(StreamOutput out) throws IOException {
- out.writeVLong(percolateCount);
- out.writeVLong(percolateTimeInMillis);
- out.writeVLong(current);
- out.writeVLong(numQueries);
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorFieldMapper.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorFieldMapper.java
index f44d454655e..338de5c333d 100644
--- a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorFieldMapper.java
+++ b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorFieldMapper.java
@@ -19,34 +19,50 @@
package org.elasticsearch.index.percolator;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.Query;
+import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
+import org.elasticsearch.index.mapper.core.BinaryFieldMapper;
import org.elasticsearch.index.mapper.core.KeywordFieldMapper;
+import org.elasticsearch.index.mapper.core.StringFieldMapper;
+import org.elasticsearch.index.query.ParsedQuery;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class PercolatorFieldMapper extends FieldMapper {
+ public static final String TYPE_NAME = ".percolator";
public static final String NAME = "query";
public static final String CONTENT_TYPE = "percolator";
public static final PercolatorFieldType FIELD_TYPE = new PercolatorFieldType();
private static final String EXTRACTED_TERMS_FIELD_NAME = "extracted_terms";
private static final String UNKNOWN_QUERY_FIELD_NAME = "unknown_query";
+ private static final String QUERY_BUILDER_FIELD_NAME = "query_builder_field";
+
public static final String EXTRACTED_TERMS_FULL_FIELD_NAME = NAME + "." + EXTRACTED_TERMS_FIELD_NAME;
public static final String UNKNOWN_QUERY_FULL_FIELD_NAME = NAME + "." + UNKNOWN_QUERY_FIELD_NAME;
+ public static final String QUERY_BUILDER_FULL_FIELD_NAME = NAME + "." + QUERY_BUILDER_FIELD_NAME;
public static class Builder extends FieldMapper.Builder {
@@ -60,19 +76,29 @@ public class PercolatorFieldMapper extends FieldMapper {
@Override
public PercolatorFieldMapper build(BuilderContext context) {
context.path().add(name);
- KeywordFieldMapper extractedTermsField = createStringFieldBuilder(EXTRACTED_TERMS_FIELD_NAME).build(context);
- KeywordFieldMapper unknownQueryField = createStringFieldBuilder(UNKNOWN_QUERY_FIELD_NAME).build(context);
+ KeywordFieldMapper extractedTermsField = createExtractQueryFieldBuilder(EXTRACTED_TERMS_FIELD_NAME).build(context);
+ KeywordFieldMapper unknownQueryField = createExtractQueryFieldBuilder(UNKNOWN_QUERY_FIELD_NAME).build(context);
+ BinaryFieldMapper queryBuilderField = createQueryBuilderFieldBuilder().build(context);
context.path().remove();
- return new PercolatorFieldMapper(name(), fieldType, defaultFieldType, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo, queryShardContext, extractedTermsField, unknownQueryField);
+ return new PercolatorFieldMapper(name(), fieldType, defaultFieldType, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo, queryShardContext, extractedTermsField, unknownQueryField, queryBuilderField);
}
- static KeywordFieldMapper.Builder createStringFieldBuilder(String name) {
+ static KeywordFieldMapper.Builder createExtractQueryFieldBuilder(String name) {
KeywordFieldMapper.Builder queryMetaDataFieldBuilder = new KeywordFieldMapper.Builder(name);
queryMetaDataFieldBuilder.docValues(false);
queryMetaDataFieldBuilder.store(false);
queryMetaDataFieldBuilder.indexOptions(IndexOptions.DOCS);
return queryMetaDataFieldBuilder;
}
+
+ static BinaryFieldMapper.Builder createQueryBuilderFieldBuilder() {
+ BinaryFieldMapper.Builder builder = new BinaryFieldMapper.Builder(QUERY_BUILDER_FIELD_NAME);
+ builder.docValues(true);
+ builder.indexOptions(IndexOptions.NONE);
+ builder.store(false);
+ builder.fieldType().setDocValuesType(DocValuesType.BINARY);
+ return builder;
+ }
}
public static class TypeParser implements FieldMapper.TypeParser {
@@ -111,26 +137,81 @@ public class PercolatorFieldMapper extends FieldMapper {
private final QueryShardContext queryShardContext;
private final KeywordFieldMapper queryTermsField;
private final KeywordFieldMapper unknownQueryField;
+ private final BinaryFieldMapper queryBuilderField;
- public PercolatorFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, MultiFields multiFields, CopyTo copyTo, QueryShardContext queryShardContext, KeywordFieldMapper queryTermsField, KeywordFieldMapper unknownQueryField) {
+ public PercolatorFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
+ Settings indexSettings, MultiFields multiFields, CopyTo copyTo, QueryShardContext queryShardContext,
+ KeywordFieldMapper queryTermsField, KeywordFieldMapper unknownQueryField,
+ BinaryFieldMapper queryBuilderField) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
this.queryShardContext = queryShardContext;
this.queryTermsField = queryTermsField;
this.unknownQueryField = unknownQueryField;
- this.mapUnmappedFieldAsString = PercolatorQueriesRegistry.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING.get(indexSettings);
+ this.queryBuilderField = queryBuilderField;
+ this.mapUnmappedFieldAsString = PercolatorQueryCache.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING.get(indexSettings);
}
@Override
public Mapper parse(ParseContext context) throws IOException {
QueryShardContext queryShardContext = new QueryShardContext(this.queryShardContext);
- Query query = PercolatorQueriesRegistry.parseQuery(queryShardContext, mapUnmappedFieldAsString, context.parser());
+ QueryBuilder> queryBuilder = parseQueryBuilder(queryShardContext.parseContext(), context.parser());
+ // Fetching of terms, shapes and indexed scripts happen during this rewrite:
+ queryBuilder = queryBuilder.rewrite(queryShardContext);
+
+ try (XContentBuilder builder = XContentFactory.contentBuilder(PercolatorQueryCache.QUERY_BUILDER_CONTENT_TYPE)) {
+ queryBuilder.toXContent(builder, new MapParams(Collections.emptyMap()));
+ builder.flush();
+ byte[] queryBuilderAsBytes = builder.bytes().toBytes();
+ context.doc().add(new Field(queryBuilderField.name(), queryBuilderAsBytes, queryBuilderField.fieldType()));
+ }
+
+ Query query = toQuery(queryShardContext, mapUnmappedFieldAsString, queryBuilder);
ExtractQueryTermsService.extractQueryTerms(query, context.doc(), queryTermsField.name(), unknownQueryField.name(), queryTermsField.fieldType());
return null;
}
+ public static Query parseQuery(QueryShardContext context, boolean mapUnmappedFieldsAsString, XContentParser parser) throws IOException {
+ return toQuery(context, mapUnmappedFieldsAsString, parseQueryBuilder(context.parseContext(), parser));
+ }
+
+ static Query toQuery(QueryShardContext context, boolean mapUnmappedFieldsAsString, QueryBuilder> queryBuilder) throws IOException {
+ context.reset();
+ // This means that fields in the query need to exist in the mapping prior to registering this query
+ // The reason that this is required, is that if a field doesn't exist then the query assumes defaults, which may be undesired.
+ //
+ // Even worse when fields mentioned in percolator queries do go added to map after the queries have been registered
+ // then the percolator queries don't work as expected any more.
+ //
+ // Query parsing can't introduce new fields in mappings (which happens when registering a percolator query),
+ // because field type can't be inferred from queries (like document do) so the best option here is to disallow
+ // the usage of unmapped fields in percolator queries to avoid unexpected behaviour
+ //
+ // if index.percolator.map_unmapped_fields_as_string is set to true, query can contain unmapped fields which will be mapped
+ // as an analyzed string.
+ context.setAllowUnmappedFields(false);
+ context.setMapUnmappedFieldAsString(mapUnmappedFieldsAsString);
+ context.parseFieldMatcher(context.getIndexSettings().getParseFieldMatcher());
+ try {
+ return queryBuilder.toQuery(context);
+ } finally {
+ context.reset();
+ }
+ }
+
+ static QueryBuilder> parseQueryBuilder(QueryParseContext context, XContentParser parser) {
+ context.reset(parser);
+ try {
+ return context.parseInnerQueryBuilder();
+ } catch (IOException e) {
+ throw new ParsingException(parser.getTokenLocation(), "Failed to parse", e);
+ } finally {
+ context.reset(null);
+ }
+ }
+
@Override
public Iterator iterator() {
- return Arrays.asList(queryTermsField, unknownQueryField).iterator();
+ return Arrays.asList(queryTermsField, unknownQueryField, queryBuilderField).iterator();
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorHighlightSubFetchPhase.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorHighlightSubFetchPhase.java
new file mode 100644
index 00000000000..c1f9720b53b
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorHighlightSubFetchPhase.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.index.percolator;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.text.Text;
+import org.elasticsearch.index.percolator.PercolatorFieldMapper;
+import org.elasticsearch.index.percolator.PercolatorQueryCache;
+import org.elasticsearch.index.query.ParsedQuery;
+import org.elasticsearch.index.query.PercolatorQuery;
+import org.elasticsearch.search.SearchParseElement;
+import org.elasticsearch.search.fetch.FetchSubPhase;
+import org.elasticsearch.search.highlight.HighlightPhase;
+import org.elasticsearch.search.highlight.SearchContextHighlight;
+import org.elasticsearch.search.internal.InternalSearchHit;
+import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.internal.SubSearchContext;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+// Highlighting in the case of the percolator query is a bit different, because the PercolatorQuery itself doesn't get highlighted,
+// but the source of the PercolatorQuery gets highlighted by each hit with type '.percolator' (percolator queries).
+public class PercolatorHighlightSubFetchPhase implements FetchSubPhase {
+
+ private final HighlightPhase highlightPhase;
+
+ @Inject
+ public PercolatorHighlightSubFetchPhase(HighlightPhase highlightPhase) {
+ this.highlightPhase = highlightPhase;
+ }
+
+ @Override
+ public boolean hitsExecutionNeeded(SearchContext context) {
+ return context.highlight() != null && locatePercolatorQuery(context.query()) != null;
+ }
+
+ @Override
+ public void hitsExecute(SearchContext context, InternalSearchHit[] hits) {
+ PercolatorQuery percolatorQuery = locatePercolatorQuery(context.query());
+ if (percolatorQuery == null) {
+ // shouldn't happen as we checked for the existence of a percolator query in hitsExecutionNeeded(...)
+ throw new IllegalStateException("couldn't locate percolator query");
+ }
+
+ List ctxs = context.searcher().getIndexReader().leaves();
+ PercolatorQueryCache queriesRegistry = context.percolatorQueryCache();
+ IndexSearcher percolatorIndexSearcher = percolatorQuery.getPercolatorIndexSearcher();
+
+ LeafReaderContext percolatorLeafReaderContext = percolatorIndexSearcher.getIndexReader().leaves().get(0);
+ FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext();
+ SubSearchContext subSearchContext =
+ createSubSearchContext(context, percolatorLeafReaderContext, percolatorQuery.getDocumentSource());
+
+ for (InternalSearchHit hit : hits) {
+ if (PercolatorFieldMapper.TYPE_NAME.equals(hit.getType())) {
+ LeafReaderContext ctx = ctxs.get(ReaderUtil.subIndex(hit.docId(), ctxs));
+ Query query = queriesRegistry.getQueries(ctx).getQuery(hit.docId() - ctx.docBase);
+ subSearchContext.parsedQuery(new ParsedQuery(query));
+ hitContext.reset(
+ new InternalSearchHit(0, "unknown", new Text(percolatorQuery.getDocumentType()), Collections.emptyMap()),
+ percolatorLeafReaderContext, 0, percolatorIndexSearcher
+ );
+ hitContext.cache().clear();
+ highlightPhase.hitExecute(subSearchContext, hitContext);
+ hit.highlightFields().putAll(hitContext.hit().getHighlightFields());
+ }
+ }
+
+ }
+
+ @Override
+ public Map parseElements() {
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public boolean hitExecutionNeeded(SearchContext context) {
+ return false;
+ }
+
+ @Override
+ public void hitExecute(SearchContext context, HitContext hitContext) {
+ }
+
+ static PercolatorQuery locatePercolatorQuery(Query query) {
+ if (query instanceof PercolatorQuery) {
+ return (PercolatorQuery) query;
+ } else if (query instanceof BooleanQuery) {
+ for (BooleanClause clause : ((BooleanQuery) query).clauses()) {
+ PercolatorQuery result = locatePercolatorQuery(clause.getQuery());
+ if (result != null) {
+ return result;
+ }
+ }
+ } else if (query instanceof ConstantScoreQuery) {
+ return locatePercolatorQuery(((ConstantScoreQuery) query).getQuery());
+ } else if (query instanceof BoostQuery) {
+ return locatePercolatorQuery(((BoostQuery) query).getQuery());
+ }
+
+ return null;
+ }
+
+ private SubSearchContext createSubSearchContext(SearchContext context, LeafReaderContext leafReaderContext, BytesReference source) {
+ SubSearchContext subSearchContext = new SubSearchContext(context);
+ subSearchContext.highlight(new SearchContextHighlight(context.highlight().fields()));
+ // Enforce highlighting by source, because MemoryIndex doesn't support stored fields.
+ subSearchContext.highlight().globalForceSource(true);
+ subSearchContext.lookup().source().setSegmentAndDocument(leafReaderContext, 0);
+ subSearchContext.lookup().source().setSource(source);
+ return subSearchContext;
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java
deleted file mode 100644
index 0a0cb9e96d9..00000000000
--- a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java
+++ /dev/null
@@ -1,236 +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.percolator;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.ParsingException;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.metrics.CounterMetric;
-import org.elasticsearch.common.metrics.MeanMetric;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
-import org.elasticsearch.common.xcontent.XContentHelper;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.index.engine.Engine;
-import org.elasticsearch.index.mapper.Uid;
-import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
-import org.elasticsearch.index.mapper.internal.UidFieldMapper;
-import org.elasticsearch.index.query.QueryShardContext;
-import org.elasticsearch.index.shard.AbstractIndexShardComponent;
-import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.percolator.PercolatorService;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Each shard will have a percolator registry even if there isn't a {@link PercolatorService#TYPE_NAME} document type in the index.
- * For shards with indices that have no {@link PercolatorService#TYPE_NAME} document type, this will hold no percolate queries.
- *
- * Once a document type has been created, the real-time percolator will start to listen to write events and update the
- * this registry with queries in real time.
- */
-public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent implements Closeable {
-
- public final static Setting INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING =
- Setting.boolSetting("index.percolator.map_unmapped_fields_as_string", false, Property.IndexScope);
-
- private final ConcurrentMap percolateQueries = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
- private final QueryShardContext queryShardContext;
- private boolean mapUnmappedFieldsAsString;
- private final MeanMetric percolateMetric = new MeanMetric();
- private final CounterMetric currentMetric = new CounterMetric();
- private final CounterMetric numberOfQueries = new CounterMetric();
-
- public PercolatorQueriesRegistry(ShardId shardId, IndexSettings indexSettings, QueryShardContext queryShardContext) {
- super(shardId, indexSettings);
- this.queryShardContext = queryShardContext;
- this.mapUnmappedFieldsAsString = indexSettings.getValue(INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING);
- }
-
- public ConcurrentMap getPercolateQueries() {
- return percolateQueries;
- }
-
- @Override
- public void close() {
- clear();
- }
-
- public void clear() {
- percolateQueries.clear();
- }
-
-
- public void addPercolateQuery(String idAsString, BytesReference source) {
- Query newquery = parsePercolatorDocument(idAsString, source);
- BytesRef id = new BytesRef(idAsString);
- percolateQueries.put(id, newquery);
- numberOfQueries.inc();
-
- }
-
- public void removePercolateQuery(String idAsString) {
- BytesRef id = new BytesRef(idAsString);
- Query query = percolateQueries.remove(id);
- if (query != null) {
- numberOfQueries.dec();
- }
- }
-
- public Query parsePercolatorDocument(String id, BytesReference source) {
- try (XContentParser sourceParser = XContentHelper.createParser(source)) {
- String currentFieldName = null;
- XContentParser.Token token = sourceParser.nextToken(); // move the START_OBJECT
- if (token != XContentParser.Token.START_OBJECT) {
- throw new ElasticsearchException("failed to parse query [" + id + "], not starting with OBJECT");
- }
- while ((token = sourceParser.nextToken()) != XContentParser.Token.END_OBJECT) {
- if (token == XContentParser.Token.FIELD_NAME) {
- currentFieldName = sourceParser.currentName();
- } else if (token == XContentParser.Token.START_OBJECT) {
- if ("query".equals(currentFieldName)) {
- return parseQuery(queryShardContext, mapUnmappedFieldsAsString, sourceParser);
- } else {
- sourceParser.skipChildren();
- }
- } else if (token == XContentParser.Token.START_ARRAY) {
- sourceParser.skipChildren();
- }
- }
- } catch (Exception e) {
- throw new PercolatorException(shardId().getIndex(), "failed to parse query [" + id + "]", e);
- }
- return null;
- }
-
- public static Query parseQuery(QueryShardContext queryShardContext, boolean mapUnmappedFieldsAsString, XContentParser parser) {
- QueryShardContext context = new QueryShardContext(queryShardContext);
- try {
- context.reset(parser);
- // This means that fields in the query need to exist in the mapping prior to registering this query
- // The reason that this is required, is that if a field doesn't exist then the query assumes defaults, which may be undesired.
- //
- // Even worse when fields mentioned in percolator queries do go added to map after the queries have been registered
- // then the percolator queries don't work as expected any more.
- //
- // Query parsing can't introduce new fields in mappings (which happens when registering a percolator query),
- // because field type can't be inferred from queries (like document do) so the best option here is to disallow
- // the usage of unmapped fields in percolator queries to avoid unexpected behaviour
- //
- // if index.percolator.map_unmapped_fields_as_string is set to true, query can contain unmapped fields which will be mapped
- // as an analyzed string.
- context.setAllowUnmappedFields(false);
- context.setMapUnmappedFieldAsString(mapUnmappedFieldsAsString);
- return context.parseInnerQuery();
- } catch (IOException e) {
- throw new ParsingException(parser.getTokenLocation(), "Failed to parse", e);
- } finally {
- context.reset(null);
- }
- }
-
- public void loadQueries(IndexReader reader) {
- logger.trace("loading percolator queries...");
- final int loadedQueries;
- try {
- Query query = new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME));
- QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger);
- IndexSearcher indexSearcher = new IndexSearcher(reader);
- indexSearcher.setQueryCache(null);
- indexSearcher.search(query, queryCollector);
- Map queries = queryCollector.queries();
- for (Map.Entry entry : queries.entrySet()) {
- percolateQueries.put(entry.getKey(), entry.getValue());
- numberOfQueries.inc();
- }
- loadedQueries = queries.size();
- } catch (Exception e) {
- throw new PercolatorException(shardId.getIndex(), "failed to load queries from percolator index", e);
- }
- logger.debug("done loading [{}] percolator queries", loadedQueries);
- }
-
- public boolean isPercolatorQuery(Engine.Index operation) {
- if (PercolatorService.TYPE_NAME.equals(operation.type())) {
- parsePercolatorDocument(operation.id(), operation.source());
- return true;
- }
- return false;
- }
-
- public boolean isPercolatorQuery(Engine.Delete operation) {
- return PercolatorService.TYPE_NAME.equals(operation.type());
- }
-
- public synchronized void updatePercolateQuery(Engine engine, String id) {
- // this can be called out of order as long as for every change to a percolator document it's invoked. This will always
- // fetch the latest change but might fetch the same change twice if updates / deletes happen concurrently.
- try (Engine.GetResult getResult = engine.get(new Engine.Get(true, new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(PercolatorService.TYPE_NAME, id))))) {
- if (getResult.exists()) {
- addPercolateQuery(id, getResult.source().source);
- } else {
- removePercolateQuery(id);
- }
- }
- }
-
- public void prePercolate() {
- currentMetric.inc();
- }
-
- public void postPercolate(long tookInNanos) {
- currentMetric.dec();
- percolateMetric.inc(tookInNanos);
- }
-
- /**
- * @return The current metrics
- */
- public PercolateStats stats() {
- return new PercolateStats(percolateMetric.count(), TimeUnit.NANOSECONDS.toMillis(percolateMetric.sum()), currentMetric.count(), -1, numberOfQueries.count());
- }
-
- // Enable when a more efficient manner is found for estimating the size of a Lucene query.
- /*private static long computeSizeInMemory(HashedBytesRef id, Query query) {
- long size = (3 * RamUsageEstimator.NUM_BYTES_INT) + RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + id.bytes.bytes.length;
- size += RamEstimator.sizeOf(query);
- return size;
- }
-
- private static final class RamEstimator {
- // we move this into it's own class to exclude it from the forbidden API checks
- // it's fine to use here!
- static long sizeOf(Query query) {
- return RamUsageEstimator.sizeOf(query);
- }
- }*/
-}
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueryCache.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueryCache.java
new file mode 100644
index 00000000000..7c9602b4909
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueryCache.java
@@ -0,0 +1,266 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.index.percolator;
+
+import com.carrotsearch.hppc.IntObjectHashMap;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.Version;
+import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.cache.Cache;
+import org.elasticsearch.common.cache.CacheBuilder;
+import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
+import org.elasticsearch.common.settings.Setting;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.XContent;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.index.AbstractIndexComponent;
+import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.IndexWarmer;
+import org.elasticsearch.index.engine.Engine;
+import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
+import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
+import org.elasticsearch.index.query.PercolatorQuery;
+import org.elasticsearch.index.query.QueryShardContext;
+import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.index.shard.ShardUtils;
+import org.elasticsearch.threadpool.ThreadPool;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.function.Supplier;
+
+public final class PercolatorQueryCache extends AbstractIndexComponent
+ implements Closeable, LeafReader.CoreClosedListener, PercolatorQuery.QueryRegistry {
+
+ public final static Setting INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING =
+ Setting.boolSetting("index.percolator.map_unmapped_fields_as_string", false, Setting.Property.IndexScope);
+
+ public final static XContentType QUERY_BUILDER_CONTENT_TYPE = XContentType.SMILE;
+
+ private final Supplier queryShardContextSupplier;
+ private final Cache