Package org.apache.lucene.search
Table Of Contents
- Search Basics
- The Query Classes
- Scoring: Introduction
- Scoring: Basics
- Changing the Scoring
- Appendix: Search Algorithm
Search Basics
Lucene offers a wide variety of Query implementations, most
 of which are in this package or the queries
 module. These implementations can be combined in a wide variety of ways to provide complex
 querying capabilities along with information about where matches took place in the document
 collection. The Query Classes section below highlights some of the more
 important Query classes. For details on implementing your own Query class, see Custom Queries -- Expert Level below.
 
To perform a search, applications usually call IndexSearcher.search(Query,int).
 
Once a Query has been created and submitted to the IndexSearcher, the scoring process begins. After some
 infrastructure setup, control finally passes to the Weight implementation and its Scorer or BulkScorer instances. See the Algorithm section for more notes on the process.
 
 
 
 
Query Classes
TermQuery 
 Of the various implementations of Query, the TermQuery is the easiest to understand and the most often
 used in applications. A TermQuery matches all the
 documents that contain the specified Term, which is a word
 that occurs in a certain Field. Thus, a TermQuery identifies and scores all Documents that have a Field with the specified string in it. Constructing a TermQuery is as simple as:
 
 TermQuery tq = new TermQuery(new Term("fieldName", "term"));
 
 In this example, the Query identifies all Documents that have the Field named "fieldName" containing the word 
 "term".
 BooleanQuery 
 Things start to get interesting when one combines multiple TermQuery instances into a BooleanQuery. A BooleanQuery contains multiple BooleanClauses, where each clause contains a sub-query
 (Query instance) and an operator (from BooleanClause.Occur) describing how that sub-query
 is combined with the other clauses:
 
- 
       SHOULD— Use this operator when a clause can occur in the result set, but is not required. If a query is made up of all SHOULD clauses, then every document in the result set matches at least one of these clauses.
- 
       MUST— Use this operator when a clause is required to occur in the result set and should contribute to the score. Every document in the result set will match all such clauses.
- 
       FILTER— Use this operator when a clause is required to occur in the result set but should not contribute to the score. Every document in the result set will match all such clauses.
- 
       MUST NOT— Use this operator when a clause must not occur in the result set. No document in the result set will match any such clauses.
BooleanClause instances. If too many clauses are added, a
 TooManyClauses exception will be
 thrown during searching. This most often occurs when a Query is rewritten into a BooleanQuery with many
 TermQuery clauses, for example by WildcardQuery. The default setting for the maximum number
 of clauses is 1024, but this can be changed via the static method IndexSearcher.setMaxClauseCount(int).
 Phrases
Another common search is to find documents containing certain phrases. This is handled in different ways:
- 
       PhraseQuery— Matches a sequence ofTerms.PhraseQueryuses a slop factor to determine how many positions may occur between any two terms in the phrase and still be considered a match. The slop is 0 by default, meaning the phrase must match exactly.
- 
       MultiPhraseQuery— A more general form of PhraseQuery that accepts multiple Terms for a position in the phrase. For example, this can be used to perform phrase queries that also incorporate synonyms.
- 
       Interval queries in the Queries module 
PointRangeQuery 
 The PointRangeQuery matches all documents
 that occur in a numeric range. For PointRangeQuery to work, you must index the values using a one
 of the numeric fields (IntPoint, LongPoint, FloatPoint, or DoublePoint).
 
PrefixQuery, WildcardQuery, RegexpQuery 
 While the PrefixQuery has a different
 implementation, it is essentially a special case of the WildcardQuery. The PrefixQuery allows an application to identify all documents
 with terms that begin with a certain string. The WildcardQuery generalizes this by allowing for the use of * (matches 0 or more
 characters) and ? (matches exactly one character) wildcards. Note that the WildcardQuery can be quite slow. Also note that WildcardQuery should not start with * and
 ?, as these are extremely slow. Some QueryParsers may not allow this by default, but
 provide a setAllowLeadingWildcard method to remove that protection. The RegexpQuery is even more general than WildcardQuery,
 allowing an application to identify all documents with terms that match a regular expression
 pattern.
 
FuzzyQuery 
 A FuzzyQuery matches documents that contain terms
 similar to the specified term. Similarity is determined using Levenshtein distance. This type of
 query can be useful when accounting for spelling variations in the collection.
 
Scoring — Introduction
Lucene scoring is the heart of why we all love Lucene. It is blazingly fast and it hides
 almost all of the complexity from the user. In a nutshell, it works. At least, that is, until it
 doesn't work, or doesn't work as one would expect it to work. Then we are left digging into
 Lucene internals or asking for help on java-user@lucene.apache.org to figure out why
 a document with five of our query terms scores lower than a different document with only one of
 the query terms.
 
While this document won't answer your specific scoring issues, it will, hopefully, point you to the places that can help you figure out the what and why of Lucene scoring.
Lucene scoring supports a number of pluggable information retrieval models, including:
These models can be plugged in via theSimilarity
 API, and offer extension hooks and parameters for tuning. In general, Lucene first finds the
 documents that need to be scored based on boolean logic in the Query specification, and then
 ranks this subset of matching documents via the retrieval model. For some valuable references on
 VSM and IR in general refer to Lucene Wiki IR references.
 The rest of this document will cover Scoring basics and explain
 how to change your Similarity. Next, it
 will cover ways you can customize the lucene internals in Custom
 Queries -- Expert Level, which gives details on implementing your own Query class and related functionality. Finally, we will finish up
 with some reference material in the Appendix.
 
Scoring — Basics
Scoring is very much dependent on the way documents are indexed, so it is important to
 understand indexing. (see Lucene
 overview before continuing on with this section) Be sure to use the useful IndexSearcher.explain(Query, doc) to understand how the score for a certain matching document
 was computed.
 
Generally, the Query determines which documents match (a binary decision), while the Similarity determines how to assign scores to the matching documents.
Fields and Documents
In Lucene, the objects we are scoring are Documents. A Document is a collection of Fields. Each
 Field has semantics about how it is created and
 stored (tokenized, stored, etc). It is important to note that Lucene
 scoring works on Fields and then combines the results to return Documents. This is important
 because two Documents with the exact same content, but one having the content in two Fields and
 the other in one Field may return different scores for the same query due to length
 normalization.
 
Score Boosting
Lucene allows influencing the score contribution of various parts of the query by wrapping
 with BoostQuery. 
 
Changing Scoring — Similarity
Changing the scoring formula
Changing Similarity is an easy way to
 influence scoring, this is done at index-time with IndexWriterConfig.setSimilarity(Similarity) and at query-time with IndexSearcher.setSimilarity(Similarity). Be sure to use the same Similarity at query-time as at
 index-time (so that norms are encoded/decoded correctly); Lucene makes no effort to verify this.
 
You can influence scoring by configuring a different built-in Similarity implementation, or by tweaking its parameters, subclassing it to override behavior. Some implementations also offer a modular API which you can extend by plugging in a different component (e.g. term frequency normalizer).
Finally, you can extend the low level Similarity directly to implement a new retrieval model.
 
See the org.apache.lucene.search.similarities package documentation for information on
 the built-in available scoring models and extending or changing Similarity.
 
Integrating field values into the score
While similarities help score a document relatively to a query, it is also common for
 documents to hold features that measure the quality of a match. Such features are best integrated
 into the score by indexing a FeatureField with
 the document at index-time, and then combining the similarity score and the feature score using a
 linear combination. For instance the below query matches the same documents as 
 originalQuery and computes scores as similarityScore + 0.7 * featureScore:
 
 Query originalQuery = new BooleanQuery.Builder()
     .add(new TermQuery(new Term("body", "apache")), Occur.SHOULD)
     .add(new TermQuery(new Term("body", "lucene")), Occur.SHOULD)
     .build();
 Query featureQuery = FeatureField.newSaturationQuery("features", "pagerank");
 Query query = new BooleanQuery.Builder()
     .add(originalQuery, Occur.MUST)
     .add(new BoostQuery(featureQuery, 0.7f), Occur.SHOULD)
     .build();
 
 A less efficient yet more flexible way of modifying scores is to index scoring features into
 doc-value fields and then combine them with the similarity score using a FunctionScoreQuery
 from the queries module. For instance
 the below example shows how to compute scores as similarityScore * Math.log(popularity)
 using the expressions module and
 assuming that values for the popularity field have been set in a NumericDocValuesField at index time:
 
   // compile an expression:
   Expression expr = JavascriptCompiler.compile("_score * ln(popularity)");
   // SimpleBindings just maps variables to DoubleValuesSource instances
   SimpleBindings bindings = new SimpleBindings();
   bindings.add("_score", DoubleValuesSource.SCORES);
   bindings.add("popularity", DoubleValuesSource.fromIntField("popularity"));
   // create a query that matches based on 'originalQuery' but
   // scores using expr
   Query query = new FunctionScoreQuery(
       originalQuery,
       expr.getDoubleValuesSource(bindings));
 
 
 Custom Queries — Expert Level
Custom queries are an expert level task, so tread carefully and be prepared to share your code if you want help.
With the warning out of the way, it is possible to change a lot more than just the Similarity when it comes to matching and scoring in Lucene. Lucene's search is a complex mechanism that is grounded by three main classes:
- Query— The abstract object representation of the user's information need.
- Weight— A specialization of a Query for a given index. This typically associates a Query object with index statistics that are later used to compute document scores.
- Scorer— The core class of the scoring process: for a given segment, scorers return- iteratorsover matches and give a way to compute the- scoreof these matches.
- BulkScorer— An abstract class that scores a range of documents. A default implementation simply iterates through the hits from- Scorer, but some queries such as- BooleanQueryhave more efficient implementations.
The Query Class
In some sense, the Query class is where it all begins.
 Without a Query, there would be nothing to score. Furthermore, the Query class is the catalyst
 for the other scoring classes as it is often responsible for creating them or coordinating the
 functionality between them. The Query class has several
 methods that are important for derived classes:
 
- createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)— A- Weightis the internal representation of the Query, so each Query implementation must provide an implementation of Weight. See the subsection on The Weight Interface below for details on implementing the Weight interface.
- rewrite(IndexSearcher searcher)— Rewrites queries into primitive queries. Primitive queries are:- TermQuery,- BooleanQuery, and other queries that implement- createWeight(IndexSearcher searcher,ScoreMode scoreMode, float boost)
The Weight Interface
The Weight interface provides an internal
 representation of the Query so that it can be reused. Any IndexSearcher dependent state should be stored in the
 Weight implementation, not in the Query class. The interface defines four main methods:
 
- scorer()— Construct a new- Scorerfor this Weight. See The Scorer Class below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents given the Query.
- explain(LeafReaderContext context, int doc)— Provide a means for explaining why a given document was scored the way it was. Typically a weight such as TermWeight that scores via a- Similaritywill make use of the Similarity's implementation:- SimScorer#explain(Explanation freq, long norm).
- matches(LeafReaderContext context, int doc)— Give information about positions and offsets of matches. This is typically useful to implement highlighting.
The Scorer Class
The Scorer abstract class provides common scoring
 functionality for all Scorer implementations and is the heart of the Lucene scoring process. The
 Scorer defines the following methods which must be implemented:
 
- iterator()— Return a- DocIdSetIteratorthat can iterate over all document that matches this Query.
- docID()— Returns the id of the- Documentthat contains the match.
- score()— Return the score of the current document. This value can be determined in any appropriate way for an application. For instance, the- TermScorersimply defers to the configured Similarity:- SimScorer.score(float freq, long norm).
- getChildren()— Returns any child subscorers underneath this scorer. This allows for users to navigate the scorer hierarchy and receive more fine-grained details on the scoring process.
The BulkScorer Class
The BulkScorer scores a range of documents. There
 is only one abstract method:
 
- score(LeafCollector,Bits,int,int)— Score all documents up to but not including the specified max document.
Why would I want to add my own Query?
In a nutshell, you want to add your own custom Query implementation when you think that Lucene's aren't appropriate for the task that you want to do. You might be doing some cutting edge research or you need more information back out of Lucene (similar to Doug adding SpanQuery functionality).
Appendix: Search Algorithm
This section is mostly notes on stepping through the Scoring process and serves as fertilizer for the earlier sections.
In the typical search application, a Query is passed to
 the IndexSearcher, beginning the scoring process.
 
Once inside the IndexSearcher, a Collector is used
 for the scoring and sorting of the search results. These important objects are involved in a
 search:
 
- The Weightobject of the Query. The Weight object is an internal representation of the Query that allows the Query to be reused by the IndexSearcher.
- The IndexSearcher that initiated the call.
- A Sortobject for specifying how to sort the results if the standard score-based sort method is not desired.
Assuming we are not sorting (since sorting doesn't affect the raw Lucene score), we call one
 of the search methods of the IndexSearcher, passing in the Weight object created by IndexSearcher.createWeight(Query,ScoreMode,float) and the number of results we want. This method
 returns a TopDocs object, which is an internal
 collection of search results. The IndexSearcher creates a TopScoreDocCollector and passes it along with the
 Weight to another expert search method (for more on the Collector mechanism, see IndexSearcher). The
 TopScoreDocCollector uses a PriorityQueue to collect
 the top results for the search.
 
At last, we are actually going to score some documents. The score method takes in the
 Collector (most likely the TopScoreDocCollector or TopFieldCollector) and does its business. Of
 course, here is where things get involved. The Scorer
 that is returned by the Weight object depends on what
 type of Query was submitted. In most real world applications with multiple query terms, the
 Scorer is going to be a BooleanScorer2
 created from BooleanWeight (see the section on custom queries for info on changing this).
 
Assuming a BooleanScorer2, we get a internal Scorer based on the required, optional and
 prohibited parts of the query. Using this internal Scorer, the BooleanScorer2 then proceeds into
 a while loop based on the DocIdSetIterator.nextDoc() method. The nextDoc() method advances to the next document matching
 the query. This is an abstract method in the Scorer class and is thus overridden by all derived
 implementations. If you have a simple OR query your internal Scorer is most likely a
 DisjunctionSumScorer, which essentially combines the scorers from the sub scorers of the OR'd
 terms.
- 
Interface Summary Interface Description BoostAttribute Add thisAttributeto aTermsEnumreturned byMultiTermQuery.getTermsEnum(Terms,AttributeSource)and update the boost on each returned term.CheckedIntConsumer<T extends Exception> LikeIntConsumer, but may throw checked exceptions.Collector Expert: Collectors are primarily meant to be used to gather raw results from a search, and implement sorting or custom result filtering, collation, etc.CollectorManager<C extends Collector,T> A manager of collectors.KnnCollector KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the gathered neighborsLeafCollector Collector decouples the score from the collected doc: the score computation is skipped entirely if it's not needed.LeafFieldComparator Expert: comparator that gets instantiated on each leaf from a top-levelFieldComparatorinstance.Matches Reports the positions and optionally offsets of all matching terms in a query for a single documentMatchesIterator An iterator over match positions (and optionally offsets) for a single document and fieldMaxNonCompetitiveBoostAttribute Add thisAttributeto a freshAttributeSourcebefore callingMultiTermQuery.getTermsEnum(Terms,AttributeSource).QueryCache A cache for queries.QueryCachingPolicy A policy defining which filters should be cached.ReferenceManager.RefreshListener Use to receive notification when a refresh has finished.SearcherLifetimeManager.Pruner SegmentCacheable Interface defining whether or not an object can be cached against aLeafReaderVectorScorer Computes the similarity score between a given query vector and different document vectors.
- 
Class Summary Class Description AbstractKnnCollector AbstractKnnCollector is the default implementation for a knn collector used for gathering kNN results and providing topDocs from the gathered neighborsAutomatonQuery AQuerythat will match terms against a finite-state machine.BlendedTermQuery AQuerythat blends index statistics across multiple terms.BlendedTermQuery.Builder A Builder forBlendedTermQuery.BlendedTermQuery.DisjunctionMaxRewrite ABlendedTermQuery.RewriteMethodthat creates aDisjunctionMaxQueryout of the sub queries.BlendedTermQuery.RewriteMethod ABlendedTermQuery.RewriteMethoddefines how queries for individual terms should be merged.BooleanClause A clause in a BooleanQuery.BooleanQuery A Query that matches documents matching boolean combinations of other queries, e.g.BooleanQuery.Builder A builder for boolean queries.BoostAttributeImpl Implementation class forBoostAttribute.BoostQuery AQuerywrapper that allows to give a boost to the wrapped query.BulkScorer This class is used to score a range of documents at once, and is returned byWeight.bulkScorer(org.apache.lucene.index.LeafReaderContext).ByteVectorSimilarityQuery Search for all (approximate) byte vectors above a similarity threshold.CachingCollector Caches all docs, and optionally also scores, coming from a search, and is then able to replay them to another collector.CollectionStatistics Contains statistics for a collection (field).ConjunctionUtils Helper methods for building conjunction iteratorsConstantScoreQuery A query that wraps another query and simply returns a constant score equal to 1 for every document that matches the query.ConstantScoreQuery.ConstantBulkScorer We return this as ourBulkScorerso that if the CSQ wraps a query with its own optimized top-level scorer (e.g.ConstantScoreScorer A constant-scoringScorer.ConstantScoreWeight A Weight that has a constant score equal to the boost of the wrapped query.ControlledRealTimeReopenThread<T> Utility class that runs a thread to manage periodicc reopens of aReferenceManager, with methods to wait for a specific index changes to become visible.DisiPriorityQueue A priority queue of DocIdSetIterators that orders by current doc ID.DisiWrapper Wrapper used inDisiPriorityQueue.DisjunctionDISIApproximation ADocIdSetIteratorwhich is a disjunction of the approximations of the provided iterators.DisjunctionMaxQuery A query that generates the union of documents produced by its subqueries, and that scores each document with the maximum score for that document as produced by any subquery, plus a tie breaking increment for any additional matching subqueries.DocIdSet A DocIdSet contains a set of doc ids.DocIdSetIterator This abstract class defines methods to iterate over a set of non-decreasing doc ids.DocIdStream A stream of doc IDs.DocValuesFieldExistsQuery Deprecated. UseFieldExistsQueryinstead.DocValuesRewriteMethod Rewrites MultiTermQueries into a filter, using DocValues for term enumeration.DoubleValues Per-segment, per-document double values, which can be calculated at search-timeDoubleValuesSource Base class for producingDoubleValuesExactPhraseMatcher Expert: Find exact phrasesExplanation Expert: Describes the score computation for document and query.FieldComparator<T> Expert: a FieldComparator compares hits so as to determine their sort order when collecting the top results withTopFieldCollector.FieldComparator.RelevanceComparator Sorts by descending relevance.FieldComparator.TermValComparator Sorts by field's natural Term sort order.FieldComparatorSource Provides aFieldComparatorfor custom field sorting.FieldDoc Expert: A ScoreDoc which also contains information about how to sort the referenced document.FieldExistsQuery AQuerythat matches documents that contain either aKnnFloatVectorField,KnnByteVectorFieldor a field that indexes norms or doc values.FieldValueHitQueue<T extends FieldValueHitQueue.Entry> Expert: A hit queue for sorting by hits by terms in more than one field.FieldValueHitQueue.Entry Extension of ScoreDoc to also store theFieldComparatorslot.FilterCollector Collectordelegator.FilteredDocIdSetIterator Abstract decorator class of a DocIdSetIterator implementation that provides on-demand filter/validation mechanism on an underlying DocIdSetIterator.FilterLeafCollector LeafCollectordelegator.FilterMatchesIterator A MatchesIterator that delegates all calls to another MatchesIteratorFilterScorable Filter aScorable, intercepting methods and optionally changing their return valuesFilterScorer AFilterScorercontains anotherScorer, which it uses as its basic source of data, possibly transforming the data along the way or providing additional functionality.FilterWeight AFilterWeightcontains anotherWeightand implements all abstract methods by calling the contained weight's method.FloatVectorSimilarityQuery Search for all (approximate) float vectors above a similarity threshold.FuzzyQuery Implements the fuzzy search query.FuzzyTermsEnum Subclass of TermsEnum for enumerating all terms that are similar to the specified filter term.HitQueue Expert: Priority queue containing hit docsImpactsDISI DocIdSetIteratorthat skips non-competitive docs thanks to the indexed impacts.IndexOrDocValuesQuery A query that uses either an index structure (points or terms) or doc values in order to run a query, depending which one is more efficient.IndexSearcher Implements search over a single IndexReader.IndexSearcher.LeafSlice A class holding a subset of theIndexSearchers leaf contexts to be executed within a single thread.IndexSortSortedNumericDocValuesRangeQuery A range query that can take advantage of the fact that the index is sorted to speed up execution.IndriAndQuery A Query that matches documents matching combinations of subqueries.IndriAndScorer Combines scores of subscorers.IndriAndWeight The Weight for IndriAndQuery, used to normalize, score and explain these queries.IndriDisjunctionScorer The Indri implemenation of a disjunction scorer which stores the subscorers for the child queries.IndriQuery A Basic abstract query that all IndriQueries can extend to implement toString, equals, getClauses, and iterator.IndriScorer The Indri parent scorer that stores the boost so that IndriScorers can use the boost outside of the term.KnnByteVectorQuery UsesKnnVectorsReader.search(String, byte[], KnnCollector, Bits)to perform nearest neighbour search.KnnFloatVectorQuery UsesKnnVectorsReader.search(String, float[], KnnCollector, Bits)to perform nearest neighbour search.KnnVectorFieldExistsQuery Deprecated. UseFieldExistsQueryinstead.KnnVectorQuery Deprecated. useKnnFloatVectorQueryinsteadLeafSimScorer Similarity.SimScoreron a specificLeafReader.LiveFieldValues<S,T> Tracks live field values across NRT reader reopens.LongValues Per-segment, per-document long values, which can be calculated at search-timeLongValuesSource Base class for producingLongValuesLongValuesSource.ConstantLongValuesSource A ConstantLongValuesSource that always returns a constant valueLRUQueryCache AQueryCachethat evicts queries using a LRU (least-recently-used) eviction policy in order to remain under a given maximum size and number of bytes used.LRUQueryCache.CacheAndCount Cache of doc ids with a count.MatchAllDocsQuery A query that matches all documents.MatchesUtils Contains static functions that aid the implementation ofMatchesandMatchesIteratorinterfaces.MatchNoDocsQuery A query that matches no documents.MaxNonCompetitiveBoostAttributeImpl Implementation class forMaxNonCompetitiveBoostAttribute.MaxScoreCache Compute maximum scores based onImpactsand keep them in a cache in order not to run expensive similarity score computations multiple times on the same data.MultiCollector MultiCollectorManager ACollectorManagerimplements which wrap a set ofCollectorManagerasMultiCollectoracts forCollector.MultiPhraseQuery A generalized version ofPhraseQuery, with the possibility of adding more than one term at the same position that are treated as a disjunction (OR).MultiPhraseQuery.Builder A builder for multi-phrase queriesMultiPhraseQuery.UnionFullPostingsEnum Slower version of UnionPostingsEnum that delegates offsets and positions, for use by MatchesIteratorMultiPhraseQuery.UnionPostingsEnum Takes the logical union of multiple PostingsEnum iterators.Multiset<T> AMultisetis a set that allows for duplicate elements.MultiTermQuery An abstractQuerythat matches documents containing a subset of terms provided by aFilteredTermsEnumenumeration.MultiTermQuery.RewriteMethod Abstract class that defines how the query is rewritten.MultiTermQuery.TopTermsBlendedFreqScoringRewrite A rewrite method that first translates each term intoBooleanClause.Occur.SHOULDclause in a BooleanQuery, but adjusts the frequencies used for scoring to be blended across the terms, otherwise the rarest term typically ranks highest (often not useful eg in the set of expanded terms in a FuzzyQuery).MultiTermQuery.TopTermsBoostOnlyBooleanQueryRewrite A rewrite method that first translates each term intoBooleanClause.Occur.SHOULDclause in a BooleanQuery, but the scores are only computed as the boost.MultiTermQuery.TopTermsScoringBooleanQueryRewrite A rewrite method that first translates each term intoBooleanClause.Occur.SHOULDclause in a BooleanQuery, and keeps the scores as computed by the query.NamedMatches Utility class to help extract the set of sub queries that have matched from a larger query.NGramPhraseQuery This is aPhraseQuerywhich is optimized for n-gram phrase query.NormsFieldExistsQuery Deprecated. UseFieldExistsQueryinstead.PhraseMatcher Base class for exact and sloppy phrase matchingPhraseQuery A Query that matches documents containing a particular sequence of terms.PhraseQuery.Builder A builder for phrase queries.PhraseQuery.PostingsAndFreq Term postings and position information for phrase matchingPhraseWeight Expert: Weight class for phrase matchingPointInSetQuery Abstract query class to find all documents whose single or multi-dimensional point values, previously indexed with e.g.PointInSetQuery.Stream Iterator of encoded point values.PointRangeQuery Abstract class for range queries against single or multidimensional points such asIntPoint.PositiveScoresOnlyCollector PrefixQuery A Query that matches documents containing terms with a specified prefix.Query The abstract base class for queries.QueryRescorer ARescorerthat uses a provided Query to assign scores to the first-pass hits.QueryVisitor Allows recursion through a query treeReferenceManager<G> Utility class to safely share instances of a certain type across multiple threads, while periodically refreshing them.RegexpQuery A fast regular expression query based on theorg.apache.lucene.util.automatonpackage.Rescorer Re-scores the topN results (TopDocs) from an original query.Scorable Allows access to the score of a QueryScorable.ChildScorable A child Scorer and its relationship to its parent.ScoreCachingWrappingScorer AScorerwhich wraps another scorer and caches the score of the current document.ScoreDoc Holds one hit inTopDocs.Scorer Expert: Common scoring functionality for different types of queries.ScorerSupplier A supplier ofScorer.ScoringRewrite<B> Base rewrite method that translates each term into a query, and keeps the scores as computed by the query.SearcherFactory Factory class used bySearcherManagerto create new IndexSearchers.SearcherLifetimeManager Keeps track of current plus old IndexSearchers, closing the old ones once they have timed out.SearcherLifetimeManager.PruneByAge Simple pruner that drops any searcher older by more than the specified seconds, than the newest searcher.SearcherManager Utility class to safely shareIndexSearcherinstances across multiple threads, while periodically reopening.SimpleCollector BaseCollectorimplementation that is used to collect all contexts.SimpleFieldComparator<T> BaseFieldComparatorimplementation that is used for all contexts.SloppyPhraseMatcher Find all slop-valid position-combinations (matches) encountered while traversing/hopping the PhrasePositions.Sort Encapsulates sort criteria for returned hits.SortedNumericSelector Selects a value from the document's list to use as the representative valueSortedNumericSortField SortField forSortedNumericDocValues.SortedNumericSortField.Provider A SortFieldProvider for this sort fieldSortedSetSelector Selects a value from the document's set to use as the representative valueSortedSetSortField SortField forSortedSetDocValues.SortedSetSortField.Provider A SortFieldProvider for this sortSortField Stores information about how to sort documents by terms in an individual field.SortField.Provider A SortFieldProvider for field sortsSortRescorer ARescorerthat re-sorts according to a provided Sort.SynonymQuery A query that treats multiple terms as synonyms.SynonymQuery.Builder A builder forSynonymQuery.TaskExecutor Executor wrapper responsible for the execution of concurrent tasks.TermInSetQuery Specialization for a disjunction over many terms that, by default, behaves like aConstantScoreQueryover aBooleanQuerycontaining onlyBooleanClause.Occur.SHOULDclauses.TermQuery A Query that matches documents containing a term.TermRangeQuery A Query that matches documents within an range of terms.TermScorer Expert: AScorerfor documents matching aTerm.TermStatistics Contains statistics for a specific termTimeLimitingCollector Deprecated. UseIndexSearcher.setTimeout(QueryTimeout)to time out queries.TimeLimitingCollector.TimerThread Thread used to timeout search requests.TimeLimitingKnnCollectorManager AKnnCollectorManagerthat collects results with a timeout.TopDocs Represents hits returned byIndexSearcher.search(Query,int).TopDocsCollector<T extends ScoreDoc> A base class for all collectors that return aTopDocsoutput.TopFieldCollector TopFieldCollectorManager Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and a sharedMaxScoreAccumulatorto propagate the minimum score across segments if the primary sort is by relevancy.TopFieldDocs Represents hits returned byIndexSearcher.search(Query,int,Sort).TopKnnCollector TopKnnCollector is a specific KnnCollector.TopScoreDocCollector TopScoreDocCollector.ScorerLeafCollector Scorable leaf collectorTopScoreDocCollectorManager Create a TopScoreDocCollectorManager which uses a shared hit counter to maintain number of hits and a sharedMaxScoreAccumulatorto propagate the minimum score across segmentsTopTermsRewrite<B> Base rewrite method for collecting only the top terms via a priority queue.TotalHitCountCollector Just counts the total number of hits.TotalHitCountCollectorManager Collector manager based onTotalHitCountCollectorthat allows users to parallelize counting the number of hits, expected to be used mostly wrapped inMultiCollectorManager.TotalHits Description of the total number of hits of a query.TwoPhaseIterator Returned byScorer.twoPhaseIterator()to expose an approximation of aDocIdSetIterator.UsageTrackingQueryCachingPolicy AQueryCachingPolicythat tracks usage statistics of recently-used filters in order to decide on which filters are worth caching.Weight Expert: Calculate query weights and build query scorers.Weight.DefaultBulkScorer Just wraps a Scorer and performs top scoring using it.WildcardQuery Implements the wildcard search query.
- 
Enum Summary Enum Description BooleanClause.Occur Specifies how clauses are to occur in matching documents.Pruning ControlsLeafFieldComparatorhow to skip documentsScoreMode Different modes of search.SortedNumericSelector.Type Type of selection to perform.SortedSetSelector.Type Type of selection to perform.SortField.Type Specifies the type of the terms to be sorted, or special types such as CUSTOMTotalHits.Relation How theTotalHits.valueshould be interpreted.
- 
Exception Summary Exception Description BooleanQuery.TooManyClauses Deprecated. CollectionTerminatedException Throw this exception inLeafCollector.collect(int)to prematurely terminate collection of the current leaf.FuzzyTermsEnum.FuzzyTermsException Thrown to indicate that there was an issue creating a fuzzy query for a given term.IndexSearcher.TooManyClauses Thrown when an attempt is made to add more thanIndexSearcher.TooManyClauses.getMaxClauseCount()clauses.IndexSearcher.TooManyNestedClauses Thrown when a client attempts to execute a Query that has more thanIndexSearcher.TooManyClauses.getMaxClauseCount()total clauses cumulatively in all of its children.TimeLimitingCollector.TimeExceededException Thrown when elapsed search time exceeds allowed search time.