Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 59157200D5A for ; Wed, 29 Nov 2017 18:12:07 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 57CD5160C04; Wed, 29 Nov 2017 17:12:07 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id A41ED160C02 for ; Wed, 29 Nov 2017 18:12:05 +0100 (CET) Received: (qmail 7816 invoked by uid 500); 29 Nov 2017 17:12:04 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 7619 invoked by uid 99); 29 Nov 2017 17:12:04 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 29 Nov 2017 17:12:04 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 28C81E0612; Wed, 29 Nov 2017 17:12:04 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jpountz@apache.org To: commits@lucene.apache.org Date: Wed, 29 Nov 2017 17:12:06 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [3/3] lucene-solr:master: LUCENE-8008: Reduce leniency in CheckHits. archived-at: Wed, 29 Nov 2017 17:12:07 -0000 LUCENE-8008: Reduce leniency in CheckHits. Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d27ddcb4 Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d27ddcb4 Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d27ddcb4 Branch: refs/heads/master Commit: d27ddcb409ddf5d388734db2428ac60edc22180a Parents: 6d34f23 Author: Adrien Grand Authored: Wed Nov 29 16:02:04 2017 +0100 Committer: Adrien Grand Committed: Wed Nov 29 18:09:38 2017 +0100 ---------------------------------------------------------------------- .../org/apache/lucene/search/BooleanWeight.java | 13 ++- .../lucene/search/DisjunctionMaxQuery.java | 5 +- .../lucene/search/DisjunctionMaxScorer.java | 4 +- .../org/apache/lucene/search/LRUQueryCache.java | 51 ++++++++++-- .../search/TestBooleanMinShouldMatch.java | 2 +- .../apache/lucene/search/TestLRUQueryCache.java | 88 ++++++++++++-------- .../lucene/search/TestMinShouldMatch2.java | 4 +- .../TestUsageTrackingFilterCachingPolicy.java | 2 +- .../lucene/expressions/TestDemoExpressions.java | 11 ++- .../lucene/queries/TestCustomScoreQuery.java | 24 +++--- .../org/apache/lucene/search/CheckHits.java | 45 ++++------ .../org/apache/lucene/util/LuceneTestCase.java | 2 +- 12 files changed, 151 insertions(+), 100 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java index dbe3d17..ed729c7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -71,7 +71,6 @@ final class BooleanWeight extends Weight { public Explanation explain(LeafReaderContext context, int doc) throws IOException { final int minShouldMatch = query.getMinimumNumberShouldMatch(); List subs = new ArrayList<>(); - float sum = 0.0f; boolean fail = false; int matchCount = 0; int shouldMatchCount = 0; @@ -83,7 +82,6 @@ final class BooleanWeight extends Weight { if (e.isMatch()) { if (c.isScoring()) { subs.add(e); - sum += e.getValue(); } else if (c.isRequired()) { subs.add(Explanation.match(0f, "match on required clause, product of:", Explanation.match(0f, Occur.FILTER + " clause"), e)); @@ -109,8 +107,15 @@ final class BooleanWeight extends Weight { } else if (shouldMatchCount < minShouldMatch) { return Explanation.noMatch("Failure to match minimum number of optional clauses: " + minShouldMatch, subs); } else { - // we have a match - return Explanation.match(sum, "sum of:", subs); + // Replicating the same floating-point errors as the scorer does is quite + // complex (essentially because of how ReqOptSumScorer casts intermediate + // contributions to the score to floats), so in order to make sure that + // explanations have the same value as the score, we pull a scorer and + // use it to compute the score. + Scorer scorer = scorer(context); + int advanced = scorer.iterator().advance(doc); + assert advanced == doc; + return Explanation.match(scorer.score(), "sum of:", subs); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java index 13237a2..d3fee9d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java @@ -156,7 +156,8 @@ public final class DisjunctionMaxQuery extends Query implements Iterable @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { boolean match = false; - float max = Float.NEGATIVE_INFINITY, sum = 0.0f; + float max = Float.NEGATIVE_INFINITY; + double sum = 0; List subs = new ArrayList<>(); for (Weight wt : weights) { Explanation e = wt.explain(context, doc); @@ -168,7 +169,7 @@ public final class DisjunctionMaxQuery extends Query implements Iterable } } if (match) { - final float score = max + (sum - max) * tieBreakerMultiplier; + final float score = (float) (max + (sum - max) * tieBreakerMultiplier); final String desc = tieBreakerMultiplier == 0.0f ? "max of:" : "max plus " + tieBreakerMultiplier + " times others of:"; return Explanation.match(score, desc, subs); } else { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java index 2356f5e..961fd43 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java @@ -47,7 +47,7 @@ final class DisjunctionMaxScorer extends DisjunctionScorer { @Override protected float score(DisiWrapper topList) throws IOException { - float scoreSum = 0; + double scoreSum = 0; float scoreMax = Float.NEGATIVE_INFINITY; for (DisiWrapper w = topList; w != null; w = w.next) { final float subScore = w.scorer.score(); @@ -56,6 +56,6 @@ final class DisjunctionMaxScorer extends DisjunctionScorer { scoreMax = subScore; } } - return scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier; + return (float) (scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index 5c828d6..beb73ad 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -102,6 +102,7 @@ public class LRUQueryCache implements QueryCache, Accountable { private final int maxSize; private final long maxRamBytesUsed; private final Predicate leavesToCache; + private final float maxCostFactor; // maps queries that are contained in the cache to a singleton so that this // cache does not store several copies of the same query private final Map uniqueQueries; @@ -123,10 +124,14 @@ public class LRUQueryCache implements QueryCache, Accountable { /** * Expert: Create a new instance that will cache at most maxSize * queries with at most maxRamBytesUsed bytes of memory, only on - * leaves that satisfy {@code leavesToCache}; + * leaves that satisfy {@code leavesToCache}. Also, only clauses whose cost is + * no more than {@code maxCostFactor} times the cost of the top-level query + * will be cached in order to not slow down queries too much due to caching. + * Pass {@link Float#POSITIVE_INFINITY} to cache regardless of costs. */ public LRUQueryCache(int maxSize, long maxRamBytesUsed, - Predicate leavesToCache) { + Predicate leavesToCache, + float maxCostFactor) { this.maxSize = maxSize; this.maxRamBytesUsed = maxRamBytesUsed; this.leavesToCache = leavesToCache; @@ -135,6 +140,10 @@ public class LRUQueryCache implements QueryCache, Accountable { cache = new IdentityHashMap<>(); lock = new ReentrantLock(); ramBytesUsed = 0; + if (maxCostFactor < 1) { + throw new IllegalArgumentException("maxCostFactor must be no less than 1, got " + maxCostFactor); + } + this.maxCostFactor = maxCostFactor; } /** @@ -147,9 +156,11 @@ public class LRUQueryCache implements QueryCache, Accountable { * 33 leaves can make it to the cache (very likely less than 10 in * practice), which is useful for this implementation since some operations * perform in linear time with the number of cached leaves. + * Only clauses whose cost is at most 100x the cost of the top-level query will + * be cached in order to not hurt latency too much because of caching. */ public LRUQueryCache(int maxSize, long maxRamBytesUsed) { - this(maxSize, maxRamBytesUsed, new MinSegmentSizePredicate(10000, .03f)); + this(maxSize, maxRamBytesUsed, new MinSegmentSizePredicate(10000, .03f), 100); } // pkg-private for testing @@ -738,11 +749,39 @@ public class LRUQueryCache implements QueryCache, Accountable { } if (docIdSet == null) { + ScorerSupplier inSupplier = in.scorerSupplier(context); + if (inSupplier == null) { + putIfAbsent(in.getQuery(), context, DocIdSet.EMPTY, cacheHelper); + return null; + } + if (policy.shouldCache(in.getQuery())) { - docIdSet = cache(context); - putIfAbsent(in.getQuery(), context, docIdSet, cacheHelper); + return new ScorerSupplier() { + + @Override + public Scorer get(long leadCost) throws IOException { + double costFactor = (double) inSupplier.cost() / leadCost; + if (costFactor >= maxCostFactor) { + // too costly, caching might make the query much slower + return inSupplier.get(leadCost); + } + DocIdSet cached = cacheImpl(new DefaultBulkScorer(inSupplier.get(Long.MAX_VALUE)), context.reader().maxDoc()); + putIfAbsent(in.getQuery(), context, cached, cacheHelper); + DocIdSetIterator iterator = cached.iterator(); + if (iterator == null) { + // DIS.iterator() is allowed to return null when empty but we want a non-null iterator here + iterator = DocIdSetIterator.empty(); + } + return new ConstantScoreScorer(CachingWrapperWeight.this, 0f, iterator); + } + + @Override + public long cost() { + return inSupplier.cost(); + } + }; } else { - return in.scorerSupplier(context); + return inSupplier; } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java index 1264b58..26ce9c7 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java @@ -380,7 +380,7 @@ public class TestBooleanMinShouldMatch extends LuceneTestCase { + CheckHits.topdocsString(top1,0,0) + CheckHits.topdocsString(top2,0,0) + "for query:" + q.toString(), - score, otherScore, CheckHits.explainToleranceDelta(score, otherScore)); + score, otherScore, 0d); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java index fa9743d..b777286 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java @@ -78,7 +78,7 @@ public class TestLRUQueryCache extends LuceneTestCase { }; public void testConcurrency() throws Throwable { - final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000), context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000), context -> random().nextBoolean(), Float.POSITIVE_INFINITY); Directory dir = newDirectory(); final RandomIndexWriter w = new RandomIndexWriter(random(), dir); final SearcherFactory searcherFactory = new SearcherFactory() { @@ -181,7 +181,7 @@ public class TestLRUQueryCache extends LuceneTestCase { w.addDocument(doc); final DirectoryReader reader = w.getReader(); final IndexSearcher searcher = newSearcher(reader); - final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true, Float.POSITIVE_INFINITY); final Query blue = new TermQuery(new Term("color", "blue")); final Query red = new TermQuery(new Term("color", "red")); @@ -242,7 +242,7 @@ public class TestLRUQueryCache extends LuceneTestCase { // different instance yet equal final Query query2 = new TermQuery(new Term("color", "blue")); - final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE, context -> true, Float.POSITIVE_INFINITY); searcher.setQueryCache(queryCache); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); @@ -264,7 +264,7 @@ public class TestLRUQueryCache extends LuceneTestCase { public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException { assumeFalse("LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)", Constants.JRE_IS_MINIMUM_JAVA9); - final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000), context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000), context -> random().nextBoolean(), Float.POSITIVE_INFINITY); // an accumulator that only sums up memory usage of referenced filters and doc id sets final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() { @Override @@ -385,7 +385,7 @@ public class TestLRUQueryCache extends LuceneTestCase { public void testRamBytesUsedConstantEntryOverhead() throws IOException { assumeFalse("LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)", Constants.JRE_IS_MINIMUM_JAVA9); - final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true, Float.POSITIVE_INFINITY); final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() { @Override @@ -434,7 +434,7 @@ public class TestLRUQueryCache extends LuceneTestCase { } public void testOnUse() throws IOException { - final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000), context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000), context -> random().nextBoolean(), Float.POSITIVE_INFINITY); Directory dir = newDirectory(); final RandomIndexWriter w = new RandomIndexWriter(random(), dir); @@ -494,7 +494,7 @@ public class TestLRUQueryCache extends LuceneTestCase { } public void testStats() throws IOException { - final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000, context -> true, Float.POSITIVE_INFINITY); Directory dir = newDirectory(); final RandomIndexWriter w = new RandomIndexWriter(random(), dir); @@ -625,7 +625,7 @@ public class TestLRUQueryCache extends LuceneTestCase { final AtomicLong ramBytesUsage = new AtomicLong(); final AtomicLong cacheSize = new AtomicLong(); - final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000, context -> true) { + final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000, context -> true, Float.POSITIVE_INFINITY) { @Override protected void onHit(Object readerCoreKey, Query query) { super.onHit(readerCoreKey, query); @@ -754,7 +754,7 @@ public class TestLRUQueryCache extends LuceneTestCase { final BooleanQuery.Builder query = new BooleanQuery.Builder(); query.add(new BoostQuery(expectedCacheKey, 42f), Occur.MUST); - final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> random().nextBoolean(), Float.POSITIVE_INFINITY); Directory dir = newDirectory(); final RandomIndexWriter w = new RandomIndexWriter(random(), dir); Document doc = new Document(); @@ -794,12 +794,11 @@ public class TestLRUQueryCache extends LuceneTestCase { doc.add(new StringField("foo", "bar", Store.YES)); doc.add(new StringField("foo", "quux", Store.YES)); w.addDocument(doc); - w.commit(); final IndexReader reader = w.getReader(); final IndexSearcher searcher = newSearcher(reader); w.close(); - final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true, Float.POSITIVE_INFINITY); searcher.setQueryCache(queryCache); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); @@ -903,7 +902,7 @@ public class TestLRUQueryCache extends LuceneTestCase { iters = atLeast(2000); } - final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed, context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed, context -> random().nextBoolean(), Float.POSITIVE_INFINITY); IndexSearcher uncachedSearcher = null; IndexSearcher cachedSearcher = null; @@ -980,7 +979,7 @@ public class TestLRUQueryCache extends LuceneTestCase { IndexReader reader = w.getReader(); // size of 1 so that 2nd query evicts from the cache - final LRUQueryCache queryCache = new LRUQueryCache(1, 10000, context -> true); + final LRUQueryCache queryCache = new LRUQueryCache(1, 10000, context -> true, Float.POSITIVE_INFINITY); final IndexSearcher searcher = newSearcher(reader); searcher.setQueryCache(queryCache); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); @@ -1014,7 +1013,7 @@ public class TestLRUQueryCache extends LuceneTestCase { IndexReader reader = w.getReader(); // size of 1 byte - final LRUQueryCache queryCache = new LRUQueryCache(1, 1, context -> random().nextBoolean()); + final LRUQueryCache queryCache = new LRUQueryCache(1, 1, context -> random().nextBoolean(), Float.POSITIVE_INFINITY); final IndexSearcher searcher = newSearcher(reader); searcher.setQueryCache(queryCache); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); @@ -1053,7 +1052,7 @@ public class TestLRUQueryCache extends LuceneTestCase { try (final IndexReader indexReader = DirectoryReader.open(directory)) { final FrequencyCountingPolicy policy = new FrequencyCountingPolicy(); final IndexSearcher indexSearcher = new IndexSearcher(indexReader); - indexSearcher.setQueryCache(new LRUQueryCache(100, 10240, context -> random().nextBoolean())); + indexSearcher.setQueryCache(new LRUQueryCache(100, 10240, context -> random().nextBoolean(), Float.POSITIVE_INFINITY)); indexSearcher.setQueryCachingPolicy(policy); final Query foo = new TermQuery(new Term("f", "foo")); final Query bar = new TermQuery(new Term("f", "bar")); @@ -1137,7 +1136,7 @@ public class TestLRUQueryCache extends LuceneTestCase { LeafReaderContext leaf = searcher.getIndexReader().leaves().get(0); AtomicBoolean scorerCalled = new AtomicBoolean(); AtomicBoolean bulkScorerCalled = new AtomicBoolean(); - LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE, context -> true); + LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE, context -> true, Float.POSITIVE_INFINITY); // test that the bulk scorer is propagated when a scorer should not be cached Weight weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false); @@ -1148,16 +1147,6 @@ public class TestLRUQueryCache extends LuceneTestCase { assertEquals(false, scorerCalled.get()); assertEquals(0, cache.getCacheCount()); - // test that the doc id set is computed using the bulk scorer - bulkScorerCalled.set(false); - weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false); - weight = new WeightWrapper(weight, scorerCalled, bulkScorerCalled); - weight = cache.doCache(weight, QueryCachingPolicy.ALWAYS_CACHE); - weight.scorer(leaf); - assertEquals(true, bulkScorerCalled.get()); - assertEquals(false, scorerCalled.get()); - assertEquals(1, cache.getCacheCount()); - searcher.getIndexReader().close(); dir.close(); } @@ -1168,7 +1157,7 @@ public class TestLRUQueryCache extends LuceneTestCase { w.addDocument(new Document()); final DirectoryReader reader = w.getReader(); final IndexSearcher searcher = newSearcher(reader); - final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true) { + final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true, Float.POSITIVE_INFINITY) { @Override protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) { super.onDocIdSetEviction(readerCoreKey, numEntries, sumRamBytesUsed); @@ -1198,17 +1187,17 @@ public class TestLRUQueryCache extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); - LRUQueryCache cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(2, 0f)); + LRUQueryCache cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(2, 0f), Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); searcher.count(new DummyQuery()); assertEquals(0, cache.getCacheCount()); - cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(1, 0f)); + cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(1, 0f), Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); searcher.count(new DummyQuery()); assertEquals(1, cache.getCacheCount()); - cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f)); + cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f), Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); searcher.count(new DummyQuery()); assertEquals(1, cache.getCacheCount()); @@ -1218,7 +1207,7 @@ public class TestLRUQueryCache extends LuceneTestCase { reader = w.getReader(); searcher = newSearcher(reader); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); - cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f)); + cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f), Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); searcher.count(new DummyQuery()); assertEquals(0, cache.getCacheCount()); @@ -1271,7 +1260,7 @@ public class TestLRUQueryCache extends LuceneTestCase { // don't cache if the reader does not expose a cache helper assertNull(reader.leaves().get(0).reader().getCoreCacheHelper()); - LRUQueryCache cache = new LRUQueryCache(2, 10000, context -> true); + LRUQueryCache cache = new LRUQueryCache(2, 10000, context -> true, Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); assertEquals(0, searcher.count(new DummyQuery())); assertEquals(0, cache.getCacheCount()); @@ -1333,7 +1322,7 @@ public class TestLRUQueryCache extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); - LRUQueryCache cache = new LRUQueryCache(2, 10000, context -> true); + LRUQueryCache cache = new LRUQueryCache(2, 10000, context -> true, Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); assertEquals(0, searcher.count(new NoCacheQuery())); @@ -1489,7 +1478,7 @@ public class TestLRUQueryCache extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader); searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); - LRUQueryCache cache = new LRUQueryCache(1, 10000, context -> true); + LRUQueryCache cache = new LRUQueryCache(1, 10000, context -> true, Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); DVCacheQuery query = new DVCacheQuery("field"); @@ -1539,4 +1528,35 @@ public class TestLRUQueryCache extends LuceneTestCase { dir.close(); } + + public void testSkipCostlyQueries() throws IOException { + Directory dir = newDirectory(); + final RandomIndexWriter w = new RandomIndexWriter(random(), dir); + Document doc1 = new Document(); + doc1.add(new StringField("foo", "bar", Store.YES)); + doc1.add(new StringField("foo", "quux", Store.YES)); + Document doc2 = new Document(); + doc2.add(new StringField("foo", "bar", Store.YES)); + w.addDocuments(Arrays.asList(doc1, doc2)); + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = newSearcher(reader); + w.close(); + + final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true, 1.5f); + searcher.setQueryCache(queryCache); + searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE); + + BooleanQuery.Builder bq = new BooleanQuery.Builder(); + bq.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER); + bq.add(new TermQuery(new Term("foo", "quux")), Occur.FILTER); + + assertEquals(Collections.emptySet(), new HashSet<>(queryCache.cachedQueries())); + searcher.search(bq.build(), 1); + // bar is not cached since its cost is 2 which is 2x the cost of the BooleanQuery + assertEquals(Collections.singleton(new TermQuery(new Term("foo", "quux"))), new HashSet<>(queryCache.cachedQueries())); + + reader.close(); + w.close(); + dir.close(); + } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java index 4d690ce..81ae187 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java @@ -152,7 +152,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase { assertEquals(doc, actualIt.nextDoc()); float expectedScore = expected.score(); float actualScore = actual.score(); - assertEquals(expectedScore, actualScore, CheckHits.explainToleranceDelta(expectedScore, actualScore)); + assertEquals(expectedScore, actualScore, 0d); } assertEquals(DocIdSetIterator.NO_MORE_DOCS, actualIt.nextDoc()); } @@ -170,7 +170,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase { assertEquals(doc, actualIt.advance(prevDoc+amount)); float expectedScore = expected.score(); float actualScore = actual.score(); - assertEquals(expectedScore, actualScore, CheckHits.explainToleranceDelta(expectedScore, actualScore)); + assertEquals(expectedScore, actualScore, 0d); prevDoc = doc; } assertEquals(DocIdSetIterator.NO_MORE_DOCS, actualIt.advance(prevDoc+amount)); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java index 9afcecc..f6e75c2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java @@ -63,7 +63,7 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase { IndexSearcher searcher = new IndexSearcher(reader); UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy(); - LRUQueryCache cache = new LRUQueryCache(10, Long.MAX_VALUE, new LRUQueryCache.MinSegmentSizePredicate(1, 0f)); + LRUQueryCache cache = new LRUQueryCache(10, Long.MAX_VALUE, new LRUQueryCache.MinSegmentSizePredicate(1, 0f), Float.POSITIVE_INFINITY); searcher.setQueryCache(cache); searcher.setQueryCachingPolicy(policy); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java ---------------------------------------------------------------------- diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java index f597626..b84b7ad 100644 --- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java +++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java @@ -24,7 +24,6 @@ import org.apache.lucene.expressions.js.VariableContext; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; -import org.apache.lucene.search.CheckHits; import org.apache.lucene.search.DoubleValuesSource; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.IndexSearcher; @@ -120,7 +119,7 @@ public class TestDemoExpressions extends LuceneTestCase { FieldDoc d = (FieldDoc) td.scoreDocs[i]; float expected = (float) Math.sqrt(d.score); float actual = ((Double)d.fields[0]).floatValue(); - assertEquals(expected, actual, CheckHits.explainToleranceDelta(expected, actual)); + assertEquals(expected, actual, 0d); } } @@ -138,7 +137,7 @@ public class TestDemoExpressions extends LuceneTestCase { FieldDoc d = (FieldDoc) td.scoreDocs[i]; float expected = 2*d.score; float actual = ((Double)d.fields[0]).floatValue(); - assertEquals(expected, actual, CheckHits.explainToleranceDelta(expected, actual)); + assertEquals(expected, actual, 0d); } } @@ -157,7 +156,7 @@ public class TestDemoExpressions extends LuceneTestCase { FieldDoc d = (FieldDoc) td.scoreDocs[i]; float expected = 2*d.score; float actual = ((Double)d.fields[0]).floatValue(); - assertEquals(expected, actual, CheckHits.explainToleranceDelta(expected, actual)); + assertEquals(expected, actual, 0d); } } @@ -177,7 +176,7 @@ public class TestDemoExpressions extends LuceneTestCase { FieldDoc d = (FieldDoc) td.scoreDocs[i]; float expected = 2*d.score; float actual = ((Double)d.fields[0]).floatValue(); - assertEquals(expected, actual, CheckHits.explainToleranceDelta(expected, actual)); + assertEquals(expected, actual, 0d); } } @@ -209,7 +208,7 @@ public class TestDemoExpressions extends LuceneTestCase { FieldDoc d = (FieldDoc) td.scoreDocs[i]; float expected = n*d.score; float actual = ((Double)d.fields[0]).floatValue(); - assertEquals(expected, actual, CheckHits.explainToleranceDelta(expected, actual)); + assertEquals(expected, actual, 0d); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java ---------------------------------------------------------------------- diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java index d001cc8..d65e646 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java @@ -34,7 +34,6 @@ import org.apache.lucene.queries.function.ValueSource; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BoostQuery; -import org.apache.lucene.search.CheckHits; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; @@ -60,8 +59,8 @@ public class TestCustomScoreQuery extends FunctionTestSetup { */ @Test public void testCustomScoreInt() throws Exception { - doTestCustomScore(INT_VALUESOURCE, 1.0); - doTestCustomScore(INT_VALUESOURCE, 4.0); + doTestCustomScore(INT_VALUESOURCE, 1f); + doTestCustomScore(INT_VALUESOURCE, 4f); } /** @@ -69,8 +68,8 @@ public class TestCustomScoreQuery extends FunctionTestSetup { */ @Test public void testCustomScoreFloat() throws Exception { - doTestCustomScore(FLOAT_VALUESOURCE, 1.0); - doTestCustomScore(FLOAT_VALUESOURCE, 6.0); + doTestCustomScore(FLOAT_VALUESOURCE, 1f); + doTestCustomScore(FLOAT_VALUESOURCE, 6f); } // must have static class otherwise serialization tests fail @@ -225,8 +224,7 @@ public class TestCustomScoreQuery extends FunctionTestSetup { } // Test that FieldScoreQuery returns docs with expected score. - private void doTestCustomScore(ValueSource valueSource, double dboost) throws Exception { - float boost = (float) dboost; + private void doTestCustomScore(ValueSource valueSource, float boost) throws Exception { FunctionQuery functionQuery = new FunctionQuery(valueSource); IndexReader r = DirectoryReader.open(dir); IndexSearcher s = newSearcher(r); @@ -242,11 +240,11 @@ public class TestCustomScoreQuery extends FunctionTestSetup { // custom query, that should score the same as q1. BooleanQuery.Builder q2CustomNeutralB = new BooleanQuery.Builder(); Query q2CustomNeutralInner = new CustomScoreQuery(q1); - q2CustomNeutralB.add(new BoostQuery(q2CustomNeutralInner, (float)Math.sqrt(dboost)), BooleanClause.Occur.SHOULD); + q2CustomNeutralB.add(new BoostQuery(q2CustomNeutralInner, (float)Math.sqrt(boost)), BooleanClause.Occur.SHOULD); // a little tricky: we split the boost across an outer BQ and CustomScoreQuery // this ensures boosting is correct across all these functions (see LUCENE-4935) Query q2CustomNeutral = q2CustomNeutralB.build(); - q2CustomNeutral = new BoostQuery(q2CustomNeutral, (float)Math.sqrt(dboost)); + q2CustomNeutral = new BoostQuery(q2CustomNeutral, (float)Math.sqrt(boost)); log(q2CustomNeutral); // custom query, that should (by default) multiply the scores of q1 by that of the field @@ -328,19 +326,19 @@ public class TestCustomScoreQuery extends FunctionTestSetup { float score2 = h2customNeutral.get(doc); logResult("score2=", s, q2, doc, score2); - assertEquals("same score (just boosted) for neutral", boost * score1, score2, CheckHits.explainToleranceDelta(boost * score1, score2)); + assertEquals("same score (just boosted) for neutral", boost * score1, score2, Math.scalb(score2, -15)); float score3 = h3CustomMul.get(doc); logResult("score3=", s, q3, doc, score3); - assertEquals("new score for custom mul", boost * fieldScore * score1, score3, CheckHits.explainToleranceDelta(boost * fieldScore * score1, score3)); + assertEquals("new score for custom mul", boost * fieldScore * score1, score3, Math.scalb(score3, -15)); float score4 = h4CustomAdd.get(doc); logResult("score4=", s, q4, doc, score4); - assertEquals("new score for custom add", boost * (fieldScore + score1), score4, CheckHits.explainToleranceDelta(boost * (fieldScore + score1), score4)); + assertEquals("new score for custom add", boost * (fieldScore + score1), score4, Math.scalb(score4, -15)); float score5 = h5CustomMulAdd.get(doc); logResult("score5=", s, q5, doc, score5); - assertEquals("new score for custom mul add", boost * fieldScore * (score1 + fieldScore), score5, CheckHits.explainToleranceDelta(boost * fieldScore * (score1 + fieldScore), score5)); + assertEquals("new score for custom mul add", boost * fieldScore * (score1 + fieldScore), score5, Math.scalb(score5, -15)); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d27ddcb4/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java ---------------------------------------------------------------------- diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java index 7696a63..3e25c0f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java @@ -33,21 +33,6 @@ import org.apache.lucene.util.LuceneTestCase; * Utility class for asserting expected hits in tests. */ public class CheckHits { - - /** - * Some explains methods calculate their values though a slightly - * different order of operations from the actual scoring method ... - * this allows for a small amount of relative variation - */ - public static float EXPLAIN_SCORE_TOLERANCE_DELTA = 0.001f; - - /** - * In general we use a relative epsilon, but some tests do crazy things - * like boost documents with 0, creating tiny tiny scores where the - * relative difference is large but the absolute difference is tiny. - * we ensure the the epsilon is always at least this big. - */ - public static float EXPLAIN_SCORE_TOLERANCE_MINIMUM = 1e-6f; /** * Tests that all documents up to maxDoc which are *not* in the @@ -313,12 +298,6 @@ public class CheckHits { (query, defaultFieldName, searcher, deep)); } - - /** returns a reasonable epsilon for comparing two floats, - * where minor differences are acceptable such as score vs. explain */ - public static float explainToleranceDelta(float f1, float f2) { - return Math.max(EXPLAIN_SCORE_TOLERANCE_MINIMUM, Math.max(Math.abs(f1), Math.abs(f2)) * EXPLAIN_SCORE_TOLERANCE_DELTA); - } private static final Pattern COMPUTED_FROM_PATTERN = Pattern.compile(".*, computed as .* from:"); @@ -340,7 +319,7 @@ public class CheckHits { float value = expl.getValue(); // TODO: clean this up if we use junit 5 (the assert message is costly) try { - Assert.assertEquals(score, value, explainToleranceDelta(score, value)); + Assert.assertEquals(score, value, 0d); } catch (Exception e) { Assert.fail(q+": score(doc="+doc+")="+score+" != explanationScore="+value+" Explanation: "+expl); } @@ -398,32 +377,42 @@ public class CheckHits { +"\" must be 'max of plus x times others', 'computed as x from:' or end with 'product of'" +" or 'sum of:' or 'max of:' - "+expl); } - float sum = 0; + double sum = 0; float product = 1; - float max = 0; + float max = Float.NEGATIVE_INFINITY; + double maxError = 0; for (int i=0; i true)); + IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25, context -> true, random().nextBoolean() ? 1.1f : Float.POSITIVE_INFINITY)); IndexSearcher.setDefaultQueryCachingPolicy(MAYBE_CACHE_POLICY); }