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/03 17:10:25 UTC

[lucene-solr] 01/02: LUCENE-10126: Fix competitive iterator wrongly skip docs

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

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

commit dffca84172a2645e0f144b6565ea64565f861267
Author: Nhat Nguyen <nh...@elastic.co>
AuthorDate: Sun Oct 3 11:57:54 2021 -0400

    LUCENE-10126: Fix competitive iterator wrongly skip docs
    
    The competitive iterator can wrongly skip a document that is advanced
    but not collected in the previous scoreRange.
---
 .../src/java/org/apache/lucene/search/Weight.java  | 30 ++++++-----
 .../lucene/search/comparators/DocComparator.java   |  2 +-
 .../search/comparators/NumericComparator.java      |  2 +-
 .../apache/lucene/search/TestSortOptimization.java | 60 ++++++++++++----------
 .../apache/lucene/search/AssertingBulkScorer.java  | 26 ++++++++--
 .../apache/lucene/search/AssertingScorable.java    | 40 ++++++++++++---
 6 files changed, 105 insertions(+), 55 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java
index a26d1a3..7ea490f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Weight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java
@@ -216,17 +216,23 @@ public abstract class Weight implements SegmentCacheable {
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
       DocIdSetIterator scorerIterator = twoPhase == null ? iterator : twoPhase.approximation();
-      DocIdSetIterator collectorIterator = collector.competitiveIterator();
+      DocIdSetIterator competitiveIterator = collector.competitiveIterator();
       DocIdSetIterator filteredIterator;
-      if (collectorIterator == null) {
+      if (competitiveIterator == null) {
         filteredIterator = scorerIterator;
       } else {
+        // Wrap CompetitiveIterator and ScorerIterator start with (i.e., calling nextDoc()) the last
+        // visited docID because ConjunctionDISI might have advanced to it in the previous
+        // scoreRange, but we didn't process due to the range limit of scoreRange.
         if (scorerIterator.docID() != -1) {
-          // Wrap ScorerIterator to start from -1 for conjunction 
           scorerIterator = new StartDISIWrapper(scorerIterator);
         }
+        if (competitiveIterator.docID() != -1) {
+          competitiveIterator = new StartDISIWrapper(competitiveIterator);
+        }
         // filter scorerIterator to keep only competitive docs as defined by collector
-        filteredIterator = ConjunctionDISI.intersectIterators(Arrays.asList(scorerIterator, collectorIterator));
+        filteredIterator =
+            ConjunctionDISI.intersectIterators(Arrays.asList(scorerIterator, competitiveIterator));
       }
       if (filteredIterator.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
         scoreAll(collector, filteredIterator, twoPhase, acceptDocs);
@@ -287,17 +293,15 @@ public abstract class Weight implements SegmentCacheable {
     }
   }
 
-  /**
-   * Wraps an internal docIdSetIterator for it to start with docID = -1
-   */
-  protected static class StartDISIWrapper extends DocIdSetIterator {
+  /** Wraps an internal docIdSetIterator for it to start with the last visited docID */
+  private static class StartDISIWrapper extends DocIdSetIterator {
     private final DocIdSetIterator in;
-    private final int min;
+    private final int startDocID;
     private int docID = -1;
 
-    public StartDISIWrapper(DocIdSetIterator in) {
+    StartDISIWrapper(DocIdSetIterator in) {
       this.in = in;
-      this.min = in.docID();
+      this.startDocID = in.docID();
     }
 
     @Override
@@ -312,8 +316,8 @@ public abstract class Weight implements SegmentCacheable {
 
     @Override
     public int advance(int target) throws IOException {
-      if (target <= min) {
-        return docID = min;
+      if (target <= startDocID) {
+        return docID = startDocID;
       }
       return docID = in.advance(target);
     }
diff --git a/lucene/core/src/java/org/apache/lucene/search/comparators/DocComparator.java b/lucene/core/src/java/org/apache/lucene/search/comparators/DocComparator.java
index 0bda127..fec4085 100644
--- a/lucene/core/src/java/org/apache/lucene/search/comparators/DocComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/comparators/DocComparator.java
@@ -139,7 +139,7 @@ public class DocComparator extends FieldComparator<Integer> {
         return null;
       } else {
         return new DocIdSetIterator() {
-          private int docID = -1;
+          private int docID = competitiveIterator.docID();
 
           @Override
           public int nextDoc() throws IOException {
diff --git a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
index 8c04748..fed13a0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
@@ -250,7 +250,7 @@ public abstract class NumericComparator<T extends Number> extends FieldComparato
     public DocIdSetIterator competitiveIterator() {
       if (enableSkipping == false) return null;
       return new DocIdSetIterator() {
-        private int docID = -1;
+        private int docID = competitiveIterator.docID();
 
         @Override
         public int nextDoc() throws IOException {
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 90e978d..390e974 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);
     sortField.setCanUsePoints();
     final Sort sort = new Sort(sortField);
@@ -76,7 +76,7 @@ public class TestSortOptimization extends LuceneTestCase {
         assertEquals(i, ((Long) fieldDoc.fields[0]).intValue());
       }
       assertTrue(collector.isEarlyTerminated());
-      assertTrue(topDocs.totalHits.value < numDocs);
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     { // paging sort with after
@@ -91,7 +91,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
@@ -106,7 +106,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
@@ -148,7 +148,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);
     sortField.setCanUsePoints();
     final Sort sort = new Sort(sortField);
@@ -185,7 +185,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,7 +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); // assert that some docs were skipped => optimization was run
+      assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
     }
 
     reader.close();
@@ -219,7 +219,7 @@ public class TestSortOptimization extends LuceneTestCase {
   public void testSortOptimizationEqualValues() throws IOException {
     final Directory dir = newDirectory();
     final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
-    final int numDocs = atLeast(10000);
+    final int numDocs = atLeast(TEST_NIGHTLY ? 50_000 : 10_000);
     for (int i = 1; i <= numDocs; ++i) {
       final Document doc = new Document();
       doc.add(new NumericDocValuesField("my_field1", 100)); // all docs have the same value of my_field1
@@ -230,7 +230,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;
 
@@ -246,7 +246,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
@@ -305,7 +305,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);
     sortField.setCanUsePoints();
     final Sort sort = new Sort(sortField);
@@ -322,7 +322,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();
@@ -362,7 +362,7 @@ public class TestSortOptimization extends LuceneTestCase {
     int numHits = 0;
     do {
       for (int i = 0; i < numIndices; i++) {
-        IndexSearcher searcher = newSearcher(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);
@@ -388,8 +388,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 {
@@ -405,7 +404,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final int numHits = 10;
     final int totalHitsThreshold = 10;
     final int[] searchAfters = {3, 10, numDocs - 10};
@@ -424,8 +423,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
@@ -442,8 +440,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
@@ -487,7 +484,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = newSearcher(reader);;
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     final int numHits = 3;
     final int totalHitsThreshold = 3;
     final Sort sort = new Sort(FIELD_DOC);
@@ -502,7 +499,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
@@ -522,7 +519,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();
@@ -551,7 +548,7 @@ public class TestSortOptimization extends LuceneTestCase {
     final IndexReader reader = DirectoryReader.open(writer);
     writer.close();
 
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     searcher.setQueryCache(null);
     final int numHits = 10;
     final int totalHitsThreshold = 10;
@@ -589,7 +586,7 @@ public class TestSortOptimization extends LuceneTestCase {
     IndexReader reader = writer.getReader();
     writer.close();
 
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     SortField sort1 = new SortField("intField", SortField.Type.LONG);
     sort1.setCanUsePoints();
     assertThrows(
@@ -646,7 +643,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
     IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     SortField sortField = new SortField("my_field", SortField.Type.LONG);
     sortField.setCanUsePoints();
     TopFieldDocs topDocs =
@@ -692,7 +689,7 @@ public class TestSortOptimization extends LuceneTestCase {
     seqNos.sort(Long::compare);
     IndexReader reader = DirectoryReader.open(writer);
     writer.close();
-    IndexSearcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
     SortField sortField = new SortField("seq_no", SortField.Type.LONG);
     sortField.setCanUsePoints();
     int visitedHits = 0;
@@ -718,4 +715,13 @@ public class TestSortOptimization extends LuceneTestCase {
     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 4445f3d..309d31a 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
@@ -73,13 +73,30 @@ final class AssertingBulkScorer extends BulkScorer {
   }
 
   @Override
-  public int score(LeafCollector collector, Bits acceptDocs, int min, final int max) throws IOException {
-    assert min >= this.max: "Scoring backward: min=" + min + " while previous max was max=" + this.max;
+  public int score(LeafCollector collector, Bits acceptDocs, int min, final int max)
+      throws IOException {
+    assert min >= this.max
+        : "Scoring backward: min=" + min + " while previous max was max=" + this.max;
     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;
@@ -92,5 +109,4 @@ final class AssertingBulkScorer extends BulkScorer {
   public String toString() {
     return "AssertingBulkScorer(" + in + ")";
   }
-
 }
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 208eb4b..662d486 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
@@ -47,18 +47,42 @@ 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
-        return in;
+      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;
     }
   }
-
 }