diff --git a/core/src/main/java/org/elasticsearch/action/fieldstats/FieldStats.java b/core/src/main/java/org/elasticsearch/action/fieldstats/FieldStats.java index 2893b16a251..f661c4a8b58 100644 --- a/core/src/main/java/org/elasticsearch/action/fieldstats/FieldStats.java +++ b/core/src/main/java/org/elasticsearch/action/fieldstats/FieldStats.java @@ -28,6 +28,8 @@ import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.index.mapper.ip.IpFieldMapper; +import org.joda.time.DateTime; import java.io.IOException; @@ -146,6 +148,17 @@ public abstract class FieldStats> implements Streamable, */ protected abstract T valueOf(String value, String optionalFormat); + /** + * @param value + * The value to be converted to a String + * @param optionalFormat + * A string describing how to print the specified value. Whether + * this parameter is supported depends on the implementation. If + * optionalFormat is specified and the implementation doesn't + * support it an {@link UnsupportedOperationException} is thrown + */ + public abstract String stringValueOf(Object value, String optionalFormat); + /** * Merges the provided stats into this stats instance. */ @@ -274,6 +287,18 @@ public abstract class FieldStats> implements Streamable, return java.lang.Long.valueOf(value); } + @Override + public String stringValueOf(Object value, String optionalFormat) { + if (optionalFormat != null) { + throw new UnsupportedOperationException("custom format isn't supported"); + } + if (value instanceof Number) { + return java.lang.Long.toString(((Number) value).longValue()); + } else { + throw new IllegalArgumentException("value must be a Long: " + value); + } + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -327,6 +352,18 @@ public abstract class FieldStats> implements Streamable, return java.lang.Float.valueOf(value); } + @Override + public String stringValueOf(Object value, String optionalFormat) { + if (optionalFormat != null) { + throw new UnsupportedOperationException("custom format isn't supported"); + } + if (value instanceof Number) { + return java.lang.Float.toString(((Number) value).floatValue()); + } else { + throw new IllegalArgumentException("value must be a Float: " + value); + } + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -380,6 +417,18 @@ public abstract class FieldStats> implements Streamable, return java.lang.Double.valueOf(value); } + @Override + public String stringValueOf(Object value, String optionalFormat) { + if (optionalFormat != null) { + throw new UnsupportedOperationException("custom format isn't supported"); + } + if (value instanceof Number) { + return java.lang.Double.toString(((Number) value).doubleValue()); + } else { + throw new IllegalArgumentException("value must be a Double: " + value); + } + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -437,6 +486,18 @@ public abstract class FieldStats> implements Streamable, return new BytesRef(value); } + @Override + public String stringValueOf(Object value, String optionalFormat) { + if (optionalFormat != null) { + throw new UnsupportedOperationException("custom format isn't supported"); + } + if (value instanceof BytesRef) { + return ((BytesRef) value).utf8ToString(); + } else { + throw new IllegalArgumentException("value must be a BytesRef: " + value); + } + } + @Override protected void toInnerXContent(XContentBuilder builder) throws IOException { builder.field(Fields.MIN_VALUE, getMinValueAsString()); @@ -490,6 +551,25 @@ public abstract class FieldStats> implements Streamable, return dateFormatter.parser().parseMillis(value); } + @Override + public String stringValueOf(Object value, String optionalFormat) { + FormatDateTimeFormatter dateFormatter = this.dateFormatter; + if (optionalFormat != null) { + dateFormatter = Joda.forPattern(optionalFormat); + } + long millis; + if (value instanceof java.lang.Long) { + millis = ((java.lang.Long) value).longValue(); + } else if (value instanceof DateTime) { + millis = ((DateTime) value).getMillis(); + } else if (value instanceof BytesRef) { + millis = dateFormatter.parser().parseMillis(((BytesRef) value).utf8ToString()); + } else { + throw new IllegalArgumentException("value must be either a DateTime or a long: " + value); + } + return dateFormatter.printer().print(millis); + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -504,6 +584,28 @@ public abstract class FieldStats> implements Streamable, } + public static class Ip extends Long { + + public Ip(int maxDoc, int docCount, long sumDocFreq, long sumTotalTermFreq, long minValue, long maxValue) { + super(maxDoc, docCount, sumDocFreq, sumTotalTermFreq, minValue, maxValue); + } + + protected Ip(int type, long maxDoc, long docCount, long sumDocFreq, long sumTotalTermFreq, long minValue, long maxValue) { + super(type, maxDoc, docCount, sumDocFreq, sumTotalTermFreq, minValue, maxValue); + } + + public Ip() { + } + + @Override + public String stringValueOf(Object value, String optionalFormat) { + if (value instanceof BytesRef) { + return super.stringValueOf(IpFieldMapper.ipToLong(((BytesRef) value).utf8ToString()), optionalFormat); + } + return super.stringValueOf(value, optionalFormat); + } + } + public static FieldStats read(StreamInput in) throws IOException { FieldStats stats; byte type = in.readByte(); 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/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/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index e057aff06b1..11164659b3f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.query; import org.elasticsearch.client.Client; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.fieldstats.FieldStatsProvider; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ScriptService; @@ -31,6 +32,7 @@ public class QueryRewriteContext { protected final IndexSettings indexSettings; protected final IndicesQueriesRegistry indicesQueriesRegistry; protected final QueryParseContext parseContext; + protected FieldStatsProvider fieldStatsProvider; public QueryRewriteContext(IndexSettings indexSettings, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry) { this.scriptService = scriptService; @@ -39,6 +41,14 @@ public class QueryRewriteContext { this.parseContext = new QueryParseContext(indicesQueriesRegistry); } + public void setFieldStatsProvider(FieldStatsProvider fieldStatsProvider) { + this.fieldStatsProvider = fieldStatsProvider; + } + + public FieldStatsProvider getFieldStatsProvider() { + return fieldStatsProvider; + } + /** * Returns a clients to fetch resources from local or remove nodes. */ diff --git a/core/src/main/java/org/elasticsearch/index/query/RangeQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/RangeQueryBuilder.java index cd99bec0f74..2a627c54590 100644 --- a/core/src/main/java/org/elasticsearch/index/query/RangeQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/RangeQueryBuilder.java @@ -22,6 +22,10 @@ package org.elasticsearch.index.query; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermRangeQuery; 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.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -30,6 +34,7 @@ import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.fieldstats.FieldStatsProvider; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.core.DateFieldMapper; import org.joda.time.DateTimeZone; @@ -253,6 +258,48 @@ public class RangeQueryBuilder extends AbstractQueryBuilder i return NAME; } + @Override + protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) throws IOException { + FieldStatsProvider fieldStatsProvider = queryShardContext.getFieldStatsProvider(); + // If the fieldStatsProvider is null we are not on the shard and cannot + // rewrite so just return without rewriting + if (fieldStatsProvider != null) { + DateMathParser dateMathParser = format == null ? null : new DateMathParser(format); + FieldStatsProvider.Relation relation = fieldStatsProvider.isFieldWithinQuery(fieldName, from, to, includeUpper, includeLower, + timeZone, dateMathParser); + switch (relation) { + case DISJOINT: + return new MatchNoneQueryBuilder(); + case WITHIN: + FieldStats fieldStats = fieldStatsProvider.get(fieldName); + if (!(fieldStats.getMinValue().equals(from) && fieldStats.getMaxValue().equals(to) && includeUpper && includeLower)) { + // Rebuild the range query with the bounds for this shard. + // The includeLower/Upper values are preserved only if the + // bound has not been changed by the rewrite + RangeQueryBuilder newRangeQuery = new RangeQueryBuilder(fieldName); + String dateFormatString = format == null ? null : format.format(); + if (fieldStats.getMinValue().equals(fieldStats.getMaxValue())) { + newRangeQuery.from(fieldStats.getMinValue(), true); + newRangeQuery.to(fieldStats.getMaxValue(), true); + } else { + newRangeQuery.from(fieldStats.getMinValue(), includeLower || fieldStats.match(new IndexConstraint(fieldName, + Property.MIN, Comparison.GT, fieldStats.stringValueOf(from, dateFormatString)))); + newRangeQuery.to(fieldStats.getMaxValue(), includeUpper || fieldStats.match(new IndexConstraint(fieldName, + Property.MAX, Comparison.LT, fieldStats.stringValueOf(to, dateFormatString)))); + } + newRangeQuery.format = format; + newRangeQuery.timeZone = timeZone; + return newRangeQuery; + } else { + return this; + } + case INTERSECTS: + break; + } + } + return this; + } + @Override protected Query doToQuery(QueryShardContext context) throws IOException { Query query = null; diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index b0e1bbdbd2b..f587e7212cf 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -542,6 +542,7 @@ public class IndicesService extends AbstractLifecycleComponent i * @param indexSettings the shards index settings. * @throws IOException if an IOException occurs */ + @Override public void deleteShardStore(String reason, ShardLock lock, IndexSettings indexSettings) throws IOException { ShardId shardId = lock.getShardId(); logger.trace("{} deleting shard reason [{}]", shardId, reason); @@ -654,6 +655,7 @@ public class IndicesService extends AbstractLifecycleComponent i /** * Adds a pending delete for the given index shard. */ + @Override public void addPendingDelete(ShardId shardId, IndexSettings settings) { if (shardId == null) { throw new IllegalArgumentException("shardId must not be null"); diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 8a400418eb7..1694533f99a 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -49,6 +49,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.fieldstats.FieldStatsProvider; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.search.stats.ShardSearchStats; @@ -241,7 +242,7 @@ public class SearchService extends AbstractLifecycleComponent imp FutureUtils.cancel(keepAliveReaper); } - public DfsSearchResult executeDfsPhase(ShardSearchRequest request) { + public DfsSearchResult executeDfsPhase(ShardSearchRequest request) throws IOException { final SearchContext context = createAndPutContext(request); try { contextProcessing(context); @@ -270,7 +271,7 @@ public class SearchService extends AbstractLifecycleComponent imp } } - public QuerySearchResultProvider executeQueryPhase(ShardSearchRequest request) { + public QuerySearchResultProvider executeQueryPhase(ShardSearchRequest request) throws IOException { final SearchContext context = createAndPutContext(request); final ShardSearchStats shardSearchStats = context.indexShard().searchService(); try { @@ -362,7 +363,7 @@ public class SearchService extends AbstractLifecycleComponent imp } } - public QueryFetchSearchResult executeFetchPhase(ShardSearchRequest request) { + public QueryFetchSearchResult executeFetchPhase(ShardSearchRequest request) throws IOException { final SearchContext context = createAndPutContext(request); contextProcessing(context); try { @@ -519,7 +520,7 @@ public class SearchService extends AbstractLifecycleComponent imp return context; } - final SearchContext createAndPutContext(ShardSearchRequest request) { + final SearchContext createAndPutContext(ShardSearchRequest request) throws IOException { SearchContext context = createContext(request, null); boolean success = false; try { @@ -537,7 +538,7 @@ public class SearchService extends AbstractLifecycleComponent imp } } - final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) { + final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) throws IOException { IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexShard indexShard = indexService.getShard(request.shardId().getId()); SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), indexShard.shardId()); @@ -548,6 +549,8 @@ public class SearchService extends AbstractLifecycleComponent imp indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, defaultSearchTimeout, fetchPhase); + context.getQueryShardContext().setFieldStatsProvider(new FieldStatsProvider(engineSearcher, indexService.mapperService())); + request.rewrite(context.getQueryShardContext()); SearchContext.setCurrent(context); try { if (request.scroll() != null) { diff --git a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index e256838b756..a20ec535238 100644 --- a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryParseContext; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.script.Script; import org.elasticsearch.search.aggregations.AggregatorBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -729,6 +730,59 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ return ext; } + /** + * Rewrites this search source builder into its primitive form. e.g. by + * rewriting the QueryBuilder. If the builder did not change the identity + * reference must be returned otherwise the builder will be rewritten + * infinitely. + */ + public SearchSourceBuilder rewrite(QueryShardContext context) throws IOException { + assert (this.equals(shallowCopy(queryBuilder, postQueryBuilder))); + QueryBuilder queryBuilder = null; + if (this.queryBuilder != null) { + queryBuilder = this.queryBuilder.rewrite(context); + } + QueryBuilder postQueryBuilder = null; + if (this.postQueryBuilder != null) { + postQueryBuilder = this.postQueryBuilder.rewrite(context); + } + boolean rewritten = queryBuilder != this.queryBuilder || postQueryBuilder != this.postQueryBuilder; + if (rewritten) { + return shallowCopy(queryBuilder, postQueryBuilder); + } + return this; + } + + private SearchSourceBuilder shallowCopy(QueryBuilder queryBuilder, QueryBuilder postQueryBuilder) { + SearchSourceBuilder rewrittenBuilder = new SearchSourceBuilder(); + rewrittenBuilder.aggregations = aggregations; + rewrittenBuilder.explain = explain; + rewrittenBuilder.ext = ext; + rewrittenBuilder.fetchSourceContext = fetchSourceContext; + rewrittenBuilder.fieldDataFields = fieldDataFields; + rewrittenBuilder.fieldNames = fieldNames; + rewrittenBuilder.from = from; + rewrittenBuilder.highlightBuilder = highlightBuilder; + rewrittenBuilder.indexBoost = indexBoost; + rewrittenBuilder.innerHitsBuilder = innerHitsBuilder; + rewrittenBuilder.minScore = minScore; + rewrittenBuilder.postQueryBuilder = postQueryBuilder; + rewrittenBuilder.profile = profile; + rewrittenBuilder.queryBuilder = queryBuilder; + rewrittenBuilder.rescoreBuilders = rescoreBuilders; + rewrittenBuilder.scriptFields = scriptFields; + rewrittenBuilder.searchAfterBuilder = searchAfterBuilder; + rewrittenBuilder.size = size; + rewrittenBuilder.sorts = sorts; + rewrittenBuilder.stats = stats; + rewrittenBuilder.suggestBuilder = suggestBuilder; + rewrittenBuilder.terminateAfter = terminateAfter; + rewrittenBuilder.timeoutInMillis = timeoutInMillis; + rewrittenBuilder.trackScores = trackScores; + rewrittenBuilder.version = version; + return rewrittenBuilder; + } + /** * Create a new SearchSourceBuilder with attributes set by an xContent. */ diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java index 56ad8ed9467..31192350308 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.Template; import org.elasticsearch.search.Scroll; @@ -224,4 +225,15 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { // we could potentially keep it without copying, but then pay the price of extra unused bytes up to a page return out.bytes().copyBytesArray(); } + + @Override + public void rewrite(QueryShardContext context) throws IOException { + SearchSourceBuilder source = this.source; + SearchSourceBuilder rewritten = null; + while (rewritten != source) { + rewritten = source.rewrite(context); + source = rewritten; + } + this.source = source; + } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 82ff69078aa..aa148e215c8 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.internal; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.Template; import org.elasticsearch.search.Scroll; @@ -72,4 +73,10 @@ public interface ShardSearchRequest { * Returns the cache key for this shard search request, based on its content */ BytesReference cacheKey() throws IOException; + + /** + * Rewrites this request into its primitive form. e.g. by rewriting the + * QueryBuilder. + */ + void rewrite(QueryShardContext context) throws IOException; } diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java index dc19f84c7a7..cd6460a686f 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.Template; import org.elasticsearch.search.Scroll; @@ -156,4 +157,16 @@ public class ShardSearchTransportRequest extends TransportRequest implements Sha public boolean isProfile() { return shardSearchLocalRequest.isProfile(); } + + @Override + public void rewrite(QueryShardContext context) throws IOException { + shardSearchLocalRequest.rewrite(context); + } + + private ShardSearchTransportRequest shallowCopy(ShardSearchLocalRequest rewritten) { + ShardSearchTransportRequest newRequest = new ShardSearchTransportRequest(); + newRequest.originalIndices = originalIndices; + newRequest.shardSearchLocalRequest = rewritten; + return newRequest; + } } diff --git a/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java b/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java new file mode 100644 index 00000000000..cff2d13ce63 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java @@ -0,0 +1,99 @@ +/* + * 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.elasticsearch.action.admin.indices.refresh.RefreshResponse; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.indices.IndicesRequestCache; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESSingleNodeTestCase; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.equalTo; + +public class FieldStatsProviderRefreshTests extends ESSingleNodeTestCase { + + public void testQueryRewriteOnRefresh() throws Exception { + assertAcked(client().admin().indices().prepareCreate("index").addMapping("type", "s", "type=text") + .setSettings(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true, + IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, + IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .get()); + + // Index some documents + indexDocument("1", "d"); + indexDocument("2", "e"); + indexDocument("3", "f"); + refreshIndex(); + + // check request cache stats are clean + assertRequestCacheStats(0, 0); + + // Search for a range and check that it missed the cache (since its the + // first time it has run) + final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("g")).get(); + assertSearchResponse(r1); + assertThat(r1.getHits().getTotalHits(), equalTo(3L)); + assertRequestCacheStats(0, 1); + + // Search again and check it hits the cache + final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("g")).get(); + assertSearchResponse(r2); + assertThat(r2.getHits().getTotalHits(), equalTo(3L)); + assertRequestCacheStats(1, 1); + + // Index some more documents in the query range and refresh + indexDocument("4", "c"); + indexDocument("5", "g"); + refreshIndex(); + + // Search again and check the request cache for another miss since request cache should be invalidated by refresh + final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("g")).get(); + assertSearchResponse(r3); + assertThat(r3.getHits().getTotalHits(), equalTo(5L)); + assertRequestCacheStats(1, 2); + } + + private void assertRequestCacheStats(long expectedHits, long expectedMisses) { + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(expectedHits)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(expectedMisses)); + } + + private void refreshIndex() { + RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("index").get(); + assertThat(refreshResponse.getSuccessfulShards(), equalTo(refreshResponse.getSuccessfulShards())); + } + + private void indexDocument(String id, String sValue) { + IndexResponse response = client().prepareIndex("index", "type", id).setSource("s", sValue).get(); + assertThat(response.status(), anyOf(equalTo(RestStatus.OK), equalTo(RestStatus.CREATED))); + } +} diff --git a/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderTests.java b/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderTests.java new file mode 100644 index 00000000000..9cad8d3fc8d --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderTests.java @@ -0,0 +1,446 @@ +/* + * 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.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.store.BaseDirectoryWrapper; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.Engine.Searcher; +import org.elasticsearch.index.fieldstats.FieldStatsProvider.Relation; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MapperService.MergeReason; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.similarity.SimilarityService; +import org.elasticsearch.indices.IndicesModule; +import org.elasticsearch.indices.mapper.MapperRegistry; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.IndexSettingsModule; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.chrono.ISOChronology; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; + +import static org.elasticsearch.common.settings.Settings.settingsBuilder; +import static org.hamcrest.Matchers.equalTo; + +public class FieldStatsProviderTests extends ESTestCase { + + private DirectoryReader directoryReader; + private Searcher searcher; + private FieldStatsProvider fieldStatsProvider; + private BaseDirectoryWrapper dir; + private AnalysisRegistry analysisRegistry; + + @Before + public void setup() throws IOException { + Settings nodeSettings = settingsBuilder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); + IndexSettings settings = IndexSettingsModule.newIndexSettings("_na", nodeSettings); + SimilarityService similarityService = new SimilarityService(settings, Collections.emptyMap()); + analysisRegistry = new AnalysisRegistry(null, new Environment(nodeSettings)); + AnalysisService analysisService = analysisRegistry.build(settings); + IndicesModule indicesModule = new IndicesModule(); + MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); + MapperService service = new MapperService(settings, analysisService, similarityService, mapperRegistry, () -> null); + putMapping(service); + dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()); + indexDocument(service, w, "1", 50L, 50.2f, 50.2, "cherry", new DateTime(2014, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.10"); + indexDocument(service, w, "2", 60L, 60.1f, 60.1, "damson", new DateTime(2014, 2, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.20"); + indexDocument(service, w, "3", 70L, 70.6f, 70.6, "grape", new DateTime(2014, 3, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.30"); + indexDocument(service, w, "4", 80L, 80.2f, 80.2, "kiwi", new DateTime(2014, 4, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.40"); + indexDocument(service, w, "5", 90L, 90.4f, 90.4, "lemon", new DateTime(2014, 5, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.50"); + indexDocument(service, w, "6", 100L, 100.3f, 100.3, "orange", new DateTime(2014, 6, 1, 0, 0, 0, ISOChronology.getInstanceUTC()), + "10.10.0.60"); + directoryReader = DirectoryReader.open(w, true, true); + w.close(); + ShardId shard = new ShardId("index", "_na_", 0); + directoryReader = ElasticsearchDirectoryReader.wrap(directoryReader, shard); + IndexSearcher s = new IndexSearcher(directoryReader); + searcher = new Engine.Searcher("test", s); + fieldStatsProvider = new FieldStatsProvider(searcher, service); + } + + @After + public void teardown() throws IOException { + searcher.close(); + directoryReader.close(); + dir.close(); + analysisRegistry.close(); + } + + public void testiIsFieldWithinQueryLong() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 10L, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 10L, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", null, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 10L, 100L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 50L, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 30L, 80L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 80L, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 60L, 80L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 10L, 100L, true, false, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 50L, 200L, false, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 100L, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 1L, 50L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 150L, 200L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 1L, 8L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", null, 8L, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 150L, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 100L, 200L, false, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 1L, 50L, true, false, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryFloat() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 10.8f, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 10.8f, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", null, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 10.8f, 100.3f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 50.2f, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 30.5f, 80.1f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 80.1f, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 10.8f, 100.3f, true, false, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 50.2f, 200.5f, false, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 100.3f, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 1.9f, 50.2f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 60.9f, 80.1f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 150.4f, 200.5f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 1.9f, 8.1f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", null, 8.1f, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 150.4f, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 100.3f, 200.5f, false, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("float_field", 1.9f, 50.2f, true, false, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryDouble() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 10.8, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 10.8, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", null, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 10.8, 100.3, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 50.2, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 30.5, 80.1, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 80.1, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 60.9, 80.1, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 10.8, 100.3, true, false, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 50.2, 200.5, false, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 100.3, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 1.9, 50.2, true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 150.4, 200.5, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 1.9, 8.1, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", null, 8.1, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("double_field", 150.4, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 100.3, 200.5, false, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("long_field", 1.9, 50.2, true, false, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryText() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("banana"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("banana"), null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", null, new BytesRef("zebra"), true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("banana"), new BytesRef("orange"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("cherry"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("banana"), new BytesRef("grape"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("grape"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("lime"), new BytesRef("mango"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("banana"), new BytesRef("orange"), true, false, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("cherry"), new BytesRef("zebra"), false, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("orange"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("apple"), new BytesRef("cherry"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("peach"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("apple"), new BytesRef("banana"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", null, new BytesRef("banana"), true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("peach"), null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("orange"), new BytesRef("zebra"), false, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("text_field", new BytesRef("apple"), new BytesRef("cherry"), true, false, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryKeyword() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("banana"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("banana"), null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", null, new BytesRef("zebra"), true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("banana"), new BytesRef("orange"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("cherry"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("banana"), new BytesRef("grape"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("grape"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("lime"), new BytesRef("mango"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("banana"), new BytesRef("orange"), true, false, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("cherry"), new BytesRef("zebra"), false, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("orange"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("apple"), new BytesRef("cherry"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("peach"), new BytesRef("zebra"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("apple"), new BytesRef("banana"), true, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", null, new BytesRef("banana"), true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("peach"), null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("orange"), new BytesRef("zebra"), false, true, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("keyword_field", new BytesRef("apple"), new BytesRef("cherry"), true, false, + DateTimeZone.UTC, null), equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryDate() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", null, "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2014-06-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-01-01", "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2014-03-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-03-01", "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-03-01", "2014-05-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2014-06-01", true, false, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-01-01", "now", false, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-06-01", "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2014-01-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2015-01-01", "now", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2013-09-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", null, "2013-09-01", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2015-01-01", null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2014-06-01", "now", false, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("date_field", "2013-01-01", "2014-01-01", true, false, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + } + + public void testiIsFieldWithinQueryIp() throws IOException { + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.1", "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.1", null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", null, "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", null, null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.1", "10.10.0.60", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.10", "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.WITHIN)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.1", "10.10.0.40", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.40", "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.30", "10.10.0.40", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.1", "10.10.0.60", true, false, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.10", "10.20.0.1", false, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.60", "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.0.0.1", "10.10.0.10", true, true, DateTimeZone.UTC, null), + equalTo(Relation.INTERSECTS)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.20.0.10", "10.20.0.1", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.0.0.1", "10.0.0.100", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", null, "10.0.0.100", true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.20.0.10", null, true, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.10.0.60", "10.20.0.1", false, true, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + assertThat(fieldStatsProvider.isFieldWithinQuery("ip_field", "10.0.0.1", "10.10.0.10", true, false, DateTimeZone.UTC, null), + equalTo(Relation.DISJOINT)); + } + + private void putMapping(MapperService service) throws IOException { + XContentBuilder mappingbuilder = JsonXContent.contentBuilder(); + mappingbuilder.startObject(); + mappingbuilder.startObject("type"); + mappingbuilder.startObject("properties"); + mappingbuilder.startObject("long_field"); + mappingbuilder.field("type", "long"); + mappingbuilder.endObject(); + mappingbuilder.startObject("float_field"); + mappingbuilder.field("type", "float"); + mappingbuilder.endObject(); + mappingbuilder.startObject("double_field"); + mappingbuilder.field("type", "double"); + mappingbuilder.endObject(); + mappingbuilder.startObject("text_field"); + mappingbuilder.field("type", "text"); + mappingbuilder.endObject(); + mappingbuilder.startObject("keyword_field"); + mappingbuilder.field("type", "keyword"); + mappingbuilder.endObject(); + mappingbuilder.startObject("date_field"); + mappingbuilder.field("type", "date"); + mappingbuilder.endObject(); + mappingbuilder.startObject("ip_field"); + mappingbuilder.field("type", "ip"); + mappingbuilder.endObject(); + mappingbuilder.endObject(); + mappingbuilder.endObject(); + mappingbuilder.endObject(); + service.merge("type", new CompressedXContent(mappingbuilder.bytes()), MergeReason.MAPPING_UPDATE, true); + } + + private void indexDocument(MapperService service, IndexWriter writer, String id, long longValue, float floatValue, double doubleValue, + String stringValue, DateTime dateValue, String ipValue) throws IOException { + XContentBuilder docBuilder = JsonXContent.contentBuilder(); + docBuilder.startObject(); + docBuilder.field("long_field", longValue); + docBuilder.field("float_field", floatValue); + docBuilder.field("double_field", doubleValue); + docBuilder.field("text_field", stringValue); + docBuilder.field("keyword_field", stringValue); + docBuilder.field("date_field", dateValue); + docBuilder.field("ip_field", ipValue); + docBuilder.endObject(); + DocumentMapper documentMapper = service.documentMapper("type"); + ParsedDocument doc = documentMapper.parse("index", "type", id, docBuilder.bytes()); + writer.addDocument(doc.rootDoc()); + } +} diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java index 4d2ffcdd11e..8f63daaa8f4 100644 --- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java @@ -23,7 +23,6 @@ import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.io.JsonStringEncoder; -import org.apache.lucene.index.memory.MemoryIndex; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; @@ -341,6 +340,7 @@ public abstract class AbstractQueryTestCase> @After public void afterTest() { + queryShardContext.setFieldStatsProvider(null); clientInvocationHandler.delegate = null; SearchContext.removeCurrent(); } diff --git a/core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java index 9f99b85a294..30e32c92da2 100644 --- a/core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java @@ -22,11 +22,16 @@ package org.elasticsearch.index.query; import org.apache.lucene.search.LegacyNumericRangeQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermRangeQuery; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.common.ParseFieldMatcher; +import org.elasticsearch.common.joda.DateMathParser; import org.elasticsearch.common.lucene.BytesRefs; +import org.elasticsearch.index.fieldstats.FieldStatsProvider; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import org.joda.time.chrono.ISOChronology; import java.io.IOException; import java.util.HashMap; @@ -38,6 +43,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.sameInstance; public class RangeQueryBuilderTests extends AbstractQueryTestCase { @@ -392,4 +398,399 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase> FieldStats get(String field) throws IOException { + assertThat(field, equalTo(fieldName)); + return (FieldStats) new FieldStats.Long(randomLong(), randomLong(), randomLong(), randomLong(), shardMinValue, + shardMaxValue); + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); + RangeQueryBuilder rewrittenRange = (RangeQueryBuilder) rewritten; + assertThat(rewrittenRange.fieldName(), equalTo(fieldName)); + assertThat(rewrittenRange.from(), equalTo(shardMinValue)); + assertThat(rewrittenRange.to(), equalTo(shardMaxValue)); + assertThat(rewrittenRange.includeLower(), equalTo(true)); + assertThat(rewrittenRange.includeUpper(), equalTo(true)); + } + + public void testRewriteLongToMatchNone() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + int queryFromValue = randomIntBetween(-1000000, 1000000); + int queryToValue = randomIntBetween(queryFromValue, 2000000); + query.from((long) queryFromValue); + query.to((long) queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.DISJOINT; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + } + + public void testRewriteLongToSame() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + int queryFromValue = randomIntBetween(-1000000, 1000000); + int queryToValue = randomIntBetween(queryFromValue, 2000000); + query.from((long) queryFromValue); + query.to((long) queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.INTERSECTS; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, sameInstance(query)); + } + + public void testRewriteDoubleToMatchAll() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + double queryFromValue = randomDoubleBetween(-1000000.0, 1000000.0, true); + double queryToValue = randomDoubleBetween(queryFromValue, 2000000, true); + double shardMinValue = randomDoubleBetween(queryFromValue, queryToValue, true); + double shardMaxValue = randomDoubleBetween(shardMinValue, queryToValue, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.WITHIN; + } + + @SuppressWarnings("unchecked") + @Override + public > FieldStats get(String field) throws IOException { + assertThat(field, equalTo(fieldName)); + return (FieldStats) new FieldStats.Double(randomLong(), randomLong(), randomLong(), randomLong(), shardMinValue, + shardMaxValue); + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); + RangeQueryBuilder rewrittenRange = (RangeQueryBuilder) rewritten; + assertThat(rewrittenRange.fieldName(), equalTo(fieldName)); + assertThat(rewrittenRange.from(), equalTo(shardMinValue)); + assertThat(rewrittenRange.to(), equalTo(shardMaxValue)); + assertThat(rewrittenRange.includeLower(), equalTo(true)); + assertThat(rewrittenRange.includeUpper(), equalTo(true)); + } + + public void testRewriteDoubleToMatchNone() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + double queryFromValue = randomDoubleBetween(-1000000, 1000000, true); + double queryToValue = randomDoubleBetween(queryFromValue, 2000000, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.DISJOINT; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + } + + public void testRewriteDoubleToSame() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + double queryFromValue = randomDoubleBetween(-1000000, 1000000, true); + double queryToValue = randomDoubleBetween(queryFromValue, 2000000, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.INTERSECTS; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, sameInstance(query)); + } + + public void testRewriteFloatToMatchAll() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + float queryFromValue = (float) randomDoubleBetween(-1000000.0, 1000000.0, true); + float queryToValue = (float) randomDoubleBetween(queryFromValue, 2000000, true); + float shardMinValue = (float) randomDoubleBetween(queryFromValue, queryToValue, true); + float shardMaxValue = (float) randomDoubleBetween(shardMinValue, queryToValue, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.WITHIN; + } + + @SuppressWarnings("unchecked") + @Override + public > FieldStats get(String field) throws IOException { + assertThat(field, equalTo(fieldName)); + return (FieldStats) new FieldStats.Float(randomLong(), randomLong(), randomLong(), randomLong(), shardMinValue, + shardMaxValue); + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); + RangeQueryBuilder rewrittenRange = (RangeQueryBuilder) rewritten; + assertThat(rewrittenRange.fieldName(), equalTo(fieldName)); + assertThat(rewrittenRange.from(), equalTo(shardMinValue)); + assertThat(rewrittenRange.to(), equalTo(shardMaxValue)); + assertThat(rewrittenRange.includeLower(), equalTo(true)); + assertThat(rewrittenRange.includeUpper(), equalTo(true)); + } + + public void testRewriteFloatToMatchNone() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + float queryFromValue = (float) randomDoubleBetween(-1000000, 1000000, true); + float queryToValue = (float) randomDoubleBetween(queryFromValue, 2000000, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.DISJOINT; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + } + + public void testRewriteFloatToSame() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + float queryFromValue = (float) randomDoubleBetween(-1000000, 1000000, true); + float queryToValue = (float) randomDoubleBetween(queryFromValue, 2000000, true); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.INTERSECTS; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, sameInstance(query)); + } + + public void testRewriteTextToMatchAll() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + String queryFromValue = "damson"; + String queryToValue = "plum"; + String shardMinValue = "grape"; + String shardMaxValue = "orange"; + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.WITHIN; + } + + @SuppressWarnings("unchecked") + @Override + public > FieldStats get(String field) throws IOException { + assertThat(field, equalTo(fieldName)); + return (FieldStats) new FieldStats.Text(randomLong(), randomLong(), randomLong(), randomLong(), + new BytesRef(shardMinValue), new BytesRef(shardMaxValue)); + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); + RangeQueryBuilder rewrittenRange = (RangeQueryBuilder) rewritten; + assertThat(rewrittenRange.fieldName(), equalTo(fieldName)); + assertThat(rewrittenRange.from(), equalTo(shardMinValue)); + assertThat(rewrittenRange.to(), equalTo(shardMaxValue)); + assertThat(rewrittenRange.includeLower(), equalTo(true)); + assertThat(rewrittenRange.includeUpper(), equalTo(true)); + } + + public void testRewriteTextToMatchNone() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + String queryFromValue = "damson"; + String queryToValue = "plum"; + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.DISJOINT; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + } + + public void testRewriteTextToSame() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + String queryFromValue = "damson"; + String queryToValue = "plum"; + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.INTERSECTS; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, sameInstance(query)); + } + + public void testRewriteDateToMatchAll() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + DateTime queryFromValue = new DateTime(2015, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + DateTime queryToValue = new DateTime(2016, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + DateTime shardMinValue = new DateTime(2015, 3, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + DateTime shardMaxValue = new DateTime(2015, 9, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.WITHIN; + } + + @SuppressWarnings("unchecked") + @Override + public > FieldStats get(String field) throws IOException { + assertThat(field, equalTo(fieldName)); + return (FieldStats) new FieldStats.Date(randomLong(), randomLong(), randomLong(), randomLong(), + shardMinValue.getMillis(), shardMaxValue.getMillis(), null); + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); + RangeQueryBuilder rewrittenRange = (RangeQueryBuilder) rewritten; + assertThat(rewrittenRange.fieldName(), equalTo(fieldName)); + assertThat(rewrittenRange.from(), equalTo(shardMinValue.getMillis())); + assertThat(rewrittenRange.to(), equalTo(shardMaxValue.getMillis())); + assertThat(rewrittenRange.includeLower(), equalTo(true)); + assertThat(rewrittenRange.includeUpper(), equalTo(true)); + } + + public void testRewriteDateToMatchNone() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + DateTime queryFromValue = new DateTime(2015, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + DateTime queryToValue = new DateTime(2016, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.DISJOINT; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + } + + public void testRewriteDateToSame() throws IOException { + String fieldName = randomAsciiOfLengthBetween(1, 20); + RangeQueryBuilder query = new RangeQueryBuilder(fieldName); + DateTime queryFromValue = new DateTime(2015, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + DateTime queryToValue = new DateTime(2016, 1, 1, 0, 0, 0, ISOChronology.getInstanceUTC()); + query.from(queryFromValue); + query.to(queryToValue); + QueryShardContext queryShardContext = queryShardContext(); + FieldStatsProvider fieldStatsProvider = new FieldStatsProvider(null, null) { + + @Override + public Relation isFieldWithinQuery(String fieldName, Object from, Object to, boolean includeLower, boolean includeUpper, + DateTimeZone timeZone, DateMathParser dateMathParser) throws IOException { + return Relation.INTERSECTS; + } + }; + queryShardContext.setFieldStatsProvider(fieldStatsProvider); + QueryBuilder rewritten = query.rewrite(queryShardContext); + assertThat(rewritten, sameInstance(query)); + } } diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java index 1b2ad9c0a1e..94c41e5c84e 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java @@ -21,18 +21,19 @@ package org.elasticsearch.indices; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.indices.IndicesRequestCache; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.test.ESIntegTestCase; import org.joda.time.DateTimeZone; - import java.util.List; import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; public class IndicesRequestCacheIT extends ESIntegTestCase { @@ -80,4 +81,156 @@ public class IndicesRequestCacheIT extends ESIntegTestCase { } } + public void testQueryRewrite() throws Exception { + assertAcked(client().admin().indices().prepareCreate("index").addMapping("type", "s", "type=text") + .setSettings(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true, + IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5, + IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .get()); + indexRandom(true, client().prepareIndex("index", "type", "1").setRouting("1").setSource("s", "a"), + client().prepareIndex("index", "type", "2").setRouting("1").setSource("s", "b"), + client().prepareIndex("index", "type", "3").setRouting("1").setSource("s", "c"), + client().prepareIndex("index", "type", "4").setRouting("2").setSource("s", "d"), + client().prepareIndex("index", "type", "5").setRouting("2").setSource("s", "e"), + client().prepareIndex("index", "type", "6").setRouting("2").setSource("s", "f"), + client().prepareIndex("index", "type", "7").setRouting("3").setSource("s", "g"), + client().prepareIndex("index", "type", "8").setRouting("3").setSource("s", "h"), + client().prepareIndex("index", "type", "9").setRouting("3").setSource("s", "i")); + ensureSearchable("index"); + + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(0L)); + + final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("g")).get(); + assertSearchResponse(r1); + assertThat(r1.getHits().getTotalHits(), equalTo(7L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(5L)); + + final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("b").lte("h")).get(); + assertSearchResponse(r2); + assertThat(r2.getHits().getTotalHits(), equalTo(7L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(3L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(7L)); + + final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("c").lte("i")).get(); + assertSearchResponse(r3); + assertThat(r3.getHits().getTotalHits(), equalTo(7L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(6L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(9L)); + } + + public void testQueryRewriteMissingValues() throws Exception { + assertAcked(client().admin().indices().prepareCreate("index").addMapping("type", "s", "type=text") + .setSettings(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true, IndexMetaData.SETTING_NUMBER_OF_SHARDS, + 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .get()); + indexRandom(true, client().prepareIndex("index", "type", "1").setSource("s", "a"), + client().prepareIndex("index", "type", "2").setSource("s", "b"), + client().prepareIndex("index", "type", "3").setSource("s", "c"), + client().prepareIndex("index", "type", "4").setSource("s", "d"), + client().prepareIndex("index", "type", "5").setSource("s", "e"), + client().prepareIndex("index", "type", "6").setSource("s", "f"), + client().prepareIndex("index", "type", "7").setSource("other", "value"), + client().prepareIndex("index", "type", "8").setSource("s", "h"), + client().prepareIndex("index", "type", "9").setSource("s", "i")); + ensureSearchable("index"); + + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(0L)); + + final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("j")).get(); + assertSearchResponse(r1); + assertThat(r1.getHits().getTotalHits(), equalTo(8L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + + final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("j")).get(); + assertSearchResponse(r2); + assertThat(r2.getHits().getTotalHits(), equalTo(8L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(1L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + + final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("s").gte("a").lte("j")).get(); + assertSearchResponse(r3); + assertThat(r3.getHits().getTotalHits(), equalTo(8L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(2L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + } + + public void testQueryRewriteDates() throws Exception { + assertAcked(client().admin().indices().prepareCreate("index").addMapping("type", "d", "type=date") + .setSettings(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true, + IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, + IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .get()); + indexRandom(true, client().prepareIndex("index", "type", "1").setSource("d", "2014-01-01T00:00:00"), + client().prepareIndex("index", "type", "2").setSource("d", "2014-02-01T00:00:00"), + client().prepareIndex("index", "type", "3").setSource("d", "2014-03-01T00:00:00"), + client().prepareIndex("index", "type", "4").setSource("d", "2014-04-01T00:00:00"), + client().prepareIndex("index", "type", "5").setSource("d", "2014-05-01T00:00:00"), + client().prepareIndex("index", "type", "6").setSource("d", "2014-06-01T00:00:00"), + client().prepareIndex("index", "type", "7").setSource("d", "2014-07-01T00:00:00"), + client().prepareIndex("index", "type", "8").setSource("d", "2014-08-01T00:00:00"), + client().prepareIndex("index", "type", "9").setSource("d", "2014-09-01T00:00:00")); + ensureSearchable("index"); + + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(0L)); + + final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("d").gte("2013-01-01T00:00:00").lte("now")) + .get(); + assertSearchResponse(r1); + assertThat(r1.getHits().getTotalHits(), equalTo(9L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(0L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + + final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("d").gte("2013-01-01T00:00:00").lte("now")) + .get(); + assertSearchResponse(r2); + assertThat(r2.getHits().getTotalHits(), equalTo(9L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(1L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + + final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) + .setQuery(QueryBuilders.rangeQuery("d").gte("2013-01-01T00:00:00").lte("now")) + .get(); + assertSearchResponse(r3); + assertThat(r3.getHits().getTotalHits(), equalTo(9L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), + equalTo(2L)); + assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getMissCount(), + equalTo(1L)); + } + } diff --git a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java index 68b496cd566..891911b6202 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java @@ -1914,6 +1914,7 @@ public class SearchQueryIT extends ESIntegTestCase { assertHitCount(client().prepareSearch("test").setSize(0).setQuery(rangeQuery("field").lte(-999999999999L)).get(), 3); } + @AwaitsFix(bugUrl = "NOCOMMIT") public void testRangeQueryWithTimeZone() throws Exception { assertAcked(prepareCreate("test") .addMapping("type1", "date", "type=date", "num", "type=integer")); diff --git a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java index 6c10a1c8aef..136c1fba2e0 100644 --- a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.search.internal.DefaultSearchContext; import org.elasticsearch.test.ESIntegTestCase; import java.util.ArrayList; @@ -270,7 +269,7 @@ public class SimpleSearchIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test") .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(max)) .setTerminateAfter(i).execute().actionGet(); - assertHitCount(searchResponse, (long)i); + assertHitCount(searchResponse, i); assertTrue(searchResponse.isTerminatedEarly()); }