diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index 7eb8d180547c1..0c06d09863353 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -130,7 +130,7 @@ public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext c } } - static final class SeqNoFieldType extends SimpleMappedFieldType { + public static final class SeqNoFieldType extends SimpleMappedFieldType { SeqNoFieldType() { } diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index 7ec6bf3e68313..e3083ba1516ca 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -55,6 +55,7 @@ import org.elasticsearch.index.IndexSortConfig; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; +import org.elasticsearch.index.mapper.SeqNoFieldMapper.SeqNoFieldType; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.SearchPhase; import org.elasticsearch.search.SearchService; @@ -226,22 +227,27 @@ static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExe CheckedConsumer, IOException> leafSorter = l -> {}; // try to rewrite numeric or date sort to the optimized distanceFeatureQuery - if ((searchContext.sort() != null) && SYS_PROP_REWRITE_SORT) { - Query rewrittenQuery = tryRewriteLongSort(searchContext, searcher.getIndexReader(), query, hasFilterCollector); - if (rewrittenQuery != null) { - query = rewrittenQuery; - // modify sorts: add sort on _score as 1st sort, and move the sort on the original field as the 2nd sort - SortField[] oldSortFields = searchContext.sort().sort.getSort(); - DocValueFormat[] oldFormats = searchContext.sort().formats; - SortField[] newSortFields = new SortField[oldSortFields.length + 1]; - DocValueFormat[] newFormats = new DocValueFormat[oldSortFields.length + 1]; - newSortFields[0] = SortField.FIELD_SCORE; - newFormats[0] = DocValueFormat.RAW; - System.arraycopy(oldSortFields, 0, newSortFields, 1, oldSortFields.length); - System.arraycopy(oldFormats, 0, newFormats, 1, oldFormats.length); - sortAndFormatsForRewrittenNumericSort = searchContext.sort(); // stash SortAndFormats to restore it later - searchContext.sort(new SortAndFormats(new Sort(newSortFields), newFormats)); - leafSorter = createLeafSorter(oldSortFields[0]); + if (canOptimizeSort(searchContext, hasFilterCollector) && SYS_PROP_REWRITE_SORT) { + Query matchAllQuery = tryRewriteMatchAllSort(searchContext); + if (matchAllQuery != null) { + query = matchAllQuery; + } else { + Query distanceQuery = tryRewriteLongSort(searchContext); + if (distanceQuery != null) { + query = distanceQuery; + // modify sorts: add sort on _score as 1st sort, and move the sort on the original field as the 2nd sort + SortField[] oldSortFields = searchContext.sort().sort.getSort(); + DocValueFormat[] oldFormats = searchContext.sort().formats; + SortField[] newSortFields = new SortField[oldSortFields.length + 1]; + DocValueFormat[] newFormats = new DocValueFormat[oldSortFields.length + 1]; + newSortFields[0] = SortField.FIELD_SCORE; + newFormats[0] = DocValueFormat.RAW; + System.arraycopy(oldSortFields, 0, newSortFields, 1, oldSortFields.length); + System.arraycopy(oldFormats, 0, newFormats, 1, oldFormats.length); + sortAndFormatsForRewrittenNumericSort = searchContext.sort(); // stash SortAndFormats to restore it later + searchContext.sort(new SortAndFormats(new Sort(newSortFields), newFormats)); + leafSorter = createLeafSorter(oldSortFields[0]); + } } } @@ -405,60 +411,132 @@ private static boolean searchWithCollectorManager(SearchContext searchContext, C return false; // no rescoring when sorting by field } - private static Query tryRewriteLongSort(SearchContext searchContext, IndexReader reader, - Query query, boolean hasFilterCollector) throws IOException { - if (searchContext.searchAfter() != null) return null; //TODO: handle sort optimization with search after - if (searchContext.scrollContext() != null) return null; - if (searchContext.collapse() != null) return null; - if (searchContext.trackScores()) return null; - if (searchContext.aggregations() != null) return null; - Sort sort = searchContext.sort().sort; - SortField sortField = sort.getSort()[0]; - if (SortField.Type.LONG.equals(IndexSortConfig.getSortFieldType(sortField)) == false) return null; - - // check if this is a field of type Long or Date, that is indexed and has doc values - String fieldName = sortField.getField(); - if (fieldName == null) return null; // happens when _score or _doc is the 1st sort field - if (searchContext.mapperService() == null) return null; // mapperService can be null in tests - final MappedFieldType fieldType = searchContext.mapperService().fullName(fieldName); - if (fieldType == null) return null; // for unmapped fields, default behaviour depending on "unmapped_type" flag - if ((fieldType.typeName().equals("long") == false) && (fieldType instanceof DateFieldType == false)) return null; - if (fieldType.indexOptions() == IndexOptions.NONE) return null; //TODO: change to pointDataDimensionCount() when implemented - if (fieldType.hasDocValues() == false) return null; + /** + * Returns true if the query can be optimized using the primary numeric field sort, false + * otherwise. + */ + private static boolean canOptimizeSort(SearchContext context, boolean hasFilterCollector) throws IOException { + if (context.mapperService() == null + || context.sort() == null + || context.collapse() != null + || context.aggregations() != null) { + return false; + } + + final Sort sort = context.sort().sort; + final IndexReader reader = context.searcher().getIndexReader(); + final int numDocs = reader.numDocs(); + final SortField sortField = sort.getSort()[0]; + if (sortField.getField() == null) { + return false; + } + final MappedFieldType fieldType = context.mapperService().fullName(sortField.getField()); + final String fieldName = fieldType.name(); + + if (canEarlyTerminate(reader, context.sort())) { + // disable this optimization if index sorting matches the query sort since it's already optimized + // by index searcher. + return false; + } + + if (SortField.Type.LONG.equals(IndexSortConfig.getSortFieldType(sortField)) == false + || fieldType == null + || fieldType.indexOptions() == IndexOptions.NONE) { + // we only handle indexed long field in this optimization + return false; + } + if ((fieldType.typeName().equals("long") == false) + && (fieldType instanceof DateFieldType == false) + && fieldType instanceof SeqNoFieldType == false) { + return false; + } // check that all sorts are actual document fields or _doc for (int i = 1; i < sort.getSort().length; i++) { SortField sField = sort.getSort()[i]; String sFieldName = sField.getField(); if (sFieldName == null) { - if (SortField.FIELD_DOC.equals(sField) == false) return null; + if (SortField.FIELD_DOC.equals(sField) == false) { + return false; + } } else { //TODO: find out how to cover _script sort that don't use _score - if (searchContext.mapperService().fullName(sFieldName) == null) return null; // could be _script sort that uses _score + if (context.mapperService().fullName(sFieldName) == null) { + return false; // could be _script sort that uses _score + } } } - // check that setting of missing values allows optimization - if (sortField.getMissingValue() == null) return null; - Long missingValue = (Long) sortField.getMissingValue(); - boolean missingValuesAccordingToSort = (sortField.getReverse() && (missingValue == Long.MIN_VALUE)) || - ((sortField.getReverse() == false) && (missingValue == Long.MAX_VALUE)); - if (missingValuesAccordingToSort == false) return null; - - int docCount = PointValues.getDocCount(reader, fieldName); - // is not worth to run optimization on small index - if (docCount <= 512) return null; + // check if the optimization makes sense with the track_total_hits setting + if (context.trackTotalHitsUpTo() == Integer.MAX_VALUE) { + // with filter, we can't pre-calculate hitsCount, we need to explicitly calculate them => optimization does't make sense + if (hasFilterCollector) { + return false; + } + // if we can't pre-calculate hitsCount based on the query type, optimization doesn't make sense + if (shortcutTotalHitCount(reader, context.query()) == -1) { + return false; + } + } // check for multiple values - if (PointValues.size(reader, fieldName) != docCount) return null; //TODO: handle multiple values + if (PointValues.size(reader, fieldName) != numDocs) { + return false; // TODO: handle multiple values + } - // check if the optimization makes sense with the track_total_hits setting - if (searchContext.trackTotalHitsUpTo() == Integer.MAX_VALUE) { - // with filter, we can't pre-calculate hitsCount, we need to explicitly calculate them => optimization does't make sense - if (hasFilterCollector) return null; - // if we can't pre-calculate hitsCount based on the query type, optimization does't make sense - if (shortcutTotalHitCount(reader, query) == -1) return null; + return true; + } + + /** + * Return a {@link SortedLongQuery} if the request is a {@link MatchAllDocsQuery} + * sorted by a numeric long field and null otherwise. + */ + private static Query tryRewriteMatchAllSort(SearchContext context) { + final Query query = context.query(); + final SortField sortField = context.sort().sort.getSort()[0]; + if ((query != null && query instanceof MatchAllDocsQuery == false) + || context.sort().sort.getSort().length > 1 + || sortField.getReverse() + || (context.searchAfter() != null && context.searchAfter().fields.length > 1)) { + return null; + } + + FieldDoc lastDoc = null; + if (context.searchAfter() != null) { + lastDoc = context.searchAfter(); + } else if (context.scrollContext() != null) { + lastDoc = (FieldDoc) context.scrollContext().lastEmittedDoc; + } + long minValue = Long.MIN_VALUE; + int minDoc = Integer.MAX_VALUE; + if (lastDoc != null) { + minValue = (long) lastDoc.fields[0]; + minDoc = lastDoc.doc; + } + return new SortedLongQuery(sortField.getField(), context.size(), minValue, minDoc); + } + + /** + * Rewrite the query into a {@link LongPoint#newDistanceFeatureQuery(String, float, long, long)} + * if the request is sorted by a numeric long field and null otherwise. + */ + private static Query tryRewriteLongSort(SearchContext context) throws IOException { + final IndexReader reader = context.searcher().getIndexReader(); + final SortField sortField = context.sort().sort.getSort()[0]; + final MappedFieldType fieldType = context.mapperService().fullName(sortField.getField()); + final String fieldName = sortField.getField(); + if (fieldType.hasDocValues() == false + || context.searchAfter() != null // TODO: handle optimization with search after + || context.scrollContext() != null // TODO: handle optimization with scroll + || context.trackScores()) { + return null; + } + + int docCount = PointValues.getDocCount(reader, fieldType.name()); + // is not worth to run optimization on small index + if (docCount <= 512) { + return null; } byte[] minValueBytes = PointValues.getMinPackedValue(reader, fieldName); @@ -471,7 +549,9 @@ private static Query tryRewriteLongSort(SearchContext searchContext, IndexReader if (minValue == maxValue) { rewrittenQuery = new DocValuesFieldExistsQuery(fieldName); } else { - if (indexFieldHasDuplicateData(reader, fieldName)) return null; + if (indexFieldHasDuplicateData(reader, fieldName)) { + return null; + } long origin = (sortField.getReverse()) ? maxValue : minValue; long pivotDistance = (maxValue - minValue) >>> 1; // division by 2 on the unsigned representation to avoid overflow if (pivotDistance == 0) { // 0 if maxValue = (minValue + 1) @@ -480,7 +560,7 @@ private static Query tryRewriteLongSort(SearchContext searchContext, IndexReader rewrittenQuery = LongPoint.newDistanceFeatureQuery(sortField.getField(), 1, origin, pivotDistance); } rewrittenQuery = new BooleanQuery.Builder() - .add(query, BooleanClause.Occur.FILTER) // filter for original query + .add(context.query(), BooleanClause.Occur.FILTER) // filter for original query .add(rewrittenQuery, BooleanClause.Occur.SHOULD) //should for rewrittenQuery .build(); return rewrittenQuery; diff --git a/server/src/main/java/org/elasticsearch/search/query/SortedLongQuery.java b/server/src/main/java/org/elasticsearch/search/query/SortedLongQuery.java new file mode 100644 index 0000000000000..a10993ac55221 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/query/SortedLongQuery.java @@ -0,0 +1,156 @@ +/* + * 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.search.query; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.ConstantScoreScorer; +import org.apache.lucene.search.ConstantScoreWeight; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.FutureArrays; +import org.apache.lucene.util.SparseFixedBitSet; + +import java.io.IOException; +import java.util.Objects; + +/** + * A query that matches only the first topN documents + * sorted by a {@link LongPoint} field and discards the other ones. + * This query can be used to retrieve the topN documents sorted by + * a field efficiently when there is no other clause in the query. + */ +class SortedLongQuery extends Query { + private final String field; + private final long from; + private final int fromMinDoc; + private final int size; + + SortedLongQuery(String field, int size) { + this(field, size, Long.MIN_VALUE, Integer.MAX_VALUE); + } + + SortedLongQuery(String field, int size, long from, int fromMinDoc) { + this.field = Objects.requireNonNull(field); + this.from = from; + this.fromMinDoc = fromMinDoc; + this.size = size; + } + + @Override + public int hashCode() { + return Objects.hash(classHash(), from, fromMinDoc, size); + } + + + @Override + public Query rewrite(IndexReader reader) throws IOException { + return this; + } + + @Override + public boolean equals(Object o) { + if (sameClassAs(o) == false) { + return false; + } + SortedLongQuery that = (SortedLongQuery) o; + return from == that.from && + fromMinDoc == that.fromMinDoc && + size == that.size && + field.equals(that.field); + } + + @Override + public String toString(String field) { + return "SortedLongQuery(" + + "field='" + field + '\'' + + ", from=" + from + + ", fromMinDoc=" + fromMinDoc + + ", size=" + size + + ')'; + } + + @Override + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return new ConstantScoreWeight(this, boost) { + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + final PointValues pointValues = context.reader().getPointValues(field); + if (pointValues == null) { + return null; + } + final Bits liveDocs = context.reader().getLiveDocs(); + byte[] minBytes = new byte[Long.SIZE]; + LongPoint.encodeDimension(from, minBytes, 0); + final SparseFixedBitSet bitSet = new SparseFixedBitSet(context.reader().numDocs()); + final int[] count = new int[1]; + try { + pointValues.intersect(new PointValues.IntersectVisitor() { + @Override + public void visit(int docID) { + throw new UnsupportedOperationException(); + } + + @Override + public void visit(int docID, byte[] packedValue) { + int cmp = FutureArrays.compareUnsigned(packedValue, 0, Long.BYTES, minBytes, 0, Long.BYTES); + if (cmp == 0 && fromMinDoc >= docID + context.docBase) { + return; + } else if (cmp < 0) { + return; + } + if (liveDocs == null || liveDocs.get(docID)) { + bitSet.set(docID); + if (++count[0] == size) { + throw new CollectionTerminatedException(); + } + } + } + + @Override + public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { + if (FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minBytes, 0, Long.BYTES) < 0) { + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + return PointValues.Relation.CELL_CROSSES_QUERY; + } + }); + } catch (CollectionTerminatedException exc) {} + if (count[0] == 0) { + return null; + } + return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(bitSet, count[0])); + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return false; + } + }; + } +} diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index 2c9fc90c22cab..2f2fd8ca649e2 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -639,6 +639,8 @@ public void testNumericLongOrDateSortOptimization() throws Exception { final String fieldNameDate = "date-field"; MappedFieldType fieldTypeLong = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); MappedFieldType fieldTypeDate = new DateFieldMapper.Builder(fieldNameDate).fieldType(); + fieldTypeLong.setName(fieldNameLong); + fieldTypeDate.setName(fieldNameDate); MapperService mapperService = mock(MapperService.class); when(mapperService.fullName(fieldNameLong)).thenReturn(fieldTypeLong); when(mapperService.fullName(fieldNameDate)).thenReturn(fieldTypeDate); @@ -660,45 +662,56 @@ public void testNumericLongOrDateSortOptimization() throws Exception { writer.close(); final IndexReader reader = DirectoryReader.open(dir); - TestSearchContext searchContext = - spy(new TestSearchContext(null, indexShard, newOptimizedContextSearcher(reader, 0))); - when(searchContext.mapperService()).thenReturn(mapperService); - - // 1. Test a sort on long field - final SortField sortFieldLong = new SortField(fieldNameLong, SortField.Type.LONG); - sortFieldLong.setMissingValue(Long.MAX_VALUE); - final Sort longSort = new Sort(sortFieldLong); - SortAndFormats sortAndFormats = new SortAndFormats(longSort, new DocValueFormat[]{DocValueFormat.RAW}); - searchContext.sort(sortAndFormats); - searchContext.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); - searchContext.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap())); - searchContext.setSize(10); - QueryPhase.executeInternal(searchContext); - assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); - - // 2. Test a sort on long field + date field - final SortField sortFieldDate = new SortField(fieldNameDate, SortField.Type.LONG); - DocValueFormat dateFormat = fieldTypeDate.docValueFormat(null, null); - final Sort longDateSort = new Sort(sortFieldLong, sortFieldDate); - sortAndFormats = new SortAndFormats(longDateSort, new DocValueFormat[]{DocValueFormat.RAW, dateFormat}); - searchContext.sort(sortAndFormats); - QueryPhase.executeInternal(searchContext); - assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); - - // 3. Test a sort on date field - sortFieldDate.setMissingValue(Long.MAX_VALUE); - final Sort dateSort = new Sort(sortFieldDate); - sortAndFormats = new SortAndFormats(dateSort, new DocValueFormat[]{dateFormat}); - searchContext.sort(sortAndFormats); - QueryPhase.executeInternal(searchContext); - assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); - - // 4. Test a sort on date field + long field - final Sort dateLongSort = new Sort(sortFieldDate, sortFieldLong); - sortAndFormats = new SortAndFormats(dateLongSort, new DocValueFormat[]{dateFormat, DocValueFormat.RAW}); - searchContext.sort(sortAndFormats); - QueryPhase.executeInternal(searchContext); - assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); + for (boolean withCollectorManager : new boolean[] {true, false}) { + TestSearchContext searchContext = + spy(new TestSearchContext(null, indexShard, newOptimizedContextSearcher(reader, withCollectorManager))); + when(searchContext.mapperService()).thenReturn(mapperService); + if (withCollectorManager) { + searchContext.parsedQuery(new ParsedQuery(new DocValuesFieldExistsQuery(fieldNameLong))); + } else { + searchContext.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); + } + + final SortField sortFieldDate = new SortField(fieldNameDate, SortField.Type.LONG); + final SortField sortFieldLong = new SortField(fieldNameLong, SortField.Type.LONG); + final DocValueFormat dateFormat = fieldTypeDate.docValueFormat(null, null); + + // 1. Test a sort on long field + sortFieldLong.setMissingValue(Long.MAX_VALUE); + final Sort longSort = new Sort(sortFieldLong); + SortAndFormats sortAndFormats = new SortAndFormats(longSort, new DocValueFormat[]{DocValueFormat.RAW}); + searchContext.sort(sortAndFormats); + searchContext.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap())); + searchContext.setSize(10); + QueryPhase.executeInternal(searchContext); + assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); + + // 2. Test a sort on long field + date field + if (withCollectorManager) { + final Sort longDateSort = new Sort(sortFieldLong, sortFieldDate); + sortAndFormats = new SortAndFormats(longDateSort, new DocValueFormat[]{DocValueFormat.RAW, dateFormat}); + searchContext.sort(sortAndFormats); + QueryPhase.executeInternal(searchContext); + assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); + } + + // 3. Test a sort on date field + sortFieldDate.setMissingValue(Long.MAX_VALUE); + final Sort dateSort = new Sort(sortFieldDate); + sortAndFormats = new SortAndFormats(dateSort, new DocValueFormat[]{dateFormat}); + searchContext.sort(sortAndFormats); + QueryPhase.executeInternal(searchContext); + assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); + + // 4. Test a sort on date field + long field + if (withCollectorManager) { + final Sort dateLongSort = new Sort(sortFieldDate, sortFieldLong); + sortAndFormats = new SortAndFormats(dateLongSort, new DocValueFormat[]{dateFormat, DocValueFormat.RAW}); + searchContext.sort(sortAndFormats); + QueryPhase.executeInternal(searchContext); + assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); + } + } reader.close(); dir.close(); } @@ -851,31 +864,31 @@ public void search(List leaves, Weight weight, Collector coll } // used to check that numeric long or date sort optimization was run - private static ContextIndexSearcher newOptimizedContextSearcher(IndexReader reader, int queryType) { + private static ContextIndexSearcher newOptimizedContextSearcher(IndexReader reader, boolean withCollectorManager) { return new ContextIndexSearcher(reader, IndexSearcher.getDefaultSimilarity(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy()) { @Override public void search(List leaves, Weight weight, CollectorManager manager, QuerySearchResult result, DocValueFormat[] formats, TotalHits totalHits) throws IOException { + assertTrue(withCollectorManager); final Query query = weight.getQuery(); assertTrue(query instanceof BooleanQuery); List clauses = ((BooleanQuery) query).clauses(); assertTrue(clauses.size() == 2); assertTrue(clauses.get(0).getOccur() == Occur.FILTER); assertTrue(clauses.get(1).getOccur() == Occur.SHOULD); - if (queryType == 0) { - assertTrue (clauses.get(1).getQuery().getClass() == - LongPoint.newDistanceFeatureQuery("random_field", 1, 1, 1).getClass() - ); - } - if (queryType == 1) assertTrue(clauses.get(1).getQuery() instanceof DocValuesFieldExistsQuery); + assertTrue (clauses.get(1).getQuery().getClass() == + LongPoint.newDistanceFeatureQuery("random_field", 1, 1, 1).getClass()); super.search(leaves, weight, manager, result, formats, totalHits); } @Override - public void search(List leaves, Weight weight, Collector collector) { - assert(false); // should not be there, expected to search with CollectorManager + public void search(List leaves, Weight weight, Collector collector) throws IOException { + assertFalse(withCollectorManager); + final Query query = weight.getQuery(); + assertTrue(query instanceof SortedLongQuery); + super.search(leaves, weight, collector); } }; } diff --git a/server/src/test/java/org/elasticsearch/search/query/SortedLongQueryTests.java b/server/src/test/java/org/elasticsearch/search/query/SortedLongQueryTests.java new file mode 100644 index 0000000000000..2c86405f10b4d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/query/SortedLongQueryTests.java @@ -0,0 +1,92 @@ +/* + * 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.search.query; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.QueryUtils; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.store.Directory; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Arrays; + +public class SortedLongQueryTests extends ESTestCase { + + public void testBasics() { + SortedLongQuery query1 = new SortedLongQuery("field", 1, 1, 1); + SortedLongQuery query2 = new SortedLongQuery("field", 1, 1, 1); + SortedLongQuery query3 = new SortedLongQuery("field", 2, 1, 1); + SortedLongQuery query4 = new SortedLongQuery("field", 1, 2, 1); + SortedLongQuery query5 = new SortedLongQuery("field", 1, 1, 2); + SortedLongQuery query6 = new SortedLongQuery("field2", 1, 1, 1); + QueryUtils.check(query1); + QueryUtils.checkEqual(query1, query2); + QueryUtils.checkUnequal(query1, query3); + QueryUtils.checkUnequal(query1, query4); + QueryUtils.checkUnequal(query1, query5); + QueryUtils.checkUnequal(query1, query6); + } + + public void testRandom() throws IOException { + final int numDocs = randomIntBetween(100, 200); + final Document doc = new Document(); + final Directory dir = newDirectory(); + final IndexWriterConfig config = new IndexWriterConfig(); + final RandomIndexWriter w = new RandomIndexWriter(random(), dir, config); + long[] values = new long[numDocs]; + for (int i = 0; i < numDocs; ++i) { + long rand = randomLong(); + values[i] = rand; + doc.add(new SortedNumericDocValuesField("number", rand)); + doc.add(new LongPoint("number", rand)); + w.addDocument(doc); + doc.clear(); + if (rarely()) { + w.commit(); + } + } + Arrays.sort(values); + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = new IndexSearcher(reader); + + Sort sort = new Sort(new SortedNumericSortField("number", SortField.Type.LONG)); + for (int size = 1; size < numDocs; size++) { + TopFieldDocs topDocs = searcher.search(new SortedLongQuery("number", size, Long.MIN_VALUE, Integer.MAX_VALUE), size, sort); + assert(topDocs.scoreDocs.length == size); + for (int j = 0; j < topDocs.scoreDocs.length; j++) { + assertEquals(values[j], ((FieldDoc) topDocs.scoreDocs[j]).fields[0]); + } + } + w.close(); + reader.close(); + dir.close(); + } +}