You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dn...@apache.org on 2021/10/01 14:02:40 UTC

[lucene] branch main updated: LUCENE-10126: Re-introduce chunk scoring logic in tests (#331)

This is an automated email from the ASF dual-hosted git repository.

dnhatn pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 5748743  LUCENE-10126: Re-introduce chunk scoring logic in tests (#331)
5748743 is described below

commit 5748743d91e240e22cf63c5476d1b6ba51592744
Author: Nhat Nguyen <nh...@elastic.co>
AuthorDate: Fri Oct 1 10:02:28 2021 -0400

    LUCENE-10126: Re-introduce chunk scoring logic in tests (#331)
    
    This commit re-introduces the chunk scoring logic in AssertingBulkScorer
    and enables it in TestSortOptimization.
---
 .../apache/lucene/search/TestSortOptimization.java | 67 +++++++++++-----------
 .../apache/lucene/search/AssertingBulkScorer.java  | 19 +++++-
 .../apache/lucene/search/AssertingScorable.java    | 30 +++++++++-
 3 files changed, 81 insertions(+), 35 deletions(-)

diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
index 65e28ee..52ef0c8 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
@@ -59,7 +59,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader);
     final SortField sortField = new SortField("my_field", SortField.Type.LONG);
     final Sort sort = new Sort(sortField);
     final int numHits = 3;
@@ -76,7 +76,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(i, ((Long) fieldDoc.fields[0]).intValue());
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value + " >= " + numDocs, topDocs.totalHits.value < numDocs);
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     { // paging sort with after
@@ -92,7 +92,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(afterValue + 1 + i, fieldDoc.fields[0]);
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < numDocs);
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     { // test that if there is the secondary sort on _score, scores are filled correctly
@@ -109,7 +109,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(1.0, score, 0.001);
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < numDocs);
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     { // test that if numeric field is a secondary sort, no optimization is run
@@ -144,7 +144,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final SortField sortField = new SortField("my_field", SortField.Type.LONG);
     final Sort sort = new Sort(sortField);
     final int numHits = 3;
@@ -183,7 +183,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final int numHits = 3;
     final int totalHitsThreshold = 3;
 
@@ -209,10 +209,7 @@ public class TestSortOptimization extends LuceneTestCase {
       searcher.search(new MatchAllDocsQuery(), collector);
       TopDocs topDocs = collector.topDocs();
       assertEquals(topDocs.scoreDocs.length, numHits);
-      assertTrue(
-          topDocs.totalHits.value + " >= " + numDocs,
-          topDocs.totalHits.value
-              < numDocs); // assert that some docs were skipped => optimization was run
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     reader.close();
@@ -236,7 +233,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final int numHits = 3;
     final int totalHitsThreshold = 3;
 
@@ -252,9 +249,7 @@ public class TestSortOptimization extends LuceneTestCase {
         FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
         assertEquals(100, fieldDoc.fields[0]);
       }
-      assertTrue(
-          topDocs.totalHits.value
-              < numDocs); // assert that some docs were skipped => optimization was run
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     { // test that sorting on a single field with equal values and after parameter
@@ -315,7 +310,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final SortField sortField = new SortField("my_field", SortField.Type.FLOAT);
     final Sort sort = new Sort(sortField);
     final int numHits = 3;
@@ -332,7 +327,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(1f * i, fieldDoc.fields[0]);
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < numDocs);
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     reader.close();
@@ -372,7 +367,8 @@ public class TestSortOptimization extends LuceneTestCase {
     int numHits = 0;
     do {
       for (int i = 0; i < numIndices; i++) {
-        IndexSearcher searcher = new IndexSearcher(readers[i]);
+        IndexSearcher searcher =
+            newSearcher(readers[i], random().nextBoolean(), random().nextBoolean());
         final TopFieldCollector collector =
             TopFieldCollector.create(sort, size, after, totalHitsThreshold);
         searcher.search(new MatchAllDocsQuery(), collector);
@@ -398,8 +394,7 @@ public class TestSortOptimization extends LuceneTestCase {
 
     final int expectedNumHits = numDocsInIndex * numIndices;
     assertEquals(expectedNumHits, numHits);
-    // check that the optimization was run, as very few docs were collected
-    assertTrue(collectedDocs < totalDocs);
+    assertNonCompetitiveHitsAreSkipped(collectedDocs, totalDocs);
   }
 
   public void testDocSortOptimizationWithAfter() throws IOException {
@@ -416,7 +411,7 @@ public class TestSortOptimization extends LuceneTestCase {
 
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final int numHits = 10;
     final int totalHitsThreshold = 10;
     final int[] searchAfters = {3, 10, numDocs - 10};
@@ -437,8 +432,7 @@ public class TestSortOptimization extends LuceneTestCase {
           assertEquals(expectedDocID, topDocs.scoreDocs[i].doc);
         }
         assertTrue(collector.isEarlyTerminated());
-        // check that very few docs were collected
-        assertTrue(topDocs.totalHits.value < numDocs);
+        assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
       }
 
       // sort by _doc + _score with search after should trigger optimization
@@ -457,8 +451,7 @@ public class TestSortOptimization extends LuceneTestCase {
           assertEquals(expectedDocID, topDocs.scoreDocs[i].doc);
         }
         assertTrue(collector.isEarlyTerminated());
-        // assert that very few docs were collected
-        assertTrue(topDocs.totalHits.value < numDocs);
+        assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
       }
 
       // sort by _doc desc should not trigger optimization
@@ -547,7 +540,7 @@ public class TestSortOptimization extends LuceneTestCase {
     {
       final TopFieldCollector collector =
           TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
-      IndexSearcher searcher = new IndexSearcher(reader);
+      IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
       searcher.search(new MatchAllDocsQuery(), collector);
       TopDocs topDocs = collector.topDocs();
       assertEquals(numHits, topDocs.scoreDocs.length);
@@ -555,7 +548,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(i, topDocs.scoreDocs[i].doc);
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < 10); // assert that very few docs were collected
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, 10);
     }
 
     // sort by _doc with a bool query should skip all non-competitive documents
@@ -566,7 +559,7 @@ public class TestSortOptimization extends LuceneTestCase {
       BooleanQuery.Builder bq = new BooleanQuery.Builder();
       bq.add(LongPoint.newRangeQuery("lf", lowerRange, Long.MAX_VALUE), BooleanClause.Occur.MUST);
       bq.add(new TermQuery(new Term("tf", "seg1")), BooleanClause.Occur.MUST);
-      IndexSearcher searcher = newSearcher(reader);
+      IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
       searcher.search(bq.build(), collector);
 
       TopDocs topDocs = collector.topDocs();
@@ -577,7 +570,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals("seg1", d.get("tf"));
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < 10); // assert that very few docs were collected
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, 10);
     }
 
     reader.close();
@@ -606,7 +599,7 @@ public class TestSortOptimization extends LuceneTestCase {
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
 
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     searcher.setQueryCache(null);
     final int numHits = 10;
     final int totalHitsThreshold = 10;
@@ -645,7 +638,7 @@ public class TestSortOptimization extends LuceneTestCase {
     IndexReader reader = writer.getReader();
     writer.close();
 
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
 
     SortField longSortOnIntField = new SortField("intField", SortField.Type.LONG);
     assertThrows(
@@ -699,7 +692,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     SortField sortField = new SortField("my_field", SortField.Type.LONG);
     TopFieldDocs topDocs =
         searcher.search(new MatchAllDocsQuery(), 1 + random().nextInt(100), new Sort(sortField));
@@ -744,7 +737,7 @@ public class TestSortOptimization extends LuceneTestCase {
     seqNos.sort(Long::compare);
     IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     SortField sortField = new SortField("seq_no", SortField.Type.LONG);
     int visitedHits = 0;
     ScoreDoc after = null;
@@ -768,4 +761,14 @@ public class TestSortOptimization extends LuceneTestCase {
     reader.close();
     dir.close();
   }
+
+  private void assertNonCompetitiveHitsAreSkipped(long collectedHits, long numDocs) {
+    if (collectedHits >= numDocs) {
+      fail(
+          "Expected some non-competitive hits are skipped; got collected_hits="
+              + collectedHits
+              + " num_docs="
+              + numDocs);
+    }
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
index 2e7cc61..a44523a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
@@ -80,8 +80,23 @@ final class AssertingBulkScorer extends BulkScorer {
     assert min <= max : "max must be greater than min, got min=" + min + ", and max=" + max;
     this.max = max;
     collector = new AssertingLeafCollector(collector, min, max);
-    final int next = in.score(collector, acceptDocs, min, max);
-    assert next >= max;
+    int next = min;
+    do {
+      final int upTo;
+      if (random.nextBoolean()) {
+        upTo = max;
+      } else {
+        final long interval;
+        if (random.nextInt(100) <= 5) {
+          interval = 1 + random.nextInt(10);
+        } else {
+          interval = 1 + random.nextInt(random.nextBoolean() ? 100 : 5000);
+        }
+        upTo = Math.toIntExact(Math.min(next + interval, max));
+      }
+      next = in.score(new AssertingLeafCollector(collector, next, upTo), acceptDocs, next, upTo);
+    } while (next < max);
+
     if (max >= maxDoc || next >= maxDoc) {
       assert next == DocIdSetIterator.NO_MORE_DOCS;
       return DocIdSetIterator.NO_MORE_DOCS;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
index 1534b0e9..9cc40bd 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
@@ -45,13 +45,41 @@ public class AssertingScorable extends FilterScorable {
     if (in instanceof AssertingScorable) {
       return in;
     }
-    return new AssertingScorable(in);
+    // If `in` is Scorer, we need to wrap it as a Scorer instead of Scorable because
+    // NumericComparator uses the iterator cost of a Scorer in sort optimization.
+    if (in instanceof Scorer) {
+      return new WrappedScorer((Scorer) in);
+    } else {
+      return new AssertingScorable(in);
+    }
+  }
+
+  private static class WrappedScorer extends FilterScorer {
+    WrappedScorer(Scorer in) {
+      super(in);
+    }
+
+    @Override
+    public float score() throws IOException {
+      return new AssertingScorable(in).score();
+    }
+
+    @Override
+    public void setMinCompetitiveScore(float minScore) throws IOException {
+      in.setMinCompetitiveScore(minScore);
+    }
+
+    @Override
+    public float getMaxScore(int upTo) throws IOException {
+      return in.getMaxScore(upTo);
+    }
   }
 
   public static Scorable unwrap(Scorable in) {
     while (true) {
       if (in instanceof AssertingScorable) in = ((AssertingScorable) in).in;
       else if (in instanceof AssertingScorer) in = ((AssertingScorer) in).in;
+      else if (in instanceof WrappedScorer) in = ((WrappedScorer) in).in;
       else return in;
     }
   }