| package org.apache.lucene.search; |
| |
| /* |
| * Licensed to the Apache Software Foundation (ASF) under one or more |
| * contributor license agreements. See the NOTICE file distributed with |
| * this work for additional information regarding copyright ownership. |
| * The ASF 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. |
| */ |
| |
| import java.io.IOException; |
| import java.util.ArrayList; |
| import java.util.Arrays; |
| import java.util.Collection; |
| import java.util.Collections; |
| import java.util.List; |
| import java.util.Objects; |
| import java.util.Set; |
| import java.util.concurrent.Callable; |
| import java.util.concurrent.ExecutionException; |
| import java.util.concurrent.ExecutorService; |
| import java.util.concurrent.Future; |
| |
| import org.apache.lucene.document.Document; |
| import org.apache.lucene.index.DirectoryReader; // javadocs |
| import org.apache.lucene.index.FieldInvertState; |
| import org.apache.lucene.index.IndexReader; |
| import org.apache.lucene.index.IndexReaderContext; |
| import org.apache.lucene.index.IndexWriter; // javadocs |
| import org.apache.lucene.index.LeafReaderContext; |
| import org.apache.lucene.index.MultiFields; |
| import org.apache.lucene.index.ReaderUtil; |
| import org.apache.lucene.index.StoredFieldVisitor; |
| import org.apache.lucene.index.Term; |
| import org.apache.lucene.index.TermContext; |
| import org.apache.lucene.index.Terms; |
| import org.apache.lucene.search.similarities.DefaultSimilarity; |
| import org.apache.lucene.search.similarities.Similarity; |
| import org.apache.lucene.store.NIOFSDirectory; // javadoc |
| import org.apache.lucene.util.Bits; |
| import org.apache.lucene.util.BytesRef; |
| import org.apache.lucene.util.ThreadInterruptedException; |
| |
| /** Implements search over a single IndexReader. |
| * |
| * <p>Applications usually need only call the inherited |
| * {@link #search(Query,int)} |
| * or {@link #search(Query,Filter,int)} methods. For |
| * performance reasons, if your index is unchanging, you |
| * should share a single IndexSearcher instance across |
| * multiple searches instead of creating a new one |
| * per-search. If your index has changed and you wish to |
| * see the changes reflected in searching, you should |
| * use {@link DirectoryReader#openIfChanged(DirectoryReader)} |
| * to obtain a new reader and |
| * then create a new IndexSearcher from that. Also, for |
| * low-latency turnaround it's best to use a near-real-time |
| * reader ({@link DirectoryReader#open(IndexWriter,boolean)}). |
| * Once you have a new {@link IndexReader}, it's relatively |
| * cheap to create a new IndexSearcher from it. |
| * |
| * <a name="thread-safety"></a><p><b>NOTE</b>: <code>{@link |
| * IndexSearcher}</code> instances are completely |
| * thread safe, meaning multiple threads can call any of its |
| * methods, concurrently. If your application requires |
| * external synchronization, you should <b>not</b> |
| * synchronize on the <code>IndexSearcher</code> instance; |
| * use your own (non-Lucene) objects instead.</p> |
| */ |
| public class IndexSearcher { |
| |
| /** A search-time {@link Similarity} that does not make use of scoring factors |
| * and may be used when scores are not needed. */ |
| private static final Similarity NON_SCORING_SIMILARITY = new Similarity() { |
| |
| @Override |
| public long computeNorm(FieldInvertState state) { |
| throw new UnsupportedOperationException("This Similarity may only be used for searching, not indexing"); |
| } |
| |
| @Override |
| public SimWeight computeWeight(CollectionStatistics collectionStats, TermStatistics... termStats) { |
| return new SimWeight() { |
| |
| @Override |
| public float getValueForNormalization() { |
| return 1f; |
| } |
| |
| @Override |
| public void normalize(float queryNorm, float boost) {} |
| |
| }; |
| } |
| |
| @Override |
| public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException { |
| return new SimScorer() { |
| |
| @Override |
| public float score(int doc, float freq) { |
| return 0f; |
| } |
| |
| @Override |
| public float computeSlopFactor(int distance) { |
| return 1f; |
| } |
| |
| @Override |
| public float computePayloadFactor(int doc, int start, int end, BytesRef payload) { |
| return 1f; |
| } |
| |
| }; |
| } |
| |
| }; |
| |
| private static QueryCache DEFAULT_QUERY_CACHE; |
| private static QueryCachingPolicy DEFAULT_CACHING_POLICY = new UsageTrackingQueryCachingPolicy(); |
| static { |
| final int maxCachedQueries = 1000; |
| // min of 32MB or 5% of the heap size |
| final long maxRamBytesUsed = Math.min(1L << 25, Runtime.getRuntime().maxMemory() / 20); |
| DEFAULT_QUERY_CACHE = new LRUQueryCache(maxCachedQueries, maxRamBytesUsed); |
| } |
| |
| final IndexReader reader; // package private for testing! |
| |
| // NOTE: these members might change in incompatible ways |
| // in the next release |
| protected final IndexReaderContext readerContext; |
| protected final List<LeafReaderContext> leafContexts; |
| /** used with executor - each slice holds a set of leafs executed within one thread */ |
| protected final LeafSlice[] leafSlices; |
| |
| // These are only used for multi-threaded search |
| private final ExecutorService executor; |
| |
| // the default Similarity |
| private static final Similarity defaultSimilarity = new DefaultSimilarity(); |
| |
| private QueryCache queryCache = DEFAULT_QUERY_CACHE; |
| private QueryCachingPolicy queryCachingPolicy = DEFAULT_CACHING_POLICY; |
| |
| /** |
| * Expert: returns a default Similarity instance. |
| * In general, this method is only called to initialize searchers and writers. |
| * User code and query implementations should respect |
| * {@link IndexSearcher#getSimilarity(boolean)}. |
| * @lucene.internal |
| */ |
| public static Similarity getDefaultSimilarity() { |
| return defaultSimilarity; |
| } |
| |
| /** |
| * Expert: Get the default {@link QueryCache} or {@code null} if the cache is disabled. |
| * @lucene.internal |
| */ |
| public static QueryCache getDefaultQueryCache() { |
| return DEFAULT_QUERY_CACHE; |
| } |
| |
| /** |
| * Expert: set the default {@link QueryCache} instance. |
| * @lucene.internal |
| */ |
| public static void setDefaultQueryCache(QueryCache defaultQueryCache) { |
| DEFAULT_QUERY_CACHE = defaultQueryCache; |
| } |
| |
| /** |
| * Expert: Get the default {@link QueryCachingPolicy}. |
| * @lucene.internal |
| */ |
| public static QueryCachingPolicy getDefaultQueryCachingPolicy() { |
| return DEFAULT_CACHING_POLICY; |
| } |
| |
| /** |
| * Expert: set the default {@link QueryCachingPolicy} instance. |
| * @lucene.internal |
| */ |
| public static void setDefaultQueryCachingPolicy(QueryCachingPolicy defaultQueryCachingPolicy) { |
| DEFAULT_CACHING_POLICY = defaultQueryCachingPolicy; |
| } |
| |
| /** The Similarity implementation used by this searcher. */ |
| private Similarity similarity = defaultSimilarity; |
| |
| /** Creates a searcher searching the provided index. */ |
| public IndexSearcher(IndexReader r) { |
| this(r, null); |
| } |
| |
| /** Runs searches for each segment separately, using the |
| * provided ExecutorService. IndexSearcher will not |
| * close/awaitTermination this ExecutorService on |
| * close; you must do so, eventually, on your own. NOTE: |
| * if you are using {@link NIOFSDirectory}, do not use |
| * the shutdownNow method of ExecutorService as this uses |
| * Thread.interrupt under-the-hood which can silently |
| * close file descriptors (see <a |
| * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>). |
| * |
| * @lucene.experimental */ |
| public IndexSearcher(IndexReader r, ExecutorService executor) { |
| this(r.getContext(), executor); |
| } |
| |
| /** |
| * Creates a searcher searching the provided top-level {@link IndexReaderContext}. |
| * <p> |
| * Given a non-<code>null</code> {@link ExecutorService} this method runs |
| * searches for each segment separately, using the provided ExecutorService. |
| * IndexSearcher will not close/awaitTermination this ExecutorService on |
| * close; you must do so, eventually, on your own. NOTE: if you are using |
| * {@link NIOFSDirectory}, do not use the shutdownNow method of |
| * ExecutorService as this uses Thread.interrupt under-the-hood which can |
| * silently close file descriptors (see <a |
| * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>). |
| * |
| * @see IndexReaderContext |
| * @see IndexReader#getContext() |
| * @lucene.experimental |
| */ |
| public IndexSearcher(IndexReaderContext context, ExecutorService executor) { |
| assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader(); |
| reader = context.reader(); |
| this.executor = executor; |
| this.readerContext = context; |
| leafContexts = context.leaves(); |
| this.leafSlices = executor == null ? null : slices(leafContexts); |
| } |
| |
| /** |
| * Creates a searcher searching the provided top-level {@link IndexReaderContext}. |
| * |
| * @see IndexReaderContext |
| * @see IndexReader#getContext() |
| * @lucene.experimental |
| */ |
| public IndexSearcher(IndexReaderContext context) { |
| this(context, null); |
| } |
| |
| /** |
| * Set the {@link QueryCache} to use when scores are not needed. |
| * A value of {@code null} indicates that query matches should never be |
| * cached. This method should be called <b>before</b> starting using this |
| * {@link IndexSearcher}. |
| * <p>NOTE: When using a query cache, queries should not be modified after |
| * they have been passed to IndexSearcher. |
| * @see QueryCache |
| * @lucene.experimental |
| */ |
| public void setQueryCache(QueryCache queryCache) { |
| this.queryCache = queryCache; |
| } |
| |
| /** |
| * Return the query cache of this {@link IndexSearcher}. This will be either |
| * the {@link #getDefaultQueryCache() default query cache} or the query cache |
| * that was last set through {@link #setQueryCache(QueryCache)}. A return |
| * value of {@code null} indicates that caching is disabled. |
| * @lucene.experimental |
| */ |
| public QueryCache getQueryCache() { |
| return queryCache; |
| } |
| |
| /** |
| * Set the {@link QueryCachingPolicy} to use for query caching. |
| * This method should be called <b>before</b> starting using this |
| * {@link IndexSearcher}. |
| * @see QueryCachingPolicy |
| * @lucene.experimental |
| */ |
| public void setQueryCachingPolicy(QueryCachingPolicy queryCachingPolicy) { |
| this.queryCachingPolicy = Objects.requireNonNull(queryCachingPolicy); |
| } |
| |
| /** |
| * Return the query cache of this {@link IndexSearcher}. This will be either |
| * the {@link #getDefaultQueryCachingPolicy() default policy} or the policy |
| * that was last set through {@link #setQueryCachingPolicy(QueryCachingPolicy)}. |
| * @lucene.experimental |
| */ |
| public QueryCachingPolicy getQueryCachingPolicy() { |
| return queryCachingPolicy; |
| } |
| |
| /** |
| * Expert: Creates an array of leaf slices each holding a subset of the given leaves. |
| * Each {@link LeafSlice} is executed in a single thread. By default there |
| * will be one {@link LeafSlice} per leaf ({@link org.apache.lucene.index.LeafReaderContext}). |
| */ |
| protected LeafSlice[] slices(List<LeafReaderContext> leaves) { |
| LeafSlice[] slices = new LeafSlice[leaves.size()]; |
| for (int i = 0; i < slices.length; i++) { |
| slices[i] = new LeafSlice(leaves.get(i)); |
| } |
| return slices; |
| } |
| |
| |
| /** Return the {@link IndexReader} this searches. */ |
| public IndexReader getIndexReader() { |
| return reader; |
| } |
| |
| /** |
| * Sugar for <code>.getIndexReader().document(docID)</code> |
| * @see IndexReader#document(int) |
| */ |
| public Document doc(int docID) throws IOException { |
| return reader.document(docID); |
| } |
| |
| /** |
| * Sugar for <code>.getIndexReader().document(docID, fieldVisitor)</code> |
| * @see IndexReader#document(int, StoredFieldVisitor) |
| */ |
| public void doc(int docID, StoredFieldVisitor fieldVisitor) throws IOException { |
| reader.document(docID, fieldVisitor); |
| } |
| |
| /** |
| * Sugar for <code>.getIndexReader().document(docID, fieldsToLoad)</code> |
| * @see IndexReader#document(int, Set) |
| */ |
| public Document doc(int docID, Set<String> fieldsToLoad) throws IOException { |
| return reader.document(docID, fieldsToLoad); |
| } |
| |
| /** Expert: Set the Similarity implementation used by this IndexSearcher. |
| * |
| */ |
| public void setSimilarity(Similarity similarity) { |
| this.similarity = similarity; |
| } |
| |
| /** Expert: Get the {@link Similarity} to use to compute scores. When |
| * {@code needsScores} is {@code false}, this method will return a simple |
| * {@link Similarity} that does not leverage scoring factors such as norms. |
| * When {@code needsScores} is {@code true}, this returns the |
| * {@link Similarity} that has been set through {@link #setSimilarity(Similarity)} |
| * or the {@link #getDefaultSimilarity()} default {@link Similarity} if none |
| * has been set explicitely. */ |
| public Similarity getSimilarity(boolean needsScores) { |
| return needsScores ? similarity : NON_SCORING_SIMILARITY; |
| } |
| |
| /** @lucene.internal |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead */ |
| @Deprecated |
| protected Query wrapFilter(Query query, Filter filter) { |
| return (filter == null) ? query : new FilteredQuery(query, filter); |
| } |
| |
| /** |
| * Count how many documents match the given query. |
| */ |
| public int count(Query query) throws IOException { |
| query = rewrite(query); |
| while (true) { |
| // remove wrappers that don't matter for counts |
| if (query instanceof ConstantScoreQuery) { |
| query = ((ConstantScoreQuery) query).getQuery(); |
| } else { |
| break; |
| } |
| } |
| |
| // some counts can be computed in constant time |
| if (query instanceof MatchAllDocsQuery) { |
| return reader.numDocs(); |
| } else if (query instanceof TermQuery && reader.hasDeletions() == false) { |
| Term term = ((TermQuery) query).getTerm(); |
| int count = 0; |
| for (LeafReaderContext leaf : reader.leaves()) { |
| count += leaf.reader().docFreq(term); |
| } |
| return count; |
| } |
| |
| // general case: create a collecor and count matches |
| final CollectorManager<TotalHitCountCollector, Integer> collectorManager = new CollectorManager<TotalHitCountCollector, Integer>() { |
| |
| @Override |
| public TotalHitCountCollector newCollector() throws IOException { |
| return new TotalHitCountCollector(); |
| } |
| |
| @Override |
| public Integer reduce(Collection<TotalHitCountCollector> collectors) throws IOException { |
| int total = 0; |
| for (TotalHitCountCollector collector : collectors) { |
| total += collector.getTotalHits(); |
| } |
| return total; |
| } |
| |
| }; |
| return search(query, collectorManager); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code> where all results are after a previous |
| * result (<code>after</code>). |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public TopDocs searchAfter(final ScoreDoc after, Query query, int numHits) throws IOException { |
| final int limit = Math.max(1, reader.maxDoc()); |
| if (after != null && after.doc >= limit) { |
| throw new IllegalArgumentException("after.doc exceeds the number of documents in the reader: after.doc=" |
| + after.doc + " limit=" + limit); |
| } |
| numHits = Math.min(numHits, limit); |
| |
| final int cappedNumHits = Math.min(numHits, limit); |
| |
| final CollectorManager<TopScoreDocCollector, TopDocs> manager = new CollectorManager<TopScoreDocCollector, TopDocs>() { |
| |
| @Override |
| public TopScoreDocCollector newCollector() throws IOException { |
| return TopScoreDocCollector.create(cappedNumHits, after); |
| } |
| |
| @Override |
| public TopDocs reduce(Collection<TopScoreDocCollector> collectors) throws IOException { |
| final TopDocs[] topDocs = new TopDocs[collectors.size()]; |
| int i = 0; |
| for (TopScoreDocCollector collector : collectors) { |
| topDocs[i++] = collector.topDocs(); |
| } |
| return TopDocs.merge(cappedNumHits, topDocs); |
| } |
| |
| }; |
| |
| return search(query, manager); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code>, applying <code>filter</code> if non-null, |
| * where all results are after a previous result (<code>after</code>). |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopDocs searchAfter(ScoreDoc after, Query query, Filter filter, int n) throws IOException { |
| return searchAfter(after, wrapFilter(query, filter), n); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code>. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public TopDocs search(Query query, int n) |
| throws IOException { |
| return searchAfter(null, query, n); |
| } |
| |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code>, applying <code>filter</code> if non-null. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopDocs search(Query query, Filter filter, int n) |
| throws IOException { |
| return search(wrapFilter(query, filter), n); |
| } |
| |
| /** Lower-level search API. |
| * |
| * <p>{@link LeafCollector#collect(int)} is called for every matching |
| * document. |
| * |
| * @param query to match documents |
| * @param filter if non-null, used to permit documents to be collected. |
| * @param results to receive hits |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final void search(Query query, Filter filter, Collector results) |
| throws IOException { |
| search(wrapFilter(query, filter), results); |
| } |
| |
| /** Lower-level search API. |
| * |
| * <p>{@link LeafCollector#collect(int)} is called for every matching document. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public void search(Query query, Collector results) |
| throws IOException { |
| search(leafContexts, createNormalizedWeight(query, results.needsScores()), results); |
| } |
| |
| /** Search implementation with arbitrary sorting. Finds |
| * the top <code>n</code> hits for <code>query</code>, applying |
| * <code>filter</code> if non-null, and sorting the hits by the criteria in |
| * <code>sort</code>. |
| * |
| * <p>NOTE: this does not compute scores by default; use |
| * {@link IndexSearcher#search(Query,Filter,int,Sort,boolean,boolean)} to |
| * control scoring. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopFieldDocs search(Query query, Filter filter, int n, |
| Sort sort) throws IOException { |
| return search(query, filter, n, sort, false, false); |
| } |
| |
| /** Search implementation with arbitrary sorting, plus |
| * control over whether hit scores and max score |
| * should be computed. Finds |
| * the top <code>n</code> hits for <code>query</code>, applying |
| * <code>filter</code> if non-null, and sorting the hits by the criteria in |
| * <code>sort</code>. If <code>doDocScores</code> is <code>true</code> |
| * then the score of each hit will be computed and |
| * returned. If <code>doMaxScore</code> is |
| * <code>true</code> then the maximum score over all |
| * collected hits will be computed. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopFieldDocs search(Query query, Filter filter, int n, |
| Sort sort, boolean doDocScores, boolean doMaxScore) throws IOException { |
| return searchAfter(null, query, filter, n, sort, doDocScores, doMaxScore); |
| } |
| |
| /** Search implementation with arbitrary sorting, plus |
| * control over whether hit scores and max score |
| * should be computed. Finds |
| * the top <code>n</code> hits for <code>query</code>, applying |
| * <code>filter</code> if non-null, and sorting the hits by the criteria in |
| * <code>sort</code>. If <code>doDocScores</code> is <code>true</code> |
| * then the score of each hit will be computed and |
| * returned. If <code>doMaxScore</code> is |
| * <code>true</code> then the maximum score over all |
| * collected hits will be computed. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public final TopFieldDocs search(Query query, int n, |
| Sort sort, boolean doDocScores, boolean doMaxScore) throws IOException { |
| return searchAfter(null, query, n, sort, doDocScores, doMaxScore); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code>, applying <code>filter</code> if non-null, |
| * where all results are after a previous result (<code>after</code>). |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopFieldDocs searchAfter(ScoreDoc after, Query query, Filter filter, int n, Sort sort) throws IOException { |
| return searchAfter(after, query, filter, n, sort, false, false); |
| } |
| |
| /** |
| * Search implementation with arbitrary sorting and no filter. |
| * @param query The query to search for |
| * @param n Return only the top n results |
| * @param sort The {@link org.apache.lucene.search.Sort} object |
| * @return The top docs, sorted according to the supplied {@link org.apache.lucene.search.Sort} instance |
| * @throws IOException if there is a low-level I/O error |
| */ |
| public TopFieldDocs search(Query query, int n, |
| Sort sort) throws IOException { |
| return search(query, null, n, sort, false, false); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code> where all results are after a previous |
| * result (<code>after</code>). |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public TopDocs searchAfter(ScoreDoc after, Query query, int n, Sort sort) throws IOException { |
| return searchAfter(after, query, null, n, sort, false, false); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code> where all results are after a previous |
| * result (<code>after</code>), allowing control over |
| * whether hit scores and max score should be computed. |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. If <code>doDocScores</code> is <code>true</code> |
| * then the score of each hit will be computed and |
| * returned. If <code>doMaxScore</code> is |
| * <code>true</code> then the maximum score over all |
| * collected hits will be computed. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| * @deprecated Use {@link BooleanQuery boolean queries} with |
| * {@link BooleanClause.Occur#FILTER} clauses instead |
| */ |
| @Deprecated |
| public final TopFieldDocs searchAfter(ScoreDoc after, Query query, Filter filter, int numHits, Sort sort, |
| boolean doDocScores, boolean doMaxScore) throws IOException { |
| if (after != null && !(after instanceof FieldDoc)) { |
| // TODO: if we fix type safety of TopFieldDocs we can |
| // remove this |
| throw new IllegalArgumentException("after must be a FieldDoc; got " + after); |
| } |
| return searchAfter((FieldDoc) after, wrapFilter(query, filter), numHits, sort, doDocScores, doMaxScore); |
| } |
| |
| /** Finds the top <code>n</code> |
| * hits for <code>query</code> where all results are after a previous |
| * result (<code>after</code>), allowing control over |
| * whether hit scores and max score should be computed. |
| * <p> |
| * By passing the bottom result from a previous page as <code>after</code>, |
| * this method can be used for efficient 'deep-paging' across potentially |
| * large result sets. If <code>doDocScores</code> is <code>true</code> |
| * then the score of each hit will be computed and |
| * returned. If <code>doMaxScore</code> is |
| * <code>true</code> then the maximum score over all |
| * collected hits will be computed. |
| * |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public final TopFieldDocs searchAfter(ScoreDoc after, Query query, int numHits, Sort sort, |
| boolean doDocScores, boolean doMaxScore) throws IOException { |
| if (after != null && !(after instanceof FieldDoc)) { |
| // TODO: if we fix type safety of TopFieldDocs we can |
| // remove this |
| throw new IllegalArgumentException("after must be a FieldDoc; got " + after); |
| } |
| return searchAfter((FieldDoc) after, query, numHits, sort, doDocScores, doMaxScore); |
| } |
| |
| private TopFieldDocs searchAfter(final FieldDoc after, Query query, int numHits, final Sort sort, |
| final boolean doDocScores, final boolean doMaxScore) throws IOException { |
| final int limit = Math.max(1, reader.maxDoc()); |
| if (after != null && after.doc >= limit) { |
| throw new IllegalArgumentException("after.doc exceeds the number of documents in the reader: after.doc=" |
| + after.doc + " limit=" + limit); |
| } |
| final int cappedNumHits = Math.min(numHits, limit); |
| |
| final CollectorManager<TopFieldCollector, TopFieldDocs> manager = new CollectorManager<TopFieldCollector, TopFieldDocs>() { |
| |
| @Override |
| public TopFieldCollector newCollector() throws IOException { |
| final boolean fillFields = true; |
| return TopFieldCollector.create(sort, cappedNumHits, after, fillFields, doDocScores, doMaxScore); |
| } |
| |
| @Override |
| public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException { |
| final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()]; |
| int i = 0; |
| for (TopFieldCollector collector : collectors) { |
| topDocs[i++] = collector.topDocs(); |
| } |
| return TopDocs.merge(sort, cappedNumHits, topDocs); |
| } |
| |
| }; |
| |
| return search(query, manager); |
| } |
| |
| /** |
| * Lower-level search API. |
| * Search all leaves using the given {@link CollectorManager}. In contrast |
| * to {@link #search(Query, Collector)}, this method will use the searcher's |
| * {@link ExecutorService} in order to parallelize execution of the collection |
| * on the configured {@link #leafSlices}. |
| * @see CollectorManager |
| * @lucene.experimental |
| */ |
| public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager) throws IOException { |
| if (executor == null) { |
| final C collector = collectorManager.newCollector(); |
| search(query, collector); |
| return collectorManager.reduce(Collections.singletonList(collector)); |
| } else { |
| final List<C> collectors = new ArrayList<>(leafSlices.length); |
| boolean needsScores = false; |
| for (int i = 0; i < leafSlices.length; ++i) { |
| final C collector = collectorManager.newCollector(); |
| collectors.add(collector); |
| needsScores |= collector.needsScores(); |
| } |
| |
| final Weight weight = createNormalizedWeight(query, needsScores); |
| final List<Future<C>> topDocsFutures = new ArrayList<>(leafSlices.length); |
| for (int i = 0; i < leafSlices.length; ++i) { |
| final LeafReaderContext[] leaves = leafSlices[i].leaves; |
| final C collector = collectors.get(i); |
| topDocsFutures.add(executor.submit(new Callable<C>() { |
| @Override |
| public C call() throws Exception { |
| search(Arrays.asList(leaves), weight, collector); |
| return collector; |
| } |
| })); |
| } |
| |
| final List<C> collectedCollectors = new ArrayList<>(); |
| for (Future<C> future : topDocsFutures) { |
| try { |
| collectedCollectors.add(future.get()); |
| } catch (InterruptedException e) { |
| throw new ThreadInterruptedException(e); |
| } catch (ExecutionException e) { |
| throw new RuntimeException(e); |
| } |
| } |
| |
| return collectorManager.reduce(collectors); |
| } |
| } |
| |
| /** |
| * Lower-level search API. |
| * |
| * <p> |
| * {@link LeafCollector#collect(int)} is called for every document. <br> |
| * |
| * <p> |
| * NOTE: this method executes the searches on all given leaves exclusively. |
| * To search across all the searchers leaves use {@link #leafContexts}. |
| * |
| * @param leaves |
| * the searchers leaves to execute the searches on |
| * @param weight |
| * to match documents |
| * @param collector |
| * to receive hits |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector) |
| throws IOException { |
| |
| // TODO: should we make this |
| // threaded...? the Collector could be sync'd? |
| // always use single thread: |
| for (LeafReaderContext ctx : leaves) { // search each subreader |
| final LeafCollector leafCollector; |
| try { |
| leafCollector = collector.getLeafCollector(ctx); |
| } catch (CollectionTerminatedException e) { |
| // there is no doc of interest in this reader context |
| // continue with the following leaf |
| continue; |
| } |
| BulkScorer scorer = weight.bulkScorer(ctx); |
| if (scorer != null) { |
| try { |
| scorer.score(leafCollector, ctx.reader().getLiveDocs()); |
| } catch (CollectionTerminatedException e) { |
| // collection was terminated prematurely |
| // continue with the following leaf |
| } |
| } |
| } |
| } |
| |
| /** Expert: called to re-write queries into primitive queries. |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| public Query rewrite(Query original) throws IOException { |
| Query query = original; |
| for (Query rewrittenQuery = query.rewrite(reader); rewrittenQuery != query; |
| rewrittenQuery = query.rewrite(reader)) { |
| query = rewrittenQuery; |
| } |
| return query; |
| } |
| |
| /** Returns an Explanation that describes how <code>doc</code> scored against |
| * <code>query</code>. |
| * |
| * <p>This is intended to be used in developing Similarity implementations, |
| * and, for good performance, should not be displayed with every hit. |
| * Computing an explanation is as expensive as executing the query over the |
| * entire index. |
| */ |
| public Explanation explain(Query query, int doc) throws IOException { |
| return explain(createNormalizedWeight(query, true), doc); |
| } |
| |
| /** Expert: low-level implementation method |
| * Returns an Explanation that describes how <code>doc</code> scored against |
| * <code>weight</code>. |
| * |
| * <p>This is intended to be used in developing Similarity implementations, |
| * and, for good performance, should not be displayed with every hit. |
| * Computing an explanation is as expensive as executing the query over the |
| * entire index. |
| * <p>Applications should call {@link IndexSearcher#explain(Query, int)}. |
| * @throws BooleanQuery.TooManyClauses If a query would exceed |
| * {@link BooleanQuery#getMaxClauseCount()} clauses. |
| */ |
| protected Explanation explain(Weight weight, int doc) throws IOException { |
| int n = ReaderUtil.subIndex(doc, leafContexts); |
| final LeafReaderContext ctx = leafContexts.get(n); |
| int deBasedDoc = doc - ctx.docBase; |
| final Bits liveDocs = ctx.reader().getLiveDocs(); |
| if (liveDocs != null && liveDocs.get(deBasedDoc) == false) { |
| return Explanation.noMatch("Document " + doc + " is deleted"); |
| } |
| return weight.explain(ctx, deBasedDoc); |
| } |
| |
| /** |
| * Creates a normalized weight for a top-level {@link Query}. |
| * The query is rewritten by this method and {@link Query#createWeight} called, |
| * afterwards the {@link Weight} is normalized. The returned {@code Weight} |
| * can then directly be used to get a {@link Scorer}. |
| * @lucene.internal |
| */ |
| public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException { |
| query = rewrite(query); |
| Weight weight = createWeight(query, needsScores); |
| float v = weight.getValueForNormalization(); |
| float norm = getSimilarity(needsScores).queryNorm(v); |
| if (Float.isInfinite(norm) || Float.isNaN(norm)) { |
| norm = 1.0f; |
| } |
| weight.normalize(norm, 1.0f); |
| return weight; |
| } |
| |
| /** |
| * Creates a {@link Weight} for the given query, potentially adding caching |
| * if possible and configured. |
| * @lucene.experimental |
| */ |
| public Weight createWeight(Query query, boolean needsScores) throws IOException { |
| final QueryCache queryCache = this.queryCache; |
| Weight weight = query.createWeight(this, needsScores); |
| if (needsScores == false && queryCache != null) { |
| weight = queryCache.doCache(weight, queryCachingPolicy); |
| } |
| return weight; |
| } |
| |
| /** |
| * Returns this searchers the top-level {@link IndexReaderContext}. |
| * @see IndexReader#getContext() |
| */ |
| /* sugar for #getReader().getTopReaderContext() */ |
| public IndexReaderContext getTopReaderContext() { |
| return readerContext; |
| } |
| |
| /** |
| * A class holding a subset of the {@link IndexSearcher}s leaf contexts to be |
| * executed within a single thread. |
| * |
| * @lucene.experimental |
| */ |
| public static class LeafSlice { |
| final LeafReaderContext[] leaves; |
| |
| public LeafSlice(LeafReaderContext... leaves) { |
| this.leaves = leaves; |
| } |
| } |
| |
| @Override |
| public String toString() { |
| return "IndexSearcher(" + reader + "; executor=" + executor + ")"; |
| } |
| |
| /** |
| * Returns {@link TermStatistics} for a term. |
| * |
| * This can be overridden for example, to return a term's statistics |
| * across a distributed collection. |
| * @lucene.experimental |
| */ |
| public TermStatistics termStatistics(Term term, TermContext context) throws IOException { |
| return new TermStatistics(term.bytes(), context.docFreq(), context.totalTermFreq()); |
| } |
| |
| /** |
| * Returns {@link CollectionStatistics} for a field. |
| * |
| * This can be overridden for example, to return a field's statistics |
| * across a distributed collection. |
| * @lucene.experimental |
| */ |
| public CollectionStatistics collectionStatistics(String field) throws IOException { |
| final int docCount; |
| final long sumTotalTermFreq; |
| final long sumDocFreq; |
| |
| assert field != null; |
| |
| Terms terms = MultiFields.getTerms(reader, field); |
| if (terms == null) { |
| docCount = 0; |
| sumTotalTermFreq = 0; |
| sumDocFreq = 0; |
| } else { |
| docCount = terms.getDocCount(); |
| sumTotalTermFreq = terms.getSumTotalTermFreq(); |
| sumDocFreq = terms.getSumDocFreq(); |
| } |
| return new CollectionStatistics(field, reader.maxDoc(), docCount, sumTotalTermFreq, sumDocFreq); |
| } |
| } |