1. I believe that this would require pulling a ScorerSupplier twice for the same segment, which is a costly operation.
2. The cost is computed in order to know whether the top-level query is likely to consume a large-enough portion of the matches of the query that we are considering caching so that caching this query wouldn't hurt latency too much. Making a bad decision here because the cost is unknown would lead to a worse situation than computing the cost on every query that we are considering caching. On both of these questions, I feel like I may be missing the point about the suggestion you are making so feel free to show a simple code change that could help me understand the change that you are suggesting. On Thu, Jul 14, 2022 at 12:26 PM Mohammad Sadiq <sadiqli...@gmail.com> wrote: > Thanks for the deep-dive Shradha. Thank you Adrien for the additional > questions and answers. > > I had a couple of questions, when looking around the cache code. > > 1. The `QueryCachingPolicy` [1] makes decisions based on `Query`. Why not > use `Weight`? > The `scorerSupplier` [2] in the `LRUQueryCache` decides whether something > should be cached by determining the cost [3] using the `Weight`. IIUC, this > was introduced because “Query caching leads to absurdly slow queries” [4]. > What if the `QueryCachingPolicy` was called with the `Weight` instead? > Since the `Query` can be obtained from the `Weight`, we can have all such > caching decisions in the policy, and de-couple that decision from the > `LRUQueryCache` class. What do you think? > > 2. Why do we invoke a possibly costly `cost()` method for every cache > addition? > During the above cost computation, we call the `supplier.cost()` method; > but its documentation [5] states that it “may be a costly operation, so it > should only be called if necessary". > This means that we’re including a (possibly) costly operation for every > cache addition. If we de-couple these, then, for cases where the cache > addition is expensive, we can use the call to `cost`, but for other cases, > we can avoid this expensive call. > > If you, or the community thinks that this is a good idea, then I can open > a JIRA, and submit a PR. > > References: > [1] > https://github.com/apache/lucene/blob/d5d6dc079395c47cd6d12dcce3bcfdd2c7d9dc63/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java > < > https://github.com/apache/lucene/blob/d5d6dc079395c47cd6d12dcce3bcfdd2c7d9dc63/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java > > > [2] > https://github.com/apache/lucene/blob/941df98c3f718371af4702c92bf6537739120064/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java#L725 > < > https://github.com/apache/lucene/blob/941df98c3f718371af4702c92bf6537739120064/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java#L725 > > > [3] > https://github.com/apache/lucene/blob/941df98c3f718371af4702c92bf6537739120064/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java#L767 > < > https://github.com/apache/lucene/blob/941df98c3f718371af4702c92bf6537739120064/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java#L767 > > > [4] https://github.com/apache/lucene-solr/pull/940/files < > https://github.com/apache/lucene-solr/pull/940/files> > [5] > https://github.com/apache/lucene/blob/d5d6dc079395c47cd6d12dcce3bcfdd2c7d9dc63/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java#L39-L40 > < > https://github.com/apache/lucene/blob/d5d6dc079395c47cd6d12dcce3bcfdd2c7d9dc63/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java#L39-L40 > > > > > Regards, > Mohammad Sadiq > > > > On 11 Jul 2022, at 10:37, Adrien Grand <jpou...@gmail.com> wrote: > > > > Hey Shradha, > > > > This correctly describes the what, but I think it could add more color > > about why the cache behaves this way to be more useful, e.g. > > - Why doesn't the cache cache all queries? Lucene is relatively good at > > evaluating a subset of the matching documents, e.g. queries sorted by > > numeric field can leverage point index structures to only look at a small > > subset of the matching docs. Yet caching a query requires consuming all > its > > matches, so it could significantly hurt latencies. It's important to not > > cache all queries to preserve the benefit of Lucene's filtering and > dynamic > > pruning capabilities. > > - A corollary of the above is that the query cache is essentially a way > to > > trade latency for throughput. Use-cases that care more about latency than > > throughput may want to disable the cache entirely. > > - LRUQueryCache takes a `skipCacheFactor` which aims at limiting the > > impact of query caching on latency by not caching clauses whose cost is > > much higher than the overall query. It only helps for filters within > > conjunctions though, not in the dynamic pruning case when we don't know > how > > many matches are going to be consumed. > > - Why are small segments never cached? Small segments are likely going to > > be merged soon, so it would be wasteful to build cache entries that would > > get evicted shortly. > > - The queries that never get cached don't get cached because a cached > > entry wouldn't perform faster than their uncached counterpart. An > inverted > > index is already a cache of the matches for every unique term of the > index. > > > > On Fri, Jul 8, 2022 at 3:20 PM Shradha Shankar <shrdsha2...@gmail.com> > > wrote: > > > >> Hello! > >> > >> I work at Amazon Product Search and I’ve recently been looking into > >> understanding how Lucene’s LRU Query Cache works. I’ve written up a > summary > >> of my understanding here. (Also attached as a markdown file with this > email) > >> > >> Will really appreciate feedback/improvements/corrections for my > >> understanding and if this is worthy of contributing to the documentation > >> for LRU QueryCache. :) > >> > >> ================================= > >> A brief overview of Lucene's Query Cache > >> > >> We first get acquainted with Lucene’s caching at IndexSearcher’s > >> createWeight method which is called for every query (and consequently > >> sub-queries within that query, eg see BooleanWeight) before we can > actually > >> find matching documents in our index and score them. Weight is really > >> another representation of a query that is specific to the statistics of > the > >> IndexSearcher being used. This definition makes it easier to see why > >> caching logic would start while creating weight for the query - we want > to > >> make a weight that will be responsible for caching matching docs per > >> segment. Since segments are specific to the IndexReader being used by > the > >> IndexSearcher, they are transitively, specific to the IndexSearcher. > >> > >> QueryCache in Lucene is an interface that has the signature for just one > >> method - doCache. doCache takes in a Weight (weight of the query in > >> question eg: TermWeight for a TermQuery) and operates on it based on the > >> rules defined by QueryCachingPolicy (yet another interface that defines > two > >> methods - onUse and shouldCache) to return a Weight. This “new” returned > >> Weight possibly wraps the original weight and bestows upon it some > caching > >> abilities. > >> > >> As of now, Lucene has one core implementation of the QueryCache and the > >> QueryCachingPolicy. All IndexSearcher instances have a default query > cache > >> - LRUQueryCache and use the default policy - > >> UsageTrackingQueryCachingPolicy.In the IndexSearcher’s createWeight > method, > >> we first create a weight for the incoming query and then subject it to > the > >> LRUQueryCache’s doCache method. An important thing to note here is that > we > >> only call doCache when the score mode passed to the search method does > not > >> need scores. Calling doCache does nothing complex; it just returns the > >> input weight wrapped as a CachingWrapperWeight that encapsulates the > >> caching policy information. No real caching has happened, yet! > >> > >> After getting a weight from the createWeight method, the IndexSearcher > >> iterates over each leaf and uses the weight to create a BulkScorer. A > >> BulkScorer, as the name suggests, is used to score a range of the > documents > >> - generally the range being all the matches found in that leaf. Given > >> context information for a leaf, every weight should know how to create a > >> bulk scorer for that leaf. In our case, the CachingWrapperWeight’s > >> BulkScorer method does a little bit extra and this is where the actual > >> caching happens! > >> > >> A brief dive into the query caching policy: While LRU says what we want > to > >> evict from a full cache, using query caching policies we can define > other > >> rules to use in conjunction with the cache’s design policy. The default > >> UsageTrackingQueryCachingPolicy dictates what queries will be put into > the > >> cache. This policy uses a ring buffer data structure optimised to track > and > >> retrieve frequencies for a given query. The policy also defines some > >> queries that will not be cached (TermQuery, MatchAllDocsQuery, > >> MatchNoDocsQuery to name a few). The OnUse method on this policy checks > if > >> the incoming query is something it would like to cache and if so, it > >> registers the frequency for this query. The > UsageTrackingQueryCachingPolicy > >> defines minimum frequencies for queries - 2 for costly queries and 5 for > >> others. When a query that can be cached and has been seen more than > minimum > >> number of times, the shouldCache method of the policy gives it a green > >> light for caching. > >> > >> A brief dive into LRUQueryCache: There are many attributes in this class > >> that track ram usage and allowed max memory size for the cache that play > >> important roles but, for easy understanding, we want to highlight two > >> attributes - Map<IndexReader.CacheKey, LeafCache> cache and Map<Query, > >> Query> uniqueQueries. uniqueQueries: A LinkedHashMap with capacity and > >> accessOrder set to true behaves like an LRU cache and uniqueQueries is > just > >> that. While the keys in uniqueQueries are Query instances, the values > are > >> also Query instances and not DocIdSet corresponding to the matches as we > >> might expect. This LRU cache is common across all segments and the > purpose > >> of this cache (map) is to store a singleton corresponding to the most > >> recently seen queries so that we don’t need to store multiple copies of > the > >> same query. cache: The cache maps a LeafCache instance for each leaf in > the > >> index represented by a CacheKey. The LeafCache isn’t a cache but is a > >> Map<Query, CacheAndCount> where Query corresponds to the a cached Query > >> instance from uniqueQueries and CacheAndCount is the actual data > structure > >> that stores the DocIdSet for the matches found for the query in that > leaf. > >> > >> Returning to the creation of the BulkScorer from the > CachingWrapperWeight, > >> we first register the query (and hence its frequency) on the policy via > the > >> onUse method. At present, Lucene only wants to cache queries on leaves > >> which have more than 10k documents and have more than 3% of the total > >> number of documents in the index. Once we determine that the segment is > >> eligible for caching, we use a CacheHelper to get the CacheKey for the > >> segment so that we can access the LeafCache corresponding to it. We > check > >> if the query has a CacheAndCount entry in the LeafCache and return it (a > >> cache hit!) or return null (a cache miss). In the case of a cache miss, > we > >> use the policy’s shouldCache to determine if this query is > cache-eligible > >> and if it is, we create a CacheAndCount instance for the query-leaf > pair. > >> We add the Query to uniqueQueries and query and CacheAndCount instance > to > >> the corresponding LeafCache for that leaf. We’ve finally cached a query > and > >> its match set! > >> > >> When creating a CacheAndCount instance for a query that is being newly > >> cached, we want to find the matches for that query. We use the actual > >> weight for that query (the one wrapped inside the CachingWrapperWeight) > to > >> create a BulkScorer and then call score on that scorer to populate a > >> LeafCollector with the matches for the query in this leaf. Based on the > >> density of the result set, we create a DocIdSet over a FixedBitSet or a > >> Roaring bit set. > >> > >> Finally, we create a DocIdSetIterator from the retrieved or created > >> CacheAndCount instance and return the most commonly used BulkScorer - > the > >> DefaultBulkScorer with a ConstantScoreScorer (score set to 0) over the > >> DocIdSetIterator. When there is a cache miss and the query is ineligible > >> for caching, we return the BulkScorer from the actual weight wrapped > inside > >> the CachingWrapperWeight, as we would if no caching logic were present! > >> > >> Thanks, > >> Shradha > >> > >> --------------------------------------------------------------------- > >> To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org > >> For additional commands, e-mail: java-user-h...@lucene.apache.org > > > > > > > > -- > > Adrien > > -- Adrien