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:12:32 UTC

[lucene-solr] branch branch_8x updated (c119c5f -> 3b52227)

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

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


    from c119c5f  SOLR-15665: Move polling logic under main (#2585)
     new be759ef  LUCENE-10126: Fix competitive iterator wrongly skip docs
     new 3b52227  LUCENE-10126 Add extra test on _doc sort (#326)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../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 | 96 ++++++++++++++++------
 .../apache/lucene/search/AssertingBulkScorer.java  | 26 ++++--
 .../apache/lucene/search/AssertingScorable.java    | 40 +++++++--
 6 files changed, 141 insertions(+), 55 deletions(-)

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

Posted by dn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit be759ef1964b1228d3cf24913099678fdce2ae2f
Author: Nhat Nguyen <nh...@elastic.co>
AuthorDate: Sun Oct 3 13:11:47 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;
     }
   }
-
 }

[lucene-solr] 02/02: LUCENE-10126 Add extra test on _doc sort (#326)

Posted by dn...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3b52227cc9841d895072981ddae104231016254d
Author: Mayya Sharipova <ma...@elastic.co>
AuthorDate: Wed Sep 29 14:49:16 2021 -0400

    LUCENE-10126 Add extra test on _doc sort (#326)
    
    Add extra test on _doc sort to test
    that search with after collects all documents
---
 .../apache/lucene/search/TestSortOptimization.java | 36 ++++++++++++++++++++++
 1 file changed, 36 insertions(+)

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 390e974..9270cc2 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
@@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
 import java.io.IOException;
@@ -465,6 +466,41 @@ public class TestSortOptimization extends LuceneTestCase {
     dir.close();
   }
 
+  public void testDocSortOptimizationWithAfterCollectsAllDocs() throws IOException {
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    final int numDocs = atLeast(TEST_NIGHTLY ? 50_000 : 5_000);
+    final boolean multipleSegments = random().nextBoolean();
+    final int numDocsInSegment = numDocs / 10 + random().nextInt(numDocs / 10);
+
+    for (int i = 1; i <= numDocs; ++i) {
+      final Document doc = new Document();
+      writer.addDocument(doc);
+      if (multipleSegments && (i % numDocsInSegment == 0)) {
+        writer.flush();
+      }
+    }
+    writer.flush();
+
+    IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = newSearcher(reader);
+    int visitedHits = 0;
+    ScoreDoc after = null;
+    while (visitedHits < numDocs) {
+      int batch = 1 + random().nextInt(500);
+      Query query = new MatchAllDocsQuery();
+      TopDocs topDocs = searcher.searchAfter(after, query, batch, new Sort(FIELD_DOC));
+      int expectedHits = Math.min(numDocs - visitedHits, batch);
+      assertEquals(expectedHits, topDocs.scoreDocs.length);
+      after = topDocs.scoreDocs[expectedHits - 1];
+      for (int i = 0; i < topDocs.scoreDocs.length; i++) {
+        assertEquals(visitedHits, topDocs.scoreDocs[i].doc);
+        visitedHits++;
+      }
+    }
+    assertEquals(visitedHits, numDocs);
+    IOUtils.close(writer, reader, dir);
+  }
 
   public void testDocSortOptimization() throws IOException {
     final Directory dir = newDirectory();