You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/11/04 20:32:12 UTC

[GitHub] [lucene-solr] mayya-sharipova opened a new pull request #2063: LUCENE-9599 Make comparator aware of index sorting

mayya-sharipova opened a new pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063


   Currently, if search sort is equal to index sort,  we have an early
   termination in TopFieldCollector. As we work to enhance comparators
   to provide skipping functionality (PR #1351), we would like to
   move this termination functionality on index sort from
   TopFieldCollector to comparators.
   
   This patch does the following:
   - Add method usesIndexSort to LeafFieldComparator
   - Make numeric comparators aware of index sort and early terminate on
     collecting all competitive hits
   - Move TermValComparator and TermOrdValComparator from FieldComparator
     to comparator package, for all comparators to be in the same package
   - Enhance TermValComparator to provide skipping functionality when
     index is sorted
   
   One item left for TODO for a following PR is to remove the logic of
   early termniation from TopFieldCollector. We can do that once
   we ensure that all BulkScorers are using iterators from collectors
   than can skip non-competitive docs.
   
   Relates to #1351


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518811869



##########
File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
##########
@@ -387,125 +346,225 @@ protected SortedDocValues getSortedDocValues(LeafReaderContext context, String f
     
     @Override
     public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-      termsIndex = getSortedDocValues(context, field);
-      currentReaderGen++;
+      return new TermOrdValLeafComparator(context);
+    }
 
-      if (topValue != null) {
-        // Recompute topOrd/SameReader
-        int ord = termsIndex.lookupTerm(topValue);
-        if (ord >= 0) {
-          topSameReader = true;
-          topOrd = ord;
-        } else {
-          topSameReader = false;
-          topOrd = -ord-2;
+    @Override
+    public BytesRef value(int slot) {
+      return values[slot];
+    }
+
+    @Override
+    public int compareValues(BytesRef val1, BytesRef val2) {
+      if (val1 == null) {
+        if (val2 == null) {
+          return 0;
         }
-      } else {
-        topOrd = missingOrd;
-        topSameReader = true;
+        return missingSortCmp;
+      } else if (val2 == null) {
+        return -missingSortCmp;
       }
-      //System.out.println("  getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
+      return val1.compareTo(val2);
+    }
 
-      if (bottomSlot != -1) {
-        // Recompute bottomOrd/SameReader
-        setBottom(bottomSlot);
+    /**
+     * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+     */
+    public class TermOrdValLeafComparator implements LeafFieldComparator {
+      private final SortedDocValues termsIndex;
+      private boolean indexSort = false; // true if a query sort is a part of the index sort
+      private DocIdSetIterator competitiveIterator;
+      private boolean collectedAllCompetitiveHits = false;
+      private boolean iteratorUpdated = false;
+
+      public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+        termsIndex = getSortedDocValues(context, field);
+        currentReaderGen++;
+        if (topValue != null) {
+          // Recompute topOrd/SameReader
+          int ord = termsIndex.lookupTerm(topValue);
+          if (ord >= 0) {
+            topSameReader = true;
+            topOrd = ord;
+          } else {
+            topSameReader = false;
+            topOrd = -ord-2;
+          }
+        } else {
+          topOrd = missingOrd;
+          topSameReader = true;
+        }
+        if (bottomSlot != -1) {
+          // Recompute bottomOrd/SameReader
+          setBottom(bottomSlot);
+        }
+        this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
       }
 
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) throws IOException {
-      bottomSlot = bottom;
+      protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+        return DocValues.getSorted(context.reader(), field);
+      }
 
-      bottomValue = values[bottomSlot];
-      if (currentReaderGen == readerGen[bottomSlot]) {
-        bottomOrd = ords[bottomSlot];
-        bottomSameReader = true;
-      } else {
-        if (bottomValue == null) {
-          // missingOrd is null for all segments
-          assert ords[bottomSlot] == missingOrd;
-          bottomOrd = missingOrd;
+      @Override
+      public void setBottom(final int slot) throws IOException {
+        bottomSlot = slot;
+        bottomValue = values[bottomSlot];
+        if (currentReaderGen == readerGen[bottomSlot]) {
+          bottomOrd = ords[bottomSlot];
           bottomSameReader = true;
-          readerGen[bottomSlot] = currentReaderGen;
         } else {
-          final int ord = termsIndex.lookupTerm(bottomValue);
-          if (ord < 0) {
-            bottomOrd = -ord - 2;
-            bottomSameReader = false;
-          } else {
-            bottomOrd = ord;
-            // exact value match
+          if (bottomValue == null) {
+            // missingOrd is null for all segments
+            assert ords[bottomSlot] == missingOrd;
+            bottomOrd = missingOrd;
             bottomSameReader = true;
-            readerGen[bottomSlot] = currentReaderGen;            
-            ords[bottomSlot] = bottomOrd;
+            readerGen[bottomSlot] = currentReaderGen;
+          } else {
+            final int ord = termsIndex.lookupTerm(bottomValue);
+            if (ord < 0) {
+              bottomOrd = -ord - 2;
+              bottomSameReader = false;
+            } else {
+              bottomOrd = ord;
+              // exact value match
+              bottomSameReader = true;
+              readerGen[bottomSlot] = currentReaderGen;
+              ords[bottomSlot] = bottomOrd;
+            }
           }
         }
       }
-    }
 
-    @Override
-    public void setTopValue(BytesRef value) {
-      // null is fine: it means the last doc of the prior
-      // search was missing this value
-      topValue = value;
-      //System.out.println("setTopValue " + topValue);
-    }
+      @Override
+      public int compareBottom(int doc) throws IOException {
+        assert bottomSlot != -1;
+        int docOrd = getOrdForDoc(doc);
+        if (docOrd == -1) {
+          docOrd = missingOrd;
+        }
+        int result;
+        if (bottomSameReader) {
+          // ord is precisely comparable, even in the equal case
+          result = bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in setBottom):
+          result = 1;
+        } else {
+          result = -1;
+        }
+        // for the index sort case, if we encounter a first doc that is non-competitive,
+        // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+        if (indexSort && (reverse ? result >= 0 : result <= 0)) {
+          collectedAllCompetitiveHits = true;
+          if (hitsThresholdReached && iteratorUpdated == false) {
+            competitiveIterator = DocIdSetIterator.empty();
+            iteratorUpdated = true;
+          }
+        }
+        return result;
+      }
 
-    @Override
-    public BytesRef value(int slot) {
-      return values[slot];
-    }
+      @Override
+      public int compareTop(int doc) throws IOException {
+        int ord = getOrdForDoc(doc);
+        if (ord == -1) {
+          ord = missingOrd;
+        }
+        if (topSameReader) {
+          // ord is precisely comparable, even in the equal case
+          return topOrd - ord;
+        } else if (ord <= topOrd) {
+          // the equals case always means doc is < value (because we set lastOrd to the lower bound)
+          return 1;
+        } else {
+          return -1;
+        }
+      }
 
-    @Override
-    public int compareTop(int doc) throws IOException {
+      @Override
+      public void copy(int slot, int doc) throws IOException {
+        int ord = getOrdForDoc(doc);
+        if (ord == -1) {
+          ord = missingOrd;
+          values[slot] = null;
+        } else {
+          assert ord >= 0;
+          if (tempBRs[slot] == null) {
+            tempBRs[slot] = new BytesRefBuilder();
+          }
+          tempBRs[slot].copyBytes(termsIndex.lookupOrd(ord));
+          values[slot] = tempBRs[slot].get();
+        }
+        ords[slot] = ord;
+        readerGen[slot] = currentReaderGen;
+      }
 
-      int ord = getOrdForDoc(doc);
-      if (ord == -1) {
-        ord = missingOrd;
+      @Override
+      public void setScorer(Scorable scorer) throws IOException {}
+
+      @Override
+      public void usesIndexSort() {
+        indexSort = true;
       }
 
-      if (topSameReader) {
-        // ord is precisely comparable, even in the equal
-        // case
-        //System.out.println("compareTop doc=" + doc + " ord=" + ord + " ret=" + (topOrd-ord));
-        return topOrd - ord;
-      } else if (ord <= topOrd) {
-        // the equals case always means doc is < value
-        // (because we set lastOrd to the lower bound)
-        return 1;
-      } else {
-        return -1;
+      @Override
+      public void setHitsThresholdReached() {
+        hitsThresholdReached = true;
+        // for the index sort case, if we collected collected all competitive hits
+        // we can update the iterator to skip the rest of docs
+        if (indexSort && collectedAllCompetitiveHits && iteratorUpdated == false) {
+          competitiveIterator = DocIdSetIterator.empty();
+          iteratorUpdated = true;
+        }
       }
-    }
 
-    @Override
-    public int compareValues(BytesRef val1, BytesRef val2) {
-      if (val1 == null) {
-        if (val2 == null) {
-          return 0;
+      @Override
+      public DocIdSetIterator competitiveIterator() {
+        if (indexSort == false) return null;
+        return new DocIdSetIterator() {
+          private int docID = -1;
+
+          @Override
+          public int nextDoc() throws IOException {
+            return advance(docID + 1);
+          }
+
+          @Override
+          public int docID() {
+            return docID;
+          }
+
+          @Override
+          public long cost() {
+            return competitiveIterator.cost();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return docID = competitiveIterator.advance(target);
+          }
+        };
+      }
+
+      private int getOrdForDoc(int doc) throws IOException {
+        if (termsIndex.advanceExact(doc)) {
+          return termsIndex.ordValue();
+        } else {
+          return -1;
         }
-        return missingSortCmp;
-      } else if (val2 == null) {
-        return -missingSortCmp;
       }
-      return val1.compareTo(val2);
     }
-
-    @Override
-    public void setScorer(Scorable scorer) {}
   }
   
   /** Sorts by field's natural Term sort order.  All
    *  comparisons are done using BytesRef.compareTo, which is
    *  slow for medium to large result sets but possibly
    *  very fast for very small results sets. */
-  public static class TermValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
+  public static class TermValComparator extends FieldComparator<BytesRef> {

Review comment:
       It seems that we [don't support index sort](https://github.com/apache/lucene-solr/blob/master/lucene/core/src/java/org/apache/lucene/search/SortField.java#L524) on `STRING_VAL` type  that will use `TermValComparator` comparator,  thus early termination on index sort is not necessary for this class.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova closed pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova closed pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518351139



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -139,6 +140,7 @@ void collectAnyHit(int doc, int hitsCollected) throws IOException {
 
     @Override
     public void setScorer(Scorable scorer) throws IOException {
+      if (canEarlyTerminate) comparator.usesIndexSort();

Review comment:
       Let's use curly braces for every if statement; I think that is the prevailing style here at least

##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -100,6 +100,7 @@ boolean thresholdCheck(int doc) throws IOException {
         // since docs are visited in doc Id order, if compare is 0, it means
         // this document is largest than anything else in the queue, and
         // therefore not competitive.
+        // TODO: remove early termination in TopFieldCollector, as this should be managed by comparators

Review comment:
       so, with this change does this essentially become a no-op? Because we will already early-terminate due to the comparator?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    final int numHits = randomIntBetween(1, numDocs - 10);
+    final int totalHitsThreshold = randomIntBetween(1, numDocs - 10);
+    {

Review comment:
       why the extra block?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,186 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {

Review comment:
       thanks!

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();

Review comment:
       If we use try-with-resources, we won't need the close() and we will safely close in case of unexpected failures

##########
File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
##########
@@ -387,125 +346,225 @@ protected SortedDocValues getSortedDocValues(LeafReaderContext context, String f
     
     @Override
     public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-      termsIndex = getSortedDocValues(context, field);
-      currentReaderGen++;
+      return new TermOrdValLeafComparator(context);
+    }
 
-      if (topValue != null) {
-        // Recompute topOrd/SameReader
-        int ord = termsIndex.lookupTerm(topValue);
-        if (ord >= 0) {
-          topSameReader = true;
-          topOrd = ord;
-        } else {
-          topSameReader = false;
-          topOrd = -ord-2;
+    @Override
+    public BytesRef value(int slot) {
+      return values[slot];
+    }
+
+    @Override
+    public int compareValues(BytesRef val1, BytesRef val2) {
+      if (val1 == null) {
+        if (val2 == null) {
+          return 0;
         }
-      } else {
-        topOrd = missingOrd;
-        topSameReader = true;
+        return missingSortCmp;
+      } else if (val2 == null) {
+        return -missingSortCmp;
       }
-      //System.out.println("  getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
+      return val1.compareTo(val2);
+    }
 
-      if (bottomSlot != -1) {
-        // Recompute bottomOrd/SameReader
-        setBottom(bottomSlot);
+    /**
+     * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+     */
+    public class TermOrdValLeafComparator implements LeafFieldComparator {
+      private final SortedDocValues termsIndex;
+      private boolean indexSort = false; // true if a query sort is a part of the index sort
+      private DocIdSetIterator competitiveIterator;
+      private boolean collectedAllCompetitiveHits = false;
+      private boolean iteratorUpdated = false;
+
+      public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+        termsIndex = getSortedDocValues(context, field);
+        currentReaderGen++;
+        if (topValue != null) {
+          // Recompute topOrd/SameReader
+          int ord = termsIndex.lookupTerm(topValue);
+          if (ord >= 0) {
+            topSameReader = true;
+            topOrd = ord;
+          } else {
+            topSameReader = false;
+            topOrd = -ord-2;
+          }
+        } else {
+          topOrd = missingOrd;
+          topSameReader = true;
+        }
+        if (bottomSlot != -1) {
+          // Recompute bottomOrd/SameReader
+          setBottom(bottomSlot);
+        }
+        this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
       }
 
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) throws IOException {
-      bottomSlot = bottom;
+      protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+        return DocValues.getSorted(context.reader(), field);
+      }
 
-      bottomValue = values[bottomSlot];
-      if (currentReaderGen == readerGen[bottomSlot]) {
-        bottomOrd = ords[bottomSlot];
-        bottomSameReader = true;
-      } else {
-        if (bottomValue == null) {
-          // missingOrd is null for all segments
-          assert ords[bottomSlot] == missingOrd;
-          bottomOrd = missingOrd;
+      @Override
+      public void setBottom(final int slot) throws IOException {
+        bottomSlot = slot;
+        bottomValue = values[bottomSlot];
+        if (currentReaderGen == readerGen[bottomSlot]) {
+          bottomOrd = ords[bottomSlot];
           bottomSameReader = true;
-          readerGen[bottomSlot] = currentReaderGen;
         } else {
-          final int ord = termsIndex.lookupTerm(bottomValue);
-          if (ord < 0) {
-            bottomOrd = -ord - 2;
-            bottomSameReader = false;
-          } else {
-            bottomOrd = ord;
-            // exact value match
+          if (bottomValue == null) {
+            // missingOrd is null for all segments
+            assert ords[bottomSlot] == missingOrd;
+            bottomOrd = missingOrd;
             bottomSameReader = true;
-            readerGen[bottomSlot] = currentReaderGen;            
-            ords[bottomSlot] = bottomOrd;
+            readerGen[bottomSlot] = currentReaderGen;
+          } else {
+            final int ord = termsIndex.lookupTerm(bottomValue);
+            if (ord < 0) {
+              bottomOrd = -ord - 2;
+              bottomSameReader = false;
+            } else {
+              bottomOrd = ord;
+              // exact value match
+              bottomSameReader = true;
+              readerGen[bottomSlot] = currentReaderGen;
+              ords[bottomSlot] = bottomOrd;
+            }
           }
         }
       }
-    }
 
-    @Override
-    public void setTopValue(BytesRef value) {
-      // null is fine: it means the last doc of the prior
-      // search was missing this value
-      topValue = value;
-      //System.out.println("setTopValue " + topValue);
-    }
+      @Override
+      public int compareBottom(int doc) throws IOException {
+        assert bottomSlot != -1;
+        int docOrd = getOrdForDoc(doc);
+        if (docOrd == -1) {
+          docOrd = missingOrd;
+        }
+        int result;
+        if (bottomSameReader) {
+          // ord is precisely comparable, even in the equal case
+          result = bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in setBottom):
+          result = 1;
+        } else {
+          result = -1;
+        }
+        // for the index sort case, if we encounter a first doc that is non-competitive,
+        // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+        if (indexSort && (reverse ? result >= 0 : result <= 0)) {
+          collectedAllCompetitiveHits = true;
+          if (hitsThresholdReached && iteratorUpdated == false) {
+            competitiveIterator = DocIdSetIterator.empty();
+            iteratorUpdated = true;

Review comment:
       I think we only set this flag once, right? Is it to avoid creating additional calls to `DocIdSetIterator.empty()`? Could we instead create a class constant EMPTY iterator and reassign every time, thus eliminating the need for `iteratorUpdated`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
##########
@@ -218,7 +237,7 @@ public void visit(int docID, byte[] packedValue) {
 
         @Override
         public DocIdSetIterator competitiveIterator() {
-            if (enableSkipping == false) return null;
+            if (enableSkipping == false && indexSort == false) return null;

Review comment:
       Hmm I guess we sometimes don't use the curly braces. Well, I prefer them; do as you see fit :)  I expect one of these days we will get automatic style enforcement and never have to speak of this again...

##########
File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
##########
@@ -387,125 +346,225 @@ protected SortedDocValues getSortedDocValues(LeafReaderContext context, String f
     
     @Override
     public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-      termsIndex = getSortedDocValues(context, field);
-      currentReaderGen++;
+      return new TermOrdValLeafComparator(context);
+    }
 
-      if (topValue != null) {
-        // Recompute topOrd/SameReader
-        int ord = termsIndex.lookupTerm(topValue);
-        if (ord >= 0) {
-          topSameReader = true;
-          topOrd = ord;
-        } else {
-          topSameReader = false;
-          topOrd = -ord-2;
+    @Override
+    public BytesRef value(int slot) {
+      return values[slot];
+    }
+
+    @Override
+    public int compareValues(BytesRef val1, BytesRef val2) {
+      if (val1 == null) {
+        if (val2 == null) {
+          return 0;
         }
-      } else {
-        topOrd = missingOrd;
-        topSameReader = true;
+        return missingSortCmp;
+      } else if (val2 == null) {
+        return -missingSortCmp;
       }
-      //System.out.println("  getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
+      return val1.compareTo(val2);
+    }
 
-      if (bottomSlot != -1) {
-        // Recompute bottomOrd/SameReader
-        setBottom(bottomSlot);
+    /**
+     * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+     */
+    public class TermOrdValLeafComparator implements LeafFieldComparator {
+      private final SortedDocValues termsIndex;
+      private boolean indexSort = false; // true if a query sort is a part of the index sort
+      private DocIdSetIterator competitiveIterator;
+      private boolean collectedAllCompetitiveHits = false;
+      private boolean iteratorUpdated = false;
+
+      public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+        termsIndex = getSortedDocValues(context, field);
+        currentReaderGen++;
+        if (topValue != null) {
+          // Recompute topOrd/SameReader
+          int ord = termsIndex.lookupTerm(topValue);
+          if (ord >= 0) {
+            topSameReader = true;
+            topOrd = ord;
+          } else {
+            topSameReader = false;
+            topOrd = -ord-2;
+          }
+        } else {
+          topOrd = missingOrd;
+          topSameReader = true;
+        }
+        if (bottomSlot != -1) {
+          // Recompute bottomOrd/SameReader
+          setBottom(bottomSlot);
+        }
+        this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
       }
 
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) throws IOException {
-      bottomSlot = bottom;
+      protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+        return DocValues.getSorted(context.reader(), field);
+      }
 
-      bottomValue = values[bottomSlot];
-      if (currentReaderGen == readerGen[bottomSlot]) {
-        bottomOrd = ords[bottomSlot];
-        bottomSameReader = true;
-      } else {
-        if (bottomValue == null) {
-          // missingOrd is null for all segments
-          assert ords[bottomSlot] == missingOrd;
-          bottomOrd = missingOrd;
+      @Override
+      public void setBottom(final int slot) throws IOException {
+        bottomSlot = slot;
+        bottomValue = values[bottomSlot];
+        if (currentReaderGen == readerGen[bottomSlot]) {
+          bottomOrd = ords[bottomSlot];
           bottomSameReader = true;
-          readerGen[bottomSlot] = currentReaderGen;
         } else {
-          final int ord = termsIndex.lookupTerm(bottomValue);
-          if (ord < 0) {
-            bottomOrd = -ord - 2;
-            bottomSameReader = false;
-          } else {
-            bottomOrd = ord;
-            // exact value match
+          if (bottomValue == null) {
+            // missingOrd is null for all segments
+            assert ords[bottomSlot] == missingOrd;
+            bottomOrd = missingOrd;
             bottomSameReader = true;
-            readerGen[bottomSlot] = currentReaderGen;            
-            ords[bottomSlot] = bottomOrd;
+            readerGen[bottomSlot] = currentReaderGen;
+          } else {
+            final int ord = termsIndex.lookupTerm(bottomValue);
+            if (ord < 0) {
+              bottomOrd = -ord - 2;
+              bottomSameReader = false;
+            } else {
+              bottomOrd = ord;
+              // exact value match
+              bottomSameReader = true;
+              readerGen[bottomSlot] = currentReaderGen;
+              ords[bottomSlot] = bottomOrd;
+            }
           }
         }
       }
-    }
 
-    @Override
-    public void setTopValue(BytesRef value) {
-      // null is fine: it means the last doc of the prior
-      // search was missing this value
-      topValue = value;
-      //System.out.println("setTopValue " + topValue);
-    }
+      @Override
+      public int compareBottom(int doc) throws IOException {
+        assert bottomSlot != -1;
+        int docOrd = getOrdForDoc(doc);
+        if (docOrd == -1) {
+          docOrd = missingOrd;
+        }
+        int result;
+        if (bottomSameReader) {
+          // ord is precisely comparable, even in the equal case
+          result = bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in setBottom):
+          result = 1;
+        } else {
+          result = -1;
+        }
+        // for the index sort case, if we encounter a first doc that is non-competitive,
+        // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+        if (indexSort && (reverse ? result >= 0 : result <= 0)) {
+          collectedAllCompetitiveHits = true;
+          if (hitsThresholdReached && iteratorUpdated == false) {
+            competitiveIterator = DocIdSetIterator.empty();
+            iteratorUpdated = true;
+          }
+        }
+        return result;
+      }
 
-    @Override
-    public BytesRef value(int slot) {
-      return values[slot];
-    }
+      @Override
+      public int compareTop(int doc) throws IOException {
+        int ord = getOrdForDoc(doc);
+        if (ord == -1) {
+          ord = missingOrd;
+        }
+        if (topSameReader) {
+          // ord is precisely comparable, even in the equal case
+          return topOrd - ord;
+        } else if (ord <= topOrd) {
+          // the equals case always means doc is < value (because we set lastOrd to the lower bound)
+          return 1;
+        } else {
+          return -1;
+        }
+      }
 
-    @Override
-    public int compareTop(int doc) throws IOException {
+      @Override
+      public void copy(int slot, int doc) throws IOException {
+        int ord = getOrdForDoc(doc);
+        if (ord == -1) {
+          ord = missingOrd;
+          values[slot] = null;
+        } else {
+          assert ord >= 0;
+          if (tempBRs[slot] == null) {
+            tempBRs[slot] = new BytesRefBuilder();
+          }
+          tempBRs[slot].copyBytes(termsIndex.lookupOrd(ord));
+          values[slot] = tempBRs[slot].get();
+        }
+        ords[slot] = ord;
+        readerGen[slot] = currentReaderGen;
+      }
 
-      int ord = getOrdForDoc(doc);
-      if (ord == -1) {
-        ord = missingOrd;
+      @Override
+      public void setScorer(Scorable scorer) throws IOException {}
+
+      @Override
+      public void usesIndexSort() {
+        indexSort = true;
       }
 
-      if (topSameReader) {
-        // ord is precisely comparable, even in the equal
-        // case
-        //System.out.println("compareTop doc=" + doc + " ord=" + ord + " ret=" + (topOrd-ord));
-        return topOrd - ord;
-      } else if (ord <= topOrd) {
-        // the equals case always means doc is < value
-        // (because we set lastOrd to the lower bound)
-        return 1;
-      } else {
-        return -1;
+      @Override
+      public void setHitsThresholdReached() {
+        hitsThresholdReached = true;
+        // for the index sort case, if we collected collected all competitive hits
+        // we can update the iterator to skip the rest of docs
+        if (indexSort && collectedAllCompetitiveHits && iteratorUpdated == false) {
+          competitiveIterator = DocIdSetIterator.empty();
+          iteratorUpdated = true;
+        }
       }
-    }
 
-    @Override
-    public int compareValues(BytesRef val1, BytesRef val2) {
-      if (val1 == null) {
-        if (val2 == null) {
-          return 0;
+      @Override
+      public DocIdSetIterator competitiveIterator() {
+        if (indexSort == false) return null;
+        return new DocIdSetIterator() {
+          private int docID = -1;
+
+          @Override
+          public int nextDoc() throws IOException {
+            return advance(docID + 1);
+          }
+
+          @Override
+          public int docID() {
+            return docID;
+          }
+
+          @Override
+          public long cost() {
+            return competitiveIterator.cost();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return docID = competitiveIterator.advance(target);
+          }
+        };
+      }
+
+      private int getOrdForDoc(int doc) throws IOException {
+        if (termsIndex.advanceExact(doc)) {
+          return termsIndex.ordValue();
+        } else {
+          return -1;
         }
-        return missingSortCmp;
-      } else if (val2 == null) {
-        return -missingSortCmp;
       }
-      return val1.compareTo(val2);
     }
-
-    @Override
-    public void setScorer(Scorable scorer) {}
   }
   
   /** Sorts by field's natural Term sort order.  All
    *  comparisons are done using BytesRef.compareTo, which is
    *  slow for medium to large result sets but possibly
    *  very fast for very small results sets. */
-  public static class TermValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
+  public static class TermValComparator extends FieldComparator<BytesRef> {

Review comment:
       So - this is unchanged, just a refactor of the LeafFieldComparator into an internal class? And we don't implement early termination in this case? Could that be a regression if we remove early termination from the Collector?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    final int numHits = randomIntBetween(1, numDocs - 10);
+    final int totalHitsThreshold = randomIntBetween(1, numDocs - 10);
+    {
+      // test that optimization is run when search sort is equal to the index sort
+      TopFieldCollector collector = TopFieldCollector.create(indexSort, numHits, null, totalHitsThreshold);
+      searcher.search(new MatchAllDocsQuery(), collector);
+      TopDocs topDocs = collector.topDocs();
+      assertTrue(collector.isEarlyTerminated());

Review comment:
       does this test distinguish between the kind of early termination we had before, and what this provides?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    final int numHits = randomIntBetween(1, numDocs - 10);
+    final int totalHitsThreshold = randomIntBetween(1, numDocs - 10);
+    {

Review comment:
       Oh I see, you can scope the variables - nice! Although in this case it seems unneccessary




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518799074



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -139,6 +140,7 @@ void collectAnyHit(int doc, int hitsCollected) throws IOException {
 
     @Override
     public void setScorer(Scorable scorer) throws IOException {
+      if (canEarlyTerminate) comparator.usesIndexSort();

Review comment:
       Addressed in c52d9a7




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518799913



##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
##########
@@ -218,7 +237,7 @@ public void visit(int docID, byte[] packedValue) {
 
         @Override
         public DocIdSetIterator competitiveIterator() {
-            if (enableSkipping == false) return null;
+            if (enableSkipping == false && indexSort == false) return null;

Review comment:
       Addressed in c52d9a7.  I also have a plan to a do a  separate PR to just fix formatting.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518316774



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,186 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {

Review comment:
       @msokolov Thanks for the feedback. Addressed in d2909aa




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518813515



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    final int numHits = randomIntBetween(1, numDocs - 10);
+    final int totalHitsThreshold = randomIntBetween(1, numDocs - 10);
+    {
+      // test that optimization is run when search sort is equal to the index sort
+      TopFieldCollector collector = TopFieldCollector.create(indexSort, numHits, null, totalHitsThreshold);
+      searcher.search(new MatchAllDocsQuery(), collector);
+      TopDocs topDocs = collector.topDocs();
+      assertTrue(collector.isEarlyTerminated());

Review comment:
       You are right, they don't distinguish this. The idea that if we remove early termination in TopFieldCollector, these tests should still work.
   
   If all this confusing, I can also upgrade this PR to remove early termination in TopFieldCollector. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518800810



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();

Review comment:
       Thank you, good suggestion.  Addressed in c52d9a7

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,249 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    boolean reverseSort = randomBoolean();
+    final SortField sortField = new SortField("field1", SortField.Type.LONG, reverseSort);
+    Sort indexSort = new Sort(sortField);
+    iwc.setIndexSort(indexSort);
+    final IndexWriter writer = new IndexWriter(dir, iwc);
+
+    final int numDocs = atLeast(50);
+    int[] sortedValues = initializeNumericValues(numDocs, reverseSort, 0);
+    int[] randomIdxs = randomIdxs(numDocs);
+
+    for (int i = 0; i < numDocs; i++) {
+      final Document doc = new Document();
+      if (sortedValues[randomIdxs[i]] > 0) {
+        doc.add(new NumericDocValuesField("field1", sortedValues[randomIdxs[i]]));
+      }
+      writer.addDocument(doc);
+      if (i == 30) {
+        writer.flush();
+      }
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    final int numHits = randomIntBetween(1, numDocs - 10);
+    final int totalHitsThreshold = randomIntBetween(1, numDocs - 10);
+    {

Review comment:
       Addressed in c52d9a7




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] msokolov commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518055212



##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/TermOrdValComparator.java
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.comparators;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import java.io.IOException;
+
+/**
+ * Comparator that sorts by field's natural Term sort order using ordinals.
+ * This is functionally equivalent to
+ * {@link org.apache.lucene.search.comparators.TermValComparator},
+ * but it first resolves the string to their relative ordinal positions
+ * (using the index returned by
+ * {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}),
+ * and does most comparisons using the ordinals.
+ * For medium to large results, this comparator will be much faster
+ * than {@link org.apache.lucene.search.comparators.TermValComparator}.
+ * For very small result sets it may be slower.
+ *
+ * The comparator provides an iterator that can efficiently skip
+ * documents when search sort is done according to the index sort.
+ */
+public class TermOrdValComparator extends FieldComparator<BytesRef> {
+  private final String field;
+  private final boolean reverse;
+  private final int[] ords; // ordinals for each slot
+  private final BytesRef[] values; // values for each slot
+  private final BytesRefBuilder[] tempBRs;
+  /* Which reader last copied a value into the slot. When
+     we compare two slots, we just compare-by-ord if the
+     readerGen is the same; else we must compare the
+     values (slower).*/
+  private final int[] readers;
+  private int currentReader = -1; // index of the current reader we are on
+  private final int missingSortCmp; // -1 – if missing values are sorted first, 1 – if sorted last
+  private final int missingOrd; // which ordinal to use for a missing value
+
+  private BytesRef topValue;
+  private boolean topSameReader;
+  private int topOrd;
+
+  private BytesRef bottomValue;
+  boolean bottomSameReader; // true if current bottom slot matches the current reader
+  int bottomSlot = -1; // bottom slot, or -1 if queue isn't full yet
+  int bottomOrd; // bottom ord (same as ords[bottomSlot] once bottomSlot is set), cached for faster comparison
+
+  protected boolean hitsThresholdReached;
+
+  public TermOrdValComparator(int numHits, String field, boolean sortMissingLast, boolean reverse) {
+    this.field = field;
+    this.reverse = reverse;
+    this.ords = new int[numHits];
+    this.values = new BytesRef[numHits];
+    tempBRs = new BytesRefBuilder[numHits];
+    readers = new int[numHits];
+    if (sortMissingLast) {
+      missingSortCmp = 1;
+      missingOrd = Integer.MAX_VALUE;
+    } else {
+      missingSortCmp = -1;
+      missingOrd = -1;
+    }
+  }
+
+  @Override
+  public int compare(int slot1, int slot2) {
+    if (readers[slot1] == readers[slot2]) {
+      return ords[slot1] - ords[slot2];
+    }
+    final BytesRef val1 = values[slot1];
+    final BytesRef val2 = values[slot2];
+    if (val1 == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return missingSortCmp;
+    } else if (val2 == null) {
+      return -missingSortCmp;
+    }
+    return val1.compareTo(val2);
+  }
+
+  @Override
+  public void setTopValue(BytesRef value) {
+    // null is accepted, this means the last doc of the prior search was missing this value
+    topValue = value;
+  }
+
+  @Override
+  public BytesRef value(int slot) {
+    return values[slot];
+  }
+
+  @Override
+  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+    return new TermOrdValLeafComparator(context);
+  }
+
+  @Override
+  public int compareValues(BytesRef val1, BytesRef val2) {
+    if (val1 == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return missingSortCmp;
+    } else if (val2 == null) {
+      return -missingSortCmp;
+    }
+    return val1.compareTo(val2);
+  }
+
+  /**
+   * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+   */
+  public class TermOrdValLeafComparator implements LeafFieldComparator {
+    private final SortedDocValues termsIndex;
+    private boolean indexSort = false; // true if a query sort is a part of the index sort
+    private DocIdSetIterator competitiveIterator;
+    private boolean collectedAllCompetitiveHits = false;
+    private boolean iteratorUpdated = false;
+
+    public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+      termsIndex = getSortedDocValues(context, field);
+      currentReader++;
+      if (topValue != null) {
+        // Recompute topOrd/SameReader
+        int ord = termsIndex.lookupTerm(topValue);
+        if (ord >= 0) {
+          topSameReader = true;
+          topOrd = ord;
+        } else {
+          topSameReader = false;
+          topOrd = -ord-2;
+        }
+      } else {
+        topOrd = missingOrd;
+        topSameReader = true;
+      }
+      if (bottomSlot != -1) {
+        // Recompute bottomOrd/SameReader
+        setBottom(bottomSlot);
+      }
+      this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
+    }
+
+    protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+      return DocValues.getSorted(context.reader(), field);
+    }
+
+    @Override
+    public void setBottom(final int slot) throws IOException {
+      bottomSlot = slot;
+      bottomValue = values[bottomSlot];
+      if (currentReader == readers[bottomSlot]) {
+        bottomOrd = ords[bottomSlot];
+        bottomSameReader = true;
+      } else {
+        if (bottomValue == null) {
+          // missingOrd is null for all segments
+          assert ords[bottomSlot] == missingOrd;
+          bottomOrd = missingOrd;
+          bottomSameReader = true;
+          readers[bottomSlot] = currentReader;
+        } else {
+          final int ord = termsIndex.lookupTerm(bottomValue);
+          if (ord < 0) {
+            bottomOrd = -ord - 2;
+            bottomSameReader = false;
+          } else {
+            bottomOrd = ord;
+            // exact value match
+            bottomSameReader = true;
+            readers[bottomSlot] = currentReader;
+            ords[bottomSlot] = bottomOrd;
+          }
+        }
+      }
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+      assert bottomSlot != -1;
+      int docOrd = getOrdForDoc(doc);
+      if (docOrd == -1) {
+        docOrd = missingOrd;
+      }
+      int result;
+      if (bottomSameReader) {
+        // ord is precisely comparable, even in the equal case
+        result = bottomOrd - docOrd;
+      } else if (bottomOrd >= docOrd) {
+        // the equals case always means bottom is > doc
+        // (because we set bottomOrd to the lower bound in setBottom):
+        result = 1;
+      } else {
+        result = -1;
+      }
+      // for the index sort case, if we encounter a first doc that is non-competitive,
+      // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+      if (indexSort && (reverse ? result >= 0 : result <= 0)) {

Review comment:
       oh it looks like there was some new stuff here related to indexSort. I wonder if it would be possible to separate the change moving these classes to top level for the sake of review? If it's a pain, perhaps you could just comment on what the changes are here?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestFieldSortOptimizationSkipping.java
##########
@@ -485,4 +491,186 @@ public void testDocSort() throws IOException {
     dir.close();
   }
 
+  public void testNumericSortOptimizationIndexSort() throws IOException {

Review comment:
       Do we have a test for the case where the index sort has multiple Sorts in it and the query sort matches its first n components?

##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/TermOrdValComparator.java
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.comparators;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import java.io.IOException;
+
+/**
+ * Comparator that sorts by field's natural Term sort order using ordinals.
+ * This is functionally equivalent to
+ * {@link org.apache.lucene.search.comparators.TermValComparator},
+ * but it first resolves the string to their relative ordinal positions
+ * (using the index returned by
+ * {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}),
+ * and does most comparisons using the ordinals.
+ * For medium to large results, this comparator will be much faster
+ * than {@link org.apache.lucene.search.comparators.TermValComparator}.
+ * For very small result sets it may be slower.
+ *
+ * The comparator provides an iterator that can efficiently skip
+ * documents when search sort is done according to the index sort.
+ */
+public class TermOrdValComparator extends FieldComparator<BytesRef> {

Review comment:
       Oh I see reverse was added..

##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/TermOrdValComparator.java
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.comparators;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import java.io.IOException;
+
+/**
+ * Comparator that sorts by field's natural Term sort order using ordinals.
+ * This is functionally equivalent to
+ * {@link org.apache.lucene.search.comparators.TermValComparator},
+ * but it first resolves the string to their relative ordinal positions
+ * (using the index returned by
+ * {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}),
+ * and does most comparisons using the ordinals.
+ * For medium to large results, this comparator will be much faster
+ * than {@link org.apache.lucene.search.comparators.TermValComparator}.
+ * For very small result sets it may be slower.
+ *
+ * The comparator provides an iterator that can efficiently skip
+ * documents when search sort is done according to the index sort.
+ */
+public class TermOrdValComparator extends FieldComparator<BytesRef> {

Review comment:
       Is this just moved to top-level class, or were there changes? It's hard to tell in the PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#issuecomment-724995524


   @msokolov @jimczi Thank you for the review.
   
   As per @jimczi's latest comment, since it makes more sense to disable sort optimization on index sort in comparators rather then move this logic from `TopFieldCollector` to comparators, I am closing this PR,  and I have opened a new much smaller  [PR](https://github.com/apache/lucene-solr/pull/2075)  that will implement this plan. 
   
   Sorry for the trouble.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#issuecomment-723131924


   @msokolov Thanks a lot for  another round of review, I tried to address it in c52d9a73


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518316315



##########
File path: lucene/core/src/java/org/apache/lucene/search/comparators/TermOrdValComparator.java
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.comparators;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import java.io.IOException;
+
+/**
+ * Comparator that sorts by field's natural Term sort order using ordinals.
+ * This is functionally equivalent to
+ * {@link org.apache.lucene.search.comparators.TermValComparator},
+ * but it first resolves the string to their relative ordinal positions
+ * (using the index returned by
+ * {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}),
+ * and does most comparisons using the ordinals.
+ * For medium to large results, this comparator will be much faster
+ * than {@link org.apache.lucene.search.comparators.TermValComparator}.
+ * For very small result sets it may be slower.
+ *
+ * The comparator provides an iterator that can efficiently skip
+ * documents when search sort is done according to the index sort.
+ */
+public class TermOrdValComparator extends FieldComparator<BytesRef> {
+  private final String field;
+  private final boolean reverse;
+  private final int[] ords; // ordinals for each slot
+  private final BytesRef[] values; // values for each slot
+  private final BytesRefBuilder[] tempBRs;
+  /* Which reader last copied a value into the slot. When
+     we compare two slots, we just compare-by-ord if the
+     readerGen is the same; else we must compare the
+     values (slower).*/
+  private final int[] readers;
+  private int currentReader = -1; // index of the current reader we are on
+  private final int missingSortCmp; // -1 – if missing values are sorted first, 1 – if sorted last
+  private final int missingOrd; // which ordinal to use for a missing value
+
+  private BytesRef topValue;
+  private boolean topSameReader;
+  private int topOrd;
+
+  private BytesRef bottomValue;
+  boolean bottomSameReader; // true if current bottom slot matches the current reader
+  int bottomSlot = -1; // bottom slot, or -1 if queue isn't full yet
+  int bottomOrd; // bottom ord (same as ords[bottomSlot] once bottomSlot is set), cached for faster comparison
+
+  protected boolean hitsThresholdReached;
+
+  public TermOrdValComparator(int numHits, String field, boolean sortMissingLast, boolean reverse) {
+    this.field = field;
+    this.reverse = reverse;
+    this.ords = new int[numHits];
+    this.values = new BytesRef[numHits];
+    tempBRs = new BytesRefBuilder[numHits];
+    readers = new int[numHits];
+    if (sortMissingLast) {
+      missingSortCmp = 1;
+      missingOrd = Integer.MAX_VALUE;
+    } else {
+      missingSortCmp = -1;
+      missingOrd = -1;
+    }
+  }
+
+  @Override
+  public int compare(int slot1, int slot2) {
+    if (readers[slot1] == readers[slot2]) {
+      return ords[slot1] - ords[slot2];
+    }
+    final BytesRef val1 = values[slot1];
+    final BytesRef val2 = values[slot2];
+    if (val1 == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return missingSortCmp;
+    } else if (val2 == null) {
+      return -missingSortCmp;
+    }
+    return val1.compareTo(val2);
+  }
+
+  @Override
+  public void setTopValue(BytesRef value) {
+    // null is accepted, this means the last doc of the prior search was missing this value
+    topValue = value;
+  }
+
+  @Override
+  public BytesRef value(int slot) {
+    return values[slot];
+  }
+
+  @Override
+  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+    return new TermOrdValLeafComparator(context);
+  }
+
+  @Override
+  public int compareValues(BytesRef val1, BytesRef val2) {
+    if (val1 == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return missingSortCmp;
+    } else if (val2 == null) {
+      return -missingSortCmp;
+    }
+    return val1.compareTo(val2);
+  }
+
+  /**
+   * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+   */
+  public class TermOrdValLeafComparator implements LeafFieldComparator {
+    private final SortedDocValues termsIndex;
+    private boolean indexSort = false; // true if a query sort is a part of the index sort
+    private DocIdSetIterator competitiveIterator;
+    private boolean collectedAllCompetitiveHits = false;
+    private boolean iteratorUpdated = false;
+
+    public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+      termsIndex = getSortedDocValues(context, field);
+      currentReader++;
+      if (topValue != null) {
+        // Recompute topOrd/SameReader
+        int ord = termsIndex.lookupTerm(topValue);
+        if (ord >= 0) {
+          topSameReader = true;
+          topOrd = ord;
+        } else {
+          topSameReader = false;
+          topOrd = -ord-2;
+        }
+      } else {
+        topOrd = missingOrd;
+        topSameReader = true;
+      }
+      if (bottomSlot != -1) {
+        // Recompute bottomOrd/SameReader
+        setBottom(bottomSlot);
+      }
+      this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
+    }
+
+    protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+      return DocValues.getSorted(context.reader(), field);
+    }
+
+    @Override
+    public void setBottom(final int slot) throws IOException {
+      bottomSlot = slot;
+      bottomValue = values[bottomSlot];
+      if (currentReader == readers[bottomSlot]) {
+        bottomOrd = ords[bottomSlot];
+        bottomSameReader = true;
+      } else {
+        if (bottomValue == null) {
+          // missingOrd is null for all segments
+          assert ords[bottomSlot] == missingOrd;
+          bottomOrd = missingOrd;
+          bottomSameReader = true;
+          readers[bottomSlot] = currentReader;
+        } else {
+          final int ord = termsIndex.lookupTerm(bottomValue);
+          if (ord < 0) {
+            bottomOrd = -ord - 2;
+            bottomSameReader = false;
+          } else {
+            bottomOrd = ord;
+            // exact value match
+            bottomSameReader = true;
+            readers[bottomSlot] = currentReader;
+            ords[bottomSlot] = bottomOrd;
+          }
+        }
+      }
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+      assert bottomSlot != -1;
+      int docOrd = getOrdForDoc(doc);
+      if (docOrd == -1) {
+        docOrd = missingOrd;
+      }
+      int result;
+      if (bottomSameReader) {
+        // ord is precisely comparable, even in the equal case
+        result = bottomOrd - docOrd;
+      } else if (bottomOrd >= docOrd) {
+        // the equals case always means bottom is > doc
+        // (because we set bottomOrd to the lower bound in setBottom):
+        result = 1;
+      } else {
+        result = -1;
+      }
+      // for the index sort case, if we encounter a first doc that is non-competitive,
+      // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+      if (indexSort && (reverse ? result >= 0 : result <= 0)) {

Review comment:
       @msokolov Thank you for the review and feedback. Sorry, indeed it difficult to see the changes this way. In commits f14f11371c8d and 98a02981e80e, I did as you suggested, I kept `TermOrdValComparator` and `TermValComparator` in FieldComparator, but still there are substantial changes,  so I will summarize them here:
   
   -  Before `TermOrdValComparator` and `TermValComparator` were extending FieldComparator<BytesRef>  and also implementing LeafFieldComparator.  Now each of these comparators have their corresponding inner classes that implement `LeafFieldComparator` and all methods related to leaf comparator were moved there.
   - `TermOrdValComparator` was enhanced to provide skipping functionality when index is sorted




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518803324



##########
File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
##########
@@ -387,125 +346,225 @@ protected SortedDocValues getSortedDocValues(LeafReaderContext context, String f
     
     @Override
     public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-      termsIndex = getSortedDocValues(context, field);
-      currentReaderGen++;
+      return new TermOrdValLeafComparator(context);
+    }
 
-      if (topValue != null) {
-        // Recompute topOrd/SameReader
-        int ord = termsIndex.lookupTerm(topValue);
-        if (ord >= 0) {
-          topSameReader = true;
-          topOrd = ord;
-        } else {
-          topSameReader = false;
-          topOrd = -ord-2;
+    @Override
+    public BytesRef value(int slot) {
+      return values[slot];
+    }
+
+    @Override
+    public int compareValues(BytesRef val1, BytesRef val2) {
+      if (val1 == null) {
+        if (val2 == null) {
+          return 0;
         }
-      } else {
-        topOrd = missingOrd;
-        topSameReader = true;
+        return missingSortCmp;
+      } else if (val2 == null) {
+        return -missingSortCmp;
       }
-      //System.out.println("  getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
+      return val1.compareTo(val2);
+    }
 
-      if (bottomSlot != -1) {
-        // Recompute bottomOrd/SameReader
-        setBottom(bottomSlot);
+    /**
+     * Leaf comparator for {@link TermOrdValComparator} that provides skipping functionality when index is sorted
+     */
+    public class TermOrdValLeafComparator implements LeafFieldComparator {
+      private final SortedDocValues termsIndex;
+      private boolean indexSort = false; // true if a query sort is a part of the index sort
+      private DocIdSetIterator competitiveIterator;
+      private boolean collectedAllCompetitiveHits = false;
+      private boolean iteratorUpdated = false;
+
+      public TermOrdValLeafComparator(LeafReaderContext context) throws IOException {
+        termsIndex = getSortedDocValues(context, field);
+        currentReaderGen++;
+        if (topValue != null) {
+          // Recompute topOrd/SameReader
+          int ord = termsIndex.lookupTerm(topValue);
+          if (ord >= 0) {
+            topSameReader = true;
+            topOrd = ord;
+          } else {
+            topSameReader = false;
+            topOrd = -ord-2;
+          }
+        } else {
+          topOrd = missingOrd;
+          topSameReader = true;
+        }
+        if (bottomSlot != -1) {
+          // Recompute bottomOrd/SameReader
+          setBottom(bottomSlot);
+        }
+        this.competitiveIterator = DocIdSetIterator.all(context.reader().maxDoc());
       }
 
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) throws IOException {
-      bottomSlot = bottom;
+      protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+        return DocValues.getSorted(context.reader(), field);
+      }
 
-      bottomValue = values[bottomSlot];
-      if (currentReaderGen == readerGen[bottomSlot]) {
-        bottomOrd = ords[bottomSlot];
-        bottomSameReader = true;
-      } else {
-        if (bottomValue == null) {
-          // missingOrd is null for all segments
-          assert ords[bottomSlot] == missingOrd;
-          bottomOrd = missingOrd;
+      @Override
+      public void setBottom(final int slot) throws IOException {
+        bottomSlot = slot;
+        bottomValue = values[bottomSlot];
+        if (currentReaderGen == readerGen[bottomSlot]) {
+          bottomOrd = ords[bottomSlot];
           bottomSameReader = true;
-          readerGen[bottomSlot] = currentReaderGen;
         } else {
-          final int ord = termsIndex.lookupTerm(bottomValue);
-          if (ord < 0) {
-            bottomOrd = -ord - 2;
-            bottomSameReader = false;
-          } else {
-            bottomOrd = ord;
-            // exact value match
+          if (bottomValue == null) {
+            // missingOrd is null for all segments
+            assert ords[bottomSlot] == missingOrd;
+            bottomOrd = missingOrd;
             bottomSameReader = true;
-            readerGen[bottomSlot] = currentReaderGen;            
-            ords[bottomSlot] = bottomOrd;
+            readerGen[bottomSlot] = currentReaderGen;
+          } else {
+            final int ord = termsIndex.lookupTerm(bottomValue);
+            if (ord < 0) {
+              bottomOrd = -ord - 2;
+              bottomSameReader = false;
+            } else {
+              bottomOrd = ord;
+              // exact value match
+              bottomSameReader = true;
+              readerGen[bottomSlot] = currentReaderGen;
+              ords[bottomSlot] = bottomOrd;
+            }
           }
         }
       }
-    }
 
-    @Override
-    public void setTopValue(BytesRef value) {
-      // null is fine: it means the last doc of the prior
-      // search was missing this value
-      topValue = value;
-      //System.out.println("setTopValue " + topValue);
-    }
+      @Override
+      public int compareBottom(int doc) throws IOException {
+        assert bottomSlot != -1;
+        int docOrd = getOrdForDoc(doc);
+        if (docOrd == -1) {
+          docOrd = missingOrd;
+        }
+        int result;
+        if (bottomSameReader) {
+          // ord is precisely comparable, even in the equal case
+          result = bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in setBottom):
+          result = 1;
+        } else {
+          result = -1;
+        }
+        // for the index sort case, if we encounter a first doc that is non-competitive,
+        // and the hits threshold is reached, we can update the iterator to skip the rest of docs
+        if (indexSort && (reverse ? result >= 0 : result <= 0)) {
+          collectedAllCompetitiveHits = true;
+          if (hitsThresholdReached && iteratorUpdated == false) {
+            competitiveIterator = DocIdSetIterator.empty();
+            iteratorUpdated = true;

Review comment:
       `iteratorUpdated` variable removed in c52d9a7.    
   `iteratorUpdated`, similarly to early termination in `TopFieldCollector` is not necessary and just extra precaution, for cases if there are some BulkScores that don't use iterators from collectors. 
   
   > Could we instead create a class constant EMPTY iterator
   
   We can't use a static iterator as we advance it once, it because invalid for all other instances. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mayya-sharipova commented on a change in pull request #2063: LUCENE-9599 Make comparator aware of index sorting

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #2063:
URL: https://github.com/apache/lucene-solr/pull/2063#discussion_r518798756



##########
File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
##########
@@ -100,6 +100,7 @@ boolean thresholdCheck(int doc) throws IOException {
         // since docs are visited in doc Id order, if compare is 0, it means
         // this document is largest than anything else in the queue, and
         // therefore not competitive.
+        // TODO: remove early termination in TopFieldCollector, as this should be managed by comparators

Review comment:
       Indeed, this is a no-op operation, and will be removed in a following PR.
   I left it for now, as we also need to ensure that all necessary BulkScorers use collectors/comparators iterators.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org