Enabling Sort Optimization to make use of Lucene (#1974)
Remove sort optimization. From Lucence 8.10.1, Numeric sort optimization is already done in Lucene Comparators. Lucene also supports sort optimization with search_after queries. Signed-off-by: Rishikesh1159 <rishireddy1159@gmail.com>
This commit is contained in:
parent
3b8db91abe
commit
3f92aecddf
|
@ -34,34 +34,24 @@ package org.opensearch.search.query;
|
||||||
|
|
||||||
import org.apache.logging.log4j.LogManager;
|
import org.apache.logging.log4j.LogManager;
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.apache.lucene.document.LongPoint;
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.PointValues;
|
|
||||||
import org.apache.lucene.queries.MinDocQuery;
|
import org.apache.lucene.queries.MinDocQuery;
|
||||||
import org.apache.lucene.queries.SearchAfterSortedDocQuery;
|
import org.apache.lucene.queries.SearchAfterSortedDocQuery;
|
||||||
import org.apache.lucene.search.BooleanClause;
|
import org.apache.lucene.search.BooleanClause;
|
||||||
import org.apache.lucene.search.BooleanQuery;
|
import org.apache.lucene.search.BooleanQuery;
|
||||||
import org.apache.lucene.search.Collector;
|
import org.apache.lucene.search.Collector;
|
||||||
import org.apache.lucene.search.CollectorManager;
|
|
||||||
import org.apache.lucene.search.ConstantScoreQuery;
|
import org.apache.lucene.search.ConstantScoreQuery;
|
||||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
|
||||||
import org.apache.lucene.search.FieldDoc;
|
import org.apache.lucene.search.FieldDoc;
|
||||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.search.ScoreDoc;
|
import org.apache.lucene.search.ScoreDoc;
|
||||||
import org.apache.lucene.search.ScoreMode;
|
|
||||||
import org.apache.lucene.search.Sort;
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.search.SortField;
|
import org.apache.lucene.search.SortField;
|
||||||
import org.apache.lucene.search.TopDocs;
|
import org.apache.lucene.search.TopDocs;
|
||||||
import org.apache.lucene.search.TopFieldCollector;
|
|
||||||
import org.apache.lucene.search.TopFieldDocs;
|
|
||||||
import org.apache.lucene.search.TotalHits;
|
import org.apache.lucene.search.TotalHits;
|
||||||
import org.apache.lucene.search.Weight;
|
|
||||||
import org.apache.lucene.util.FutureArrays;
|
|
||||||
import org.opensearch.action.search.SearchShardTask;
|
import org.opensearch.action.search.SearchShardTask;
|
||||||
import org.opensearch.common.Booleans;
|
import org.opensearch.common.Booleans;
|
||||||
import org.opensearch.common.CheckedConsumer;
|
|
||||||
import org.opensearch.common.lucene.Lucene;
|
import org.opensearch.common.lucene.Lucene;
|
||||||
import org.opensearch.common.lucene.search.TopDocsAndMaxScore;
|
import org.opensearch.common.lucene.search.TopDocsAndMaxScore;
|
||||||
import org.opensearch.common.util.concurrent.QueueResizingOpenSearchThreadPoolExecutor;
|
import org.opensearch.common.util.concurrent.QueueResizingOpenSearchThreadPoolExecutor;
|
||||||
|
@ -85,12 +75,7 @@ import org.opensearch.tasks.TaskCancelledException;
|
||||||
import org.opensearch.threadpool.ThreadPool;
|
import org.opensearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
import static org.opensearch.search.query.QueryCollectorContext.createEarlyTerminationCollectorContext;
|
import static org.opensearch.search.query.QueryCollectorContext.createEarlyTerminationCollectorContext;
|
||||||
|
@ -98,7 +83,6 @@ import static org.opensearch.search.query.QueryCollectorContext.createFilteredCo
|
||||||
import static org.opensearch.search.query.QueryCollectorContext.createMinScoreCollectorContext;
|
import static org.opensearch.search.query.QueryCollectorContext.createMinScoreCollectorContext;
|
||||||
import static org.opensearch.search.query.QueryCollectorContext.createMultiCollectorContext;
|
import static org.opensearch.search.query.QueryCollectorContext.createMultiCollectorContext;
|
||||||
import static org.opensearch.search.query.TopDocsCollectorContext.createTopDocsCollectorContext;
|
import static org.opensearch.search.query.TopDocsCollectorContext.createTopDocsCollectorContext;
|
||||||
import static org.opensearch.search.query.TopDocsCollectorContext.shortcutTotalHitCount;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Query phase of a search request, used to run the query and get back from each shard information about the matching documents
|
* Query phase of a search request, used to run the query and get back from each shard information about the matching documents
|
||||||
|
@ -183,7 +167,6 @@ public class QueryPhase {
|
||||||
*/
|
*/
|
||||||
static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExecutionException {
|
static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExecutionException {
|
||||||
final ContextIndexSearcher searcher = searchContext.searcher();
|
final ContextIndexSearcher searcher = searchContext.searcher();
|
||||||
SortAndFormats sortAndFormatsForRewrittenNumericSort = null;
|
|
||||||
final IndexReader reader = searcher.getIndexReader();
|
final IndexReader reader = searcher.getIndexReader();
|
||||||
QuerySearchResult queryResult = searchContext.queryResult();
|
QuerySearchResult queryResult = searchContext.queryResult();
|
||||||
queryResult.searchTimedOut(false);
|
queryResult.searchTimedOut(false);
|
||||||
|
@ -252,26 +235,9 @@ public class QueryPhase {
|
||||||
// this collector can filter documents during the collection
|
// this collector can filter documents during the collection
|
||||||
hasFilterCollector = true;
|
hasFilterCollector = true;
|
||||||
}
|
}
|
||||||
|
// optimizing sort on Numerics (long and date)
|
||||||
CheckedConsumer<List<LeafReaderContext>, IOException> leafSorter = l -> {};
|
|
||||||
// try to rewrite numeric or date sort to the optimized distanceFeatureQuery
|
|
||||||
if ((searchContext.sort() != null) && SYS_PROP_REWRITE_SORT) {
|
if ((searchContext.sort() != null) && SYS_PROP_REWRITE_SORT) {
|
||||||
Query rewrittenQuery = tryRewriteLongSort(searchContext, searcher.getIndexReader(), query, hasFilterCollector);
|
enhanceSortOnNumeric(searchContext, searcher.getIndexReader());
|
||||||
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]);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean timeoutSet = scrollContext == null
|
boolean timeoutSet = scrollContext == null
|
||||||
|
@ -303,20 +269,7 @@ public class QueryPhase {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
boolean shouldRescore;
|
boolean shouldRescore = searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, timeoutSet);
|
||||||
// if we are optimizing sort and there are no other collectors
|
|
||||||
if (sortAndFormatsForRewrittenNumericSort != null && collectors.size() == 0 && searchContext.getProfilers() == null) {
|
|
||||||
shouldRescore = searchWithCollectorManager(searchContext, searcher, query, leafSorter, timeoutSet);
|
|
||||||
} else {
|
|
||||||
shouldRescore = searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, timeoutSet);
|
|
||||||
}
|
|
||||||
|
|
||||||
// if we rewrote numeric long or date sort, restore fieldDocs based on the original sort
|
|
||||||
if (sortAndFormatsForRewrittenNumericSort != null) {
|
|
||||||
searchContext.sort(sortAndFormatsForRewrittenNumericSort); // restore SortAndFormats
|
|
||||||
restoreTopFieldDocs(queryResult, sortAndFormatsForRewrittenNumericSort);
|
|
||||||
}
|
|
||||||
|
|
||||||
ExecutorService executor = searchContext.indexShard().getThreadPool().executor(ThreadPool.Names.SEARCH);
|
ExecutorService executor = searchContext.indexShard().getThreadPool().executor(ThreadPool.Names.SEARCH);
|
||||||
if (executor instanceof QueueResizingOpenSearchThreadPoolExecutor) {
|
if (executor instanceof QueueResizingOpenSearchThreadPoolExecutor) {
|
||||||
QueueResizingOpenSearchThreadPoolExecutor rExecutor = (QueueResizingOpenSearchThreadPoolExecutor) executor;
|
QueueResizingOpenSearchThreadPoolExecutor rExecutor = (QueueResizingOpenSearchThreadPoolExecutor) executor;
|
||||||
|
@ -379,182 +332,25 @@ public class QueryPhase {
|
||||||
return topDocsFactory.shouldRescore();
|
return topDocsFactory.shouldRescore();
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
private static void enhanceSortOnNumeric(SearchContext searchContext, IndexReader reader) {
|
||||||
* We use collectorManager during sort optimization, where
|
|
||||||
* we have already checked that there are no other collectors, no filters,
|
|
||||||
* no search after, no scroll, no collapse, no track scores.
|
|
||||||
* Absence of all other collectors and parameters allows us to use TopFieldCollector directly.
|
|
||||||
*/
|
|
||||||
private static boolean searchWithCollectorManager(
|
|
||||||
SearchContext searchContext,
|
|
||||||
ContextIndexSearcher searcher,
|
|
||||||
Query query,
|
|
||||||
CheckedConsumer<List<LeafReaderContext>, IOException> leafSorter,
|
|
||||||
boolean timeoutSet
|
|
||||||
) throws IOException {
|
|
||||||
final IndexReader reader = searchContext.searcher().getIndexReader();
|
|
||||||
final int numHits = Math.min(searchContext.from() + searchContext.size(), Math.max(1, reader.numDocs()));
|
|
||||||
final SortAndFormats sortAndFormats = searchContext.sort();
|
|
||||||
|
|
||||||
int totalHitsThreshold;
|
|
||||||
TotalHits totalHits;
|
|
||||||
if (searchContext.trackTotalHitsUpTo() == SearchContext.TRACK_TOTAL_HITS_DISABLED) {
|
|
||||||
totalHitsThreshold = 1;
|
|
||||||
totalHits = new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
|
|
||||||
} else {
|
|
||||||
int hitCount = shortcutTotalHitCount(reader, query);
|
|
||||||
if (hitCount == -1) {
|
|
||||||
totalHitsThreshold = searchContext.trackTotalHitsUpTo();
|
|
||||||
totalHits = null; // will be computed via the collector
|
|
||||||
} else {
|
|
||||||
totalHitsThreshold = 1;
|
|
||||||
totalHits = new TotalHits(hitCount, TotalHits.Relation.EQUAL_TO); // don't compute hit counts via the collector
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
CollectorManager<TopFieldCollector, TopFieldDocs> sharedManager = TopFieldCollector.createSharedManager(
|
|
||||||
sortAndFormats.sort,
|
|
||||||
numHits,
|
|
||||||
null,
|
|
||||||
totalHitsThreshold
|
|
||||||
);
|
|
||||||
|
|
||||||
List<LeafReaderContext> leaves = new ArrayList<>(searcher.getIndexReader().leaves());
|
|
||||||
leafSorter.accept(leaves);
|
|
||||||
try {
|
|
||||||
Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.TOP_SCORES, 1f);
|
|
||||||
searcher.search(leaves, weight, sharedManager, searchContext.queryResult(), sortAndFormats.formats, totalHits);
|
|
||||||
} catch (TimeExceededException e) {
|
|
||||||
assert timeoutSet : "TimeExceededException thrown even though timeout wasn't set";
|
|
||||||
if (searchContext.request().allowPartialSearchResults() == false) {
|
|
||||||
// Can't rethrow TimeExceededException because not serializable
|
|
||||||
throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Time exceeded");
|
|
||||||
}
|
|
||||||
searchContext.queryResult().searchTimedOut(true);
|
|
||||||
}
|
|
||||||
return false; // no rescoring when sorting by field
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Query tryRewriteLongSort(SearchContext searchContext, IndexReader reader, Query query, boolean hasFilterCollector)
|
|
||||||
throws IOException {
|
|
||||||
if ((searchContext.from() + searchContext.size()) <= 0) return null;
|
|
||||||
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;
|
|
||||||
if (canEarlyTerminate(reader, searchContext.sort())) {
|
if (canEarlyTerminate(reader, searchContext.sort())) {
|
||||||
// disable this optimization if index sorting matches the query sort since it's already optimized by index searcher
|
// disable this optimization if index sorting matches the query sort since it's already optimized by index searcher
|
||||||
return null;
|
return;
|
||||||
}
|
}
|
||||||
Sort sort = searchContext.sort().sort;
|
Sort sort = searchContext.sort().sort;
|
||||||
SortField sortField = sort.getSort()[0];
|
SortField sortField = sort.getSort()[0];
|
||||||
if (SortField.Type.LONG.equals(IndexSortConfig.getSortFieldType(sortField)) == false) return null;
|
if (SortField.Type.LONG.equals(IndexSortConfig.getSortFieldType(sortField)) == false) return;
|
||||||
|
|
||||||
// check if this is a field of type Long or Date, that is indexed and has doc values
|
// check if this is a field of type Long or Date, that is indexed and has doc values
|
||||||
String fieldName = sortField.getField();
|
String fieldName = sortField.getField();
|
||||||
if (fieldName == null) return null; // happens when _score or _doc is the 1st sort field
|
if (fieldName == null) return; // happens when _score or _doc is the 1st sort field
|
||||||
if (searchContext.mapperService() == null) return null; // mapperService can be null in tests
|
if (searchContext.mapperService() == null) return; // mapperService can be null in tests
|
||||||
final MappedFieldType fieldType = searchContext.mapperService().fieldType(fieldName);
|
final MappedFieldType fieldType = searchContext.mapperService().fieldType(fieldName);
|
||||||
if (fieldType == null) return null; // for unmapped fields, default behaviour depending on "unmapped_type" flag
|
if (fieldType == null) return; // for unmapped fields, default behaviour depending on "unmapped_type" flag
|
||||||
if ((fieldType.typeName().equals("long") == false) && (fieldType instanceof DateFieldType == false)) return null;
|
if ((fieldType.typeName().equals("long") == false) && (fieldType instanceof DateFieldType == false)) return;
|
||||||
if (fieldType.isSearchable() == false) return null;
|
if (fieldType.isSearchable() == false) return;
|
||||||
if (fieldType.hasDocValues() == false) return null;
|
if (fieldType.hasDocValues() == false) return;
|
||||||
|
sortField.setCanUsePoints();
|
||||||
// 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;
|
|
||||||
} else {
|
|
||||||
// TODO: find out how to cover _script sort that don't use _score
|
|
||||||
if (searchContext.mapperService().fieldType(sFieldName) == null) return null; // 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 for multiple values
|
|
||||||
if (PointValues.size(reader, fieldName) != docCount) return null; // 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
byte[] minValueBytes = PointValues.getMinPackedValue(reader, fieldName);
|
|
||||||
byte[] maxValueBytes = PointValues.getMaxPackedValue(reader, fieldName);
|
|
||||||
if ((maxValueBytes == null) || (minValueBytes == null)) return null;
|
|
||||||
long minValue = LongPoint.decodeDimension(minValueBytes, 0);
|
|
||||||
long maxValue = LongPoint.decodeDimension(maxValueBytes, 0);
|
|
||||||
|
|
||||||
Query rewrittenQuery;
|
|
||||||
if (minValue == maxValue) {
|
|
||||||
rewrittenQuery = new DocValuesFieldExistsQuery(fieldName);
|
|
||||||
} else {
|
|
||||||
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)
|
|
||||||
pivotDistance = 1;
|
|
||||||
}
|
|
||||||
rewrittenQuery = LongPoint.newDistanceFeatureQuery(sortField.getField(), 1, origin, pivotDistance);
|
|
||||||
}
|
|
||||||
rewrittenQuery = new BooleanQuery.Builder().add(query, BooleanClause.Occur.FILTER) // filter for original query
|
|
||||||
.add(rewrittenQuery, BooleanClause.Occur.SHOULD) // should for rewrittenQuery
|
|
||||||
.build();
|
|
||||||
return rewrittenQuery;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a sorter of {@link LeafReaderContext} that orders leaves depending on the minimum
|
|
||||||
* value and the sort order of the provided <code>sortField</code>.
|
|
||||||
*/
|
|
||||||
static CheckedConsumer<List<LeafReaderContext>, IOException> createLeafSorter(SortField sortField) {
|
|
||||||
return leaves -> {
|
|
||||||
long[] sortValues = new long[leaves.size()];
|
|
||||||
long missingValue = (long) sortField.getMissingValue();
|
|
||||||
for (LeafReaderContext ctx : leaves) {
|
|
||||||
PointValues values = ctx.reader().getPointValues(sortField.getField());
|
|
||||||
if (values == null) {
|
|
||||||
sortValues[ctx.ord] = missingValue;
|
|
||||||
} else {
|
|
||||||
byte[] sortValue = sortField.getReverse() ? values.getMaxPackedValue() : values.getMinPackedValue();
|
|
||||||
sortValues[ctx.ord] = sortValue == null ? missingValue : LongPoint.decodeDimension(sortValue, 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
Comparator<LeafReaderContext> comparator = Comparator.comparingLong(l -> sortValues[l.ord]);
|
|
||||||
if (sortField.getReverse()) {
|
|
||||||
comparator = comparator.reversed();
|
|
||||||
}
|
|
||||||
Collections.sort(leaves, comparator);
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Restore fieldsDocs to remove the first _score
|
|
||||||
*/
|
|
||||||
private static void restoreTopFieldDocs(QuerySearchResult result, SortAndFormats originalSortAndFormats) {
|
|
||||||
TopDocs topDocs = result.topDocs().topDocs;
|
|
||||||
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
|
|
||||||
FieldDoc fieldDoc = (FieldDoc) scoreDoc;
|
|
||||||
fieldDoc.fields = Arrays.copyOfRange(fieldDoc.fields, 1, fieldDoc.fields.length);
|
|
||||||
}
|
|
||||||
TopFieldDocs newTopDocs = new TopFieldDocs(topDocs.totalHits, topDocs.scoreDocs, originalSortAndFormats.sort.getSort());
|
|
||||||
result.topDocs(new TopDocsAndMaxScore(newTopDocs, Float.NaN), originalSortAndFormats.formats);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -591,81 +387,5 @@ public class QueryPhase {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns true if more than 50% of data in the index have the same value
|
|
||||||
* The evaluation is approximation based on finding the median value and estimating its count
|
|
||||||
*/
|
|
||||||
private static boolean indexFieldHasDuplicateData(IndexReader reader, String field) throws IOException {
|
|
||||||
long docsNoDupl = 0; // number of docs in segments with NO duplicate data that would benefit optimization
|
|
||||||
long docsDupl = 0; // number of docs in segments with duplicate data that would NOT benefit optimization
|
|
||||||
for (LeafReaderContext lrc : reader.leaves()) {
|
|
||||||
PointValues pointValues = lrc.reader().getPointValues(field);
|
|
||||||
if (pointValues == null) continue;
|
|
||||||
int docCount = pointValues.getDocCount();
|
|
||||||
if (docCount <= 512) { // skipping small segments as estimateMedianCount doesn't work well on them
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
assert (pointValues.size() == docCount); // TODO: modify the code to handle multiple values
|
|
||||||
int duplDocCount = docCount / 2; // expected doc count of duplicate data
|
|
||||||
if (pointsHaveDuplicateData(pointValues, duplDocCount)) {
|
|
||||||
docsDupl += docCount;
|
|
||||||
} else {
|
|
||||||
docsNoDupl += docCount;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return (docsDupl > docsNoDupl);
|
|
||||||
}
|
|
||||||
|
|
||||||
static boolean pointsHaveDuplicateData(PointValues pointValues, int duplDocCount) throws IOException {
|
|
||||||
long minValue = LongPoint.decodeDimension(pointValues.getMinPackedValue(), 0);
|
|
||||||
long maxValue = LongPoint.decodeDimension(pointValues.getMaxPackedValue(), 0);
|
|
||||||
boolean hasDuplicateData = true;
|
|
||||||
while ((minValue < maxValue) && hasDuplicateData) {
|
|
||||||
long midValue = Math.floorDiv(minValue, 2) + Math.floorDiv(maxValue, 2); // to avoid overflow first divide each value by 2
|
|
||||||
long countLeft = estimatePointCount(pointValues, minValue, midValue);
|
|
||||||
long countRight = estimatePointCount(pointValues, midValue + 1, maxValue);
|
|
||||||
if ((countLeft >= countRight) && (countLeft > duplDocCount)) {
|
|
||||||
maxValue = midValue;
|
|
||||||
} else if ((countRight > countLeft) && (countRight > duplDocCount)) {
|
|
||||||
minValue = midValue + 1;
|
|
||||||
} else {
|
|
||||||
hasDuplicateData = false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return hasDuplicateData;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static long estimatePointCount(PointValues pointValues, long minValue, long maxValue) {
|
|
||||||
final byte[] minValueAsBytes = new byte[Long.BYTES];
|
|
||||||
LongPoint.encodeDimension(minValue, minValueAsBytes, 0);
|
|
||||||
final byte[] maxValueAsBytes = new byte[Long.BYTES];
|
|
||||||
LongPoint.encodeDimension(maxValue, maxValueAsBytes, 0);
|
|
||||||
|
|
||||||
PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
|
|
||||||
@Override
|
|
||||||
public void grow(int count) {}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void visit(int docID) {}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void visit(int docID, byte[] packedValue) {}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
|
||||||
if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0
|
|
||||||
|| FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0) {
|
|
||||||
return PointValues.Relation.CELL_OUTSIDE_QUERY;
|
|
||||||
}
|
|
||||||
if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0
|
|
||||||
|| FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0) {
|
|
||||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
|
||||||
}
|
|
||||||
return PointValues.Relation.CELL_INSIDE_QUERY;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
return pointValues.estimatePointCount(visitor);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class TimeExceededException extends RuntimeException {}
|
private static class TimeExceededException extends RuntimeException {}
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,11 +51,9 @@ import org.apache.lucene.index.NoMergePolicy;
|
||||||
import org.apache.lucene.index.RandomIndexWriter;
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.queries.MinDocQuery;
|
import org.apache.lucene.queries.MinDocQuery;
|
||||||
import org.apache.lucene.search.BooleanClause;
|
|
||||||
import org.apache.lucene.search.BooleanClause.Occur;
|
import org.apache.lucene.search.BooleanClause.Occur;
|
||||||
import org.apache.lucene.search.BooleanQuery;
|
import org.apache.lucene.search.BooleanQuery;
|
||||||
import org.apache.lucene.search.Collector;
|
import org.apache.lucene.search.Collector;
|
||||||
import org.apache.lucene.search.CollectorManager;
|
|
||||||
import org.apache.lucene.search.ConstantScoreQuery;
|
import org.apache.lucene.search.ConstantScoreQuery;
|
||||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
||||||
import org.apache.lucene.search.FieldComparator;
|
import org.apache.lucene.search.FieldComparator;
|
||||||
|
@ -82,14 +80,8 @@ import org.apache.lucene.search.join.ScoreMode;
|
||||||
import org.apache.lucene.search.spans.SpanNearQuery;
|
import org.apache.lucene.search.spans.SpanNearQuery;
|
||||||
import org.apache.lucene.search.spans.SpanTermQuery;
|
import org.apache.lucene.search.spans.SpanTermQuery;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.IOContext;
|
|
||||||
import org.apache.lucene.store.IndexInput;
|
|
||||||
import org.apache.lucene.store.IndexOutput;
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.FixedBitSet;
|
import org.apache.lucene.util.FixedBitSet;
|
||||||
import org.apache.lucene.util.bkd.BKDConfig;
|
|
||||||
import org.apache.lucene.util.bkd.BKDReader;
|
|
||||||
import org.apache.lucene.util.bkd.BKDWriter;
|
|
||||||
import org.opensearch.action.search.SearchShardTask;
|
import org.opensearch.action.search.SearchShardTask;
|
||||||
import org.opensearch.common.settings.Settings;
|
import org.opensearch.common.settings.Settings;
|
||||||
import org.opensearch.index.mapper.DateFieldMapper;
|
import org.opensearch.index.mapper.DateFieldMapper;
|
||||||
|
@ -114,13 +106,13 @@ import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import static org.opensearch.search.query.QueryPhase.pointsHaveDuplicateData;
|
|
||||||
import static org.opensearch.search.query.TopDocsCollectorContext.hasInfMaxScore;
|
|
||||||
import static org.hamcrest.Matchers.anyOf;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
|
||||||
import static org.hamcrest.Matchers.instanceOf;
|
|
||||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
|
import static org.hamcrest.Matchers.anyOf;
|
||||||
|
import static org.hamcrest.Matchers.instanceOf;
|
||||||
|
import static org.opensearch.search.query.TopDocsCollectorContext.hasInfMaxScore;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
@ -675,7 +667,7 @@ public class QueryPhaseTests extends IndexShardTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testNumericLongOrDateSortOptimization() throws Exception {
|
public void testEnhanceSortOnNumeric() throws Exception {
|
||||||
final String fieldNameLong = "long-field";
|
final String fieldNameLong = "long-field";
|
||||||
final String fieldNameDate = "date-field";
|
final String fieldNameDate = "date-field";
|
||||||
MappedFieldType fieldTypeLong = new NumberFieldMapper.NumberFieldType(fieldNameLong, NumberFieldMapper.NumberType.LONG);
|
MappedFieldType fieldTypeLong = new NumberFieldMapper.NumberFieldType(fieldNameLong, NumberFieldMapper.NumberType.LONG);
|
||||||
|
@ -684,153 +676,144 @@ public class QueryPhaseTests extends IndexShardTestCase {
|
||||||
when(mapperService.fieldType(fieldNameLong)).thenReturn(fieldTypeLong);
|
when(mapperService.fieldType(fieldNameLong)).thenReturn(fieldTypeLong);
|
||||||
when(mapperService.fieldType(fieldNameDate)).thenReturn(fieldTypeDate);
|
when(mapperService.fieldType(fieldNameDate)).thenReturn(fieldTypeDate);
|
||||||
// enough docs to have a tree with several leaf nodes
|
// enough docs to have a tree with several leaf nodes
|
||||||
final int numDocs = 3500 * 20;
|
final int numDocs = 3500 * 5;
|
||||||
Directory dir = newDirectory();
|
Directory dir = newDirectory();
|
||||||
IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(null));
|
IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
long firstValue = randomLongBetween(-10000000L, 10000000L);
|
||||||
|
long longValue = firstValue;
|
||||||
|
long dateValue = randomLongBetween(0, 3000000000000L);
|
||||||
for (int i = 1; i <= numDocs; ++i) {
|
for (int i = 1; i <= numDocs; ++i) {
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
long longValue = randomLongBetween(-10000000L, 10000000L);
|
|
||||||
doc.add(new LongPoint(fieldNameLong, longValue));
|
doc.add(new LongPoint(fieldNameLong, longValue));
|
||||||
doc.add(new NumericDocValuesField(fieldNameLong, longValue));
|
doc.add(new NumericDocValuesField(fieldNameLong, longValue));
|
||||||
longValue = randomLongBetween(0, 3000000000000L);
|
|
||||||
doc.add(new LongPoint(fieldNameDate, longValue));
|
doc.add(new LongPoint(fieldNameDate, dateValue));
|
||||||
doc.add(new NumericDocValuesField(fieldNameDate, longValue));
|
doc.add(new NumericDocValuesField(fieldNameDate, dateValue));
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
|
longValue++;
|
||||||
|
dateValue++;
|
||||||
if (i % 3500 == 0) writer.commit();
|
if (i % 3500 == 0) writer.commit();
|
||||||
}
|
}
|
||||||
writer.close();
|
writer.close();
|
||||||
final IndexReader reader = DirectoryReader.open(dir);
|
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);
|
final SortField sortFieldLong = new SortField(fieldNameLong, SortField.Type.LONG);
|
||||||
sortFieldLong.setMissingValue(Long.MAX_VALUE);
|
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);
|
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);
|
sortFieldDate.setMissingValue(Long.MAX_VALUE);
|
||||||
|
DocValueFormat dateFormat = fieldTypeDate.docValueFormat(null, null);
|
||||||
|
final Sort longSort = new Sort(sortFieldLong);
|
||||||
|
final Sort longDateSort = new Sort(sortFieldLong, sortFieldDate);
|
||||||
final Sort dateSort = new Sort(sortFieldDate);
|
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);
|
final Sort dateLongSort = new Sort(sortFieldDate, sortFieldLong);
|
||||||
sortAndFormats = new SortAndFormats(dateLongSort, new DocValueFormat[] { dateFormat, DocValueFormat.RAW });
|
SortAndFormats longSortAndFormats = new SortAndFormats(longSort, new DocValueFormat[] { DocValueFormat.RAW });
|
||||||
searchContext.sort(sortAndFormats);
|
SortAndFormats longDateSortAndFormats = new SortAndFormats(longDateSort, new DocValueFormat[] { DocValueFormat.RAW, dateFormat });
|
||||||
QueryPhase.executeInternal(searchContext);
|
SortAndFormats dateSortAndFormats = new SortAndFormats(dateSort, new DocValueFormat[] { dateFormat });
|
||||||
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true);
|
SortAndFormats dateLongSortAndFormats = new SortAndFormats(dateLongSort, new DocValueFormat[] { dateFormat, DocValueFormat.RAW });
|
||||||
|
ParsedQuery query = new ParsedQuery(new MatchAllDocsQuery());
|
||||||
|
SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap());
|
||||||
|
|
||||||
// 5. Test that sort optimization is run when from > 0 and size = 0
|
// 1. Test a sort on long field
|
||||||
{
|
{
|
||||||
sortAndFormats = new SortAndFormats(longSort, new DocValueFormat[] { DocValueFormat.RAW });
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
searchContext.sort(sortAndFormats);
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
searchContext.from(5);
|
searchContext.sort(longSortAndFormats);
|
||||||
searchContext.setSize(0);
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.setSize(10);
|
||||||
QueryPhase.executeInternal(searchContext);
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false);
|
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// 6. Test that sort optimization is NOT run with from = 0 and size= 0
|
// 2. Test a sort on long field + date field
|
||||||
{
|
{
|
||||||
sortAndFormats = new SortAndFormats(longSort, new DocValueFormat[] { DocValueFormat.RAW });
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
|
||||||
when(searchContext.mapperService()).thenReturn(mapperService);
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
searchContext.sort(sortAndFormats);
|
searchContext.sort(longDateSortAndFormats);
|
||||||
searchContext.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
|
searchContext.parsedQuery(query);
|
||||||
searchContext.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()));
|
searchContext.setTask(task);
|
||||||
searchContext.setSize(0);
|
searchContext.setSize(10);
|
||||||
|
|
||||||
QueryPhase.executeInternal(searchContext);
|
QueryPhase.executeInternal(searchContext);
|
||||||
TotalHits totalHits = searchContext.queryResult().topDocs().topDocs.totalHits;
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
assertEquals(TotalHits.Relation.EQUAL_TO, totalHits.relation);
|
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true);
|
||||||
assertEquals(numDocs, totalHits.value);
|
}
|
||||||
|
|
||||||
|
// 3. Test a sort on date field
|
||||||
|
{
|
||||||
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
|
searchContext.sort(dateSortAndFormats);
|
||||||
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.setSize(10);
|
||||||
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
|
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
// 4. Test a sort on date field + long field
|
||||||
|
{
|
||||||
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
|
searchContext.sort(dateLongSortAndFormats);
|
||||||
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.setSize(10);
|
||||||
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
|
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
// 5. Test that sort optimization is run when from > 0 and size = 0
|
||||||
|
{
|
||||||
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
|
searchContext.sort(longSortAndFormats);
|
||||||
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.from(5);
|
||||||
|
searchContext.setSize(0);
|
||||||
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
|
assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
// 6. Test that sort optimization works with from = 0 and size= 0
|
||||||
|
{
|
||||||
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
|
searchContext.sort(longSortAndFormats);
|
||||||
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.setSize(0);
|
||||||
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
// 7. Test that sort optimization works with search after
|
||||||
|
{
|
||||||
|
TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard, newContextSearcher(reader)));
|
||||||
|
when(searchContext.mapperService()).thenReturn(mapperService);
|
||||||
|
int afterDocument = (int) randomLongBetween(0, 50);
|
||||||
|
long afterValue = firstValue + afterDocument;
|
||||||
|
FieldDoc after = new FieldDoc(afterDocument, Float.NaN, new Long[] { afterValue });
|
||||||
|
searchContext.searchAfter(after);
|
||||||
|
searchContext.sort(longSortAndFormats);
|
||||||
|
searchContext.parsedQuery(query);
|
||||||
|
searchContext.setTask(task);
|
||||||
|
searchContext.setSize(10);
|
||||||
|
QueryPhase.executeInternal(searchContext);
|
||||||
|
assertTrue(searchContext.sort().sort.getSort()[0].getCanUsePoints());
|
||||||
|
final TopDocs topDocs = searchContext.queryResult().topDocs().topDocs;
|
||||||
|
long topValue = (long) ((FieldDoc) topDocs.scoreDocs[0]).fields[0];
|
||||||
|
assertThat(topValue, greaterThan(afterValue));
|
||||||
|
assertSortResults(topDocs, (long) numDocs, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
reader.close();
|
reader.close();
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testIndexHasDuplicateData() throws IOException {
|
|
||||||
int docsCount = 5000;
|
|
||||||
int maxPointsInLeafNode = 40;
|
|
||||||
float duplicateRatio = 0.7f;
|
|
||||||
long duplicateValue = randomLongBetween(-10000000L, 10000000L);
|
|
||||||
BKDConfig config = new BKDConfig(1, 1, 8, maxPointsInLeafNode);
|
|
||||||
try (Directory dir = newDirectory()) {
|
|
||||||
BKDWriter w = new BKDWriter(docsCount, dir, "tmp", config, 1, docsCount);
|
|
||||||
byte[] longBytes = new byte[8];
|
|
||||||
for (int docId = 0; docId < docsCount; docId++) {
|
|
||||||
long value = randomFloat() < duplicateRatio ? duplicateValue : randomLongBetween(-10000000L, 10000000L);
|
|
||||||
LongPoint.encodeDimension(value, longBytes, 0);
|
|
||||||
w.add(longBytes, docId);
|
|
||||||
}
|
|
||||||
try (
|
|
||||||
IndexOutput metaout = dir.createOutput("bkdmeta", IOContext.DEFAULT);
|
|
||||||
IndexOutput indexout = dir.createOutput("bkdindex", IOContext.DEFAULT);
|
|
||||||
IndexOutput dataout = dir.createOutput("bkddata", IOContext.DEFAULT)
|
|
||||||
) {
|
|
||||||
w.finish(metaout, indexout, dataout).run();
|
|
||||||
}
|
|
||||||
try (
|
|
||||||
IndexInput metain = dir.openInput("bkdmeta", IOContext.DEFAULT);
|
|
||||||
IndexInput indexin = dir.openInput("bkdindex", IOContext.DEFAULT);
|
|
||||||
IndexInput datain = dir.openInput("bkddata", IOContext.DEFAULT)
|
|
||||||
) {
|
|
||||||
BKDReader r = new BKDReader(metain, indexin, datain);
|
|
||||||
assertTrue(pointsHaveDuplicateData(r, r.getDocCount() / 2));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testIndexHasNoDuplicateData() throws IOException {
|
|
||||||
int docsCount = 5000;
|
|
||||||
int maxPointsInLeafNode = 40;
|
|
||||||
float duplicateRatio = 0.3f;
|
|
||||||
long duplicateValue = randomLongBetween(-10000000L, 10000000L);
|
|
||||||
BKDConfig config = new BKDConfig(1, 1, 8, maxPointsInLeafNode);
|
|
||||||
try (Directory dir = newDirectory()) {
|
|
||||||
BKDWriter w = new BKDWriter(docsCount, dir, "tmp", config, 1, docsCount);
|
|
||||||
byte[] longBytes = new byte[8];
|
|
||||||
for (int docId = 0; docId < docsCount; docId++) {
|
|
||||||
long value = randomFloat() < duplicateRatio ? duplicateValue : randomLongBetween(-10000000L, 10000000L);
|
|
||||||
LongPoint.encodeDimension(value, longBytes, 0);
|
|
||||||
w.add(longBytes, docId);
|
|
||||||
}
|
|
||||||
long indexFP;
|
|
||||||
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
|
|
||||||
Runnable finalizer = w.finish(out, out, out);
|
|
||||||
indexFP = out.getFilePointer();
|
|
||||||
finalizer.run();
|
|
||||||
;
|
|
||||||
}
|
|
||||||
try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
|
|
||||||
in.seek(indexFP);
|
|
||||||
BKDReader r = new BKDReader(in, in, in);
|
|
||||||
assertFalse(pointsHaveDuplicateData(r, r.getDocCount() / 2));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testMaxScoreQueryVisitor() {
|
public void testMaxScoreQueryVisitor() {
|
||||||
BitSetProducer producer = context -> new FixedBitSet(1);
|
BitSetProducer producer = context -> new FixedBitSet(1);
|
||||||
Query query = new OpenSearchToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested");
|
Query query = new OpenSearchToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested");
|
||||||
|
@ -1023,47 +1006,6 @@ public class QueryPhaseTests extends IndexShardTestCase {
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
// used to check that numeric long or date sort optimization was run
|
|
||||||
private static ContextIndexSearcher newOptimizedContextSearcher(IndexReader reader, int queryType) throws IOException {
|
|
||||||
return new ContextIndexSearcher(
|
|
||||||
reader,
|
|
||||||
IndexSearcher.getDefaultSimilarity(),
|
|
||||||
IndexSearcher.getDefaultQueryCache(),
|
|
||||||
IndexSearcher.getDefaultQueryCachingPolicy(),
|
|
||||||
true
|
|
||||||
) {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void search(
|
|
||||||
List<LeafReaderContext> leaves,
|
|
||||||
Weight weight,
|
|
||||||
CollectorManager manager,
|
|
||||||
QuerySearchResult result,
|
|
||||||
DocValueFormat[] formats,
|
|
||||||
TotalHits totalHits
|
|
||||||
) throws IOException {
|
|
||||||
final Query query = weight.getQuery();
|
|
||||||
assertTrue(query instanceof BooleanQuery);
|
|
||||||
List<BooleanClause> 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);
|
|
||||||
super.search(leaves, weight, manager, result, formats, totalHits);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void search(List<LeafReaderContext> leaves, Weight weight, Collector collector) {
|
|
||||||
assert (false); // should not be there, expected to search with CollectorManager
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class AssertingEarlyTerminationFilterCollector extends FilterCollector {
|
private static class AssertingEarlyTerminationFilterCollector extends FilterCollector {
|
||||||
private final int size;
|
private final int size;
|
||||||
|
|
||||||
|
|
|
@ -109,7 +109,7 @@ public class TestSearchContext extends SearchContext {
|
||||||
private int terminateAfter = DEFAULT_TERMINATE_AFTER;
|
private int terminateAfter = DEFAULT_TERMINATE_AFTER;
|
||||||
private SearchContextAggregations aggregations;
|
private SearchContextAggregations aggregations;
|
||||||
private ScrollContext scrollContext;
|
private ScrollContext scrollContext;
|
||||||
|
private FieldDoc searchAfter;
|
||||||
private final long originNanoTime = System.nanoTime();
|
private final long originNanoTime = System.nanoTime();
|
||||||
private final Map<String, SearchExtBuilder> searchExtBuilders = new HashMap<>();
|
private final Map<String, SearchExtBuilder> searchExtBuilders = new HashMap<>();
|
||||||
|
|
||||||
|
@ -393,13 +393,14 @@ public class TestSearchContext extends SearchContext {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SearchContext searchAfter(FieldDoc searchAfter) {
|
public SearchContext searchAfter(FieldDoc searchAfterDoc) {
|
||||||
return null;
|
this.searchAfter = searchAfterDoc;
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FieldDoc searchAfter() {
|
public FieldDoc searchAfter() {
|
||||||
return null;
|
return searchAfter;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
Loading…
Reference in New Issue