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/03/13 22:19:02 UTC

[GitHub] [lucene-solr] mayya-sharipova opened a new pull request #1351: Collectors to skip noncompetitive documents

mayya-sharipova opened a new pull request #1351: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351
 
 
   Similar how scorers can update their iterators to skip non-competitive
   documents, collectors and comparators should also provide and update
   iterators that allow them to skip non-competive documents
   
   This could be useful if we want to sort by some field.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r401227632
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
 
 Review comment:
   Addressed in 6c628f7

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394659789
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   Indeed it is more straightforward to just update an iterator in `setBottom` function of a comparator.
   
   But I was thinking it is better to have a special function for two reasons:
   1)  After updating an iterator, in `TopFieldCollector` we need to change 
   `totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;`
   
   2)  we also need to check `hitsThresholdChecker.isThresholdReached()`, and passing not strictly related object `hitsThresholdChecker` to a comparator's constructor doesn't look nice to me. 
   
   Please let me know if you think otherwise

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615300443
 
 
   I have caught up with @jimczi  offline, and it could be how selective a query iterator is important for performance. It is possible that if  a query iterator is already selective enough there is no point to materialize a collector's iterator based on points.  
   I am going to run benchmarks on MatchAll query to investigate that.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-608442689
 
 
   @romseygeek Thank you for the review and suggestions, I will work on them.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r403471265
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
 ##########
 @@ -91,6 +91,7 @@
   private String field;
   private Type type;  // defaults to determining type dynamically
   boolean reverse = false;  // defaults to natural order
+  private boolean skipNonCompetitiveDocs = false; // if true, sortField will use a comparator that can skip non-competitive docs
 
 Review comment:
   I'd rather not have this on SortField for now. This is an old API that never required fields to be indexed. I'd rather have new SortField implementations for now, and later look at how we can enable this in SortField.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393623417
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
+        final byte[] maxValueAsBytes = new byte[Long.BYTES];
+        final byte[] minValueAsBytes = new byte[Long.BYTES];
+        if (reverse == false) {
+            LongPoint.encodeDimension(bottom, maxValueAsBytes, 0);
+        } else {
+            LongPoint.encodeDimension(bottom, minValueAsBytes, 0);
+        };
+
+        DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
+        PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
+            DocIdSetBuilder.BulkAdder adder;
+            @Override
+            public void grow(int count) {
+                adder = result.grow(count);
+            }
+
+            @Override
+            public void visit(int docID) {
+                if (docID <= maxDocVisited) {
+                    return; // Already visited or skipped
+                }
+                adder.add(docID);
+            }
+
+            @Override
+            public void visit(int docID, byte[] packedValue) {
+                if (docID <= maxDocVisited) {
+                    return;  // Already visited or skipped
+                }
+                if ((reverse == false) && (Arrays.compareUnsigned(packedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0)) {
+                    return; // Doc's value is too high
+                }
+                if ((reverse == true) && (Arrays.compareUnsigned(packedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0)) {
+                    return;  // Doc's value is too low,
+                }
+                adder.add(docID); // doc is competitive
+            }
+
+            @Override
+            public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                if ((reverse == false) && (Arrays.compareUnsigned(minPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0)) {
+                   return PointValues.Relation.CELL_OUTSIDE_QUERY;
+                }
+                if ((reverse == true) && (Arrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0)) {
+                    return PointValues.Relation.CELL_OUTSIDE_QUERY;
+                }
+                return PointValues.Relation.CELL_CROSSES_QUERY;
+            }
+        };
+        pointValues.intersect(visitor);
 
 Review comment:
   we should update the iterator only if it allows to skip "lots" of documents, in distance feature query we set the threshold to a 8x reduction. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r396127831
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   The name seems to indicate that this is something that compares IteratorSuppliers, when in fact it is something that is a comparator that also supplies iterators. I'm not sure I understand yet where it fits, but given that, a better name might be IterableComparator?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   Do you know the reason why `hitsThresholdChecker.isThresholdReached`  checks for greater than numHits and not greater or equal numHits?
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I think it is better to have a previous implementation with a dedicate `updateIterator` function called from `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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r401228325
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
+   * optionally returns an iterator over competitive documents
 
 Review comment:
   Thanks @jpountz 
   
   > It's probably worth making explicit as null iterators are elsewhere interpreted as matching no documents
   
   What is the way to make this explicit?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394683505
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   For 1. we could set the totalHitsRelation when we reach the total hits threshold in the TOP_DOCS mode ? 
   For 2. I wonder if we could pass the hitsThresholdChecker to the LeafFieldComparator like we do for the scorer ?
   This way we can update the iterator internally when a new bottom is set or when `compareBottom` is called ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394671963
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -201,20 +201,54 @@ public long cost() {
     @Override
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
+      DocIdSetIterator scorerIterator = twoPhase == null? iterator: twoPhase.approximation();
+      DocIdSetIterator combinedIterator = collector.iterator() == null ? scorerIterator: combineScorerAndCollectorIterators(scorerIterator, collector);
       if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
-        scoreAll(collector, iterator, twoPhase, acceptDocs);
+        scoreAll(collector, combinedIterator, twoPhase, acceptDocs);
         return DocIdSetIterator.NO_MORE_DOCS;
       } else {
         int doc = scorer.docID();
-        if (doc < min) {
-          if (twoPhase == null) {
-            doc = iterator.advance(min);
-          } else {
-            doc = twoPhase.approximation().advance(min);
+        if (doc < min) scorerIterator.advance(min);
+        return scoreRange(collector, combinedIterator, twoPhase, acceptDocs, doc, max);
+      }
+    }
+
+    // conjunction iterator between scorer's iterator and collector's iterator
 
 Review comment:
   you can replace this with `ConjunctionDISI#intersectIterators` ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407527282
 
 

 ##########
 File path: lucene/test-framework/src/java/org/apache/lucene/search/AssertingLeafCollector.java
 ##########
 @@ -50,5 +50,9 @@ public void collect(int doc) throws IOException {
     lastCollected = doc;
   }
 
+  @Override
+  public DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return super.filterIterator(scorerIterator);
 
 Review comment:
   Is this necessary?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408712107
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,16 @@
    */
   void collect(int doc) throws IOException;
 
+  /**
+   * Optionally creates a view of the scorerIterator where only competitive documents
+   * in the scorerIterator are kept and non-competitive are skipped.
+   *
+   * Collectors should delegate this method to their comparators if
+   * their comparators provide the skipping functionality over non-competitive docs.
+   * The default is to return the same iterator which is interpreted as the collector doesn't filter any documents.
+   */
+  default DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return scorerIterator;
+  }
 
 Review comment:
   This allows for some hacks like returning an iterator that matches more docs than the scorer. I liked the previous approach that returned an iterator better.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I will keep thinking how to organize it better.
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410691342
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -201,19 +202,20 @@ public long cost() {
     @Override
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
+      DocIdSetIterator scorerIterator = twoPhase == null? iterator : twoPhase.approximation();
 
 Review comment:
   ```suggestion
         DocIdSetIterator scorerIterator = twoPhase == null ? iterator : twoPhase.approximation();
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408888656
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,16 @@
    */
   void collect(int doc) throws IOException;
 
+  /**
+   * Optionally creates a view of the scorerIterator where only competitive documents
+   * in the scorerIterator are kept and non-competitive are skipped.
+   *
+   * Collectors should delegate this method to their comparators if
+   * their comparators provide the skipping functionality over non-competitive docs.
+   * The default is to return the same iterator which is interpreted as the collector doesn't filter any documents.
+   */
+  default DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return scorerIterator;
+  }
 
 Review comment:
   Oh that's a good point. +1 to just return an iterator based on the comparator, and do the conjuncion/combination in `BulkScorer`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407524764
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -24,37 +24,53 @@
   /**
    * Produced scorers will allow visiting all matches and get their score.
    */
-  COMPLETE {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  },
+  COMPLETE(true, true),
 
   /**
    * Produced scorers will allow visiting all matches but scores won't be
    * available.
    */
-  COMPLETE_NO_SCORES {
-    @Override
-    public boolean needsScores() {
-      return false;
-    }
-  },
+  COMPLETE_NO_SCORES(true, false),
 
   /**
    * Produced scorers will optionally allow skipping over non-competitive
    * hits using the {@link Scorer#setMinCompetitiveScore(float)} API.
    */
-  TOP_SCORES {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  };
+  TOP_SCORES(false, true),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS(false, false),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES(false, true);
 
 Review comment:
   Do we need to distinguish between `TOP_SCORES` and `TOP_DOCS_WITH_SCORES`?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394663323
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -249,6 +249,10 @@ static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPha
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase, Bits acceptDocs) throws IOException {
       if (twoPhase == null) {
+        if (collector.iterator() != null) {
 
 Review comment:
   @jimczi Thanks for the initial review.   
   
   >  I wonder how this would look like if we build a conjunction from the main query and the collector iterator directly when building the weight (in IndexSearcher) 
   
   I could not think of any clever way  to do this in `IndexSearcher`,  I would appreciate your help if you can suggest any such way. I just redesigned `DefaultBulkScorer` to use a conjunction of a scorer's and collector's iterators.
   
   I looked at classes that override `BulkScorer` and many of them still refer to a default `BulkScorer`, and for those that don't such as `BooleanScorer` I found its logic to be too complex to combine with  a collector's iterator. 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-605327672
 
 
   @msokolov Thank for suggesting additional benchmarks that we can use.
   Below are the results on the dataset `wikimedium10m`.
   
   First I will repeat the results from the previous round of benchmarking:
   
   topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   
   
   ---
     topN=10, **taskRepeatCount = 500**, concurrentSearchers = False
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       184.60 |    (8.2%) |                 3046.19 |    (4.4%) |
   | HighTermMonthSort     |       209.43 |    (6.5%) |                  253.90 |   (10.5%) |
   | HighTermDayOfYearSort |       130.97 |    (5.8%) |                   73.25 |   (11.8%) |
   
   This seemed to speed up all operations, and here the speedups for `TermDTSort` even bigger: 16.5x times. There is also seems to be more regression for `HighTermDayOfYearSort`.
   
   ---
     **topN=500**,  taskRepeatCount = 20, concurrentSearchers = False
   
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       210.24 |    (9.7%) |                  537.65 |    (6.7%) |
   | HighTermMonthSort     |       116.02 |    (8.9%) |                  189.96 |   (13.5%) |
   | HighTermDayOfYearSort |        42.33 |    (7.6%) |                   67.93 |    (9.3%) |
   
   With increased `topN` the sort optimization has less speedups up to 2x, as it is expected as it will be possible to run it only after collecting `topN` docs.
   
   ---
   topN=10, taskRepeatCount = 20, **concurrentSearchers = True**
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       132.09 |   (14.3%) |                  287.93 |   (11.8%) |
   | HighTermMonthSort     |       211.01 |   (12.2%) |                  116.46 |    (7.1%) |
   | HighTermDayOfYearSort |        72.28 |    (6.1%) |                   68.21 |   (11.4%) |
   
   With the concurrent searchers the speedups are also smaller up to 2x. This is expected as now segments are spread between several TopFieldCollects/Comparators and they don't exchange bottom values.  As a follow-up on this PR, we can think how we can have a global bottom value similar how `MaxScoreAccumulator` is used to set up a global competitive min score. 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394663323
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -249,6 +249,10 @@ static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPha
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase, Bits acceptDocs) throws IOException {
       if (twoPhase == null) {
+        if (collector.iterator() != null) {
 
 Review comment:
   @jimczi Thanks for the initial review.   
   
   >  I wonder how this would look like if we build a conjunction from the main query and the collector iterator directly when building the weight (in IndexSearcher) 
   
   I could not think of any clever way  to do this in `IndexSearcher`,  I would appreciate your help if you can suggest any such way. I just redesigned `DefaultBulkScorer` to use a conjunction of a scorer's and collector's iterators.
   
   I looked at classes that override `BulkScorer` and many of them still refer to a default `BulkScorer`, and for those that don't such as `BooleanScorer` I found its logic to be too complex to understand and for me to combine with  a collector's iterator. 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410690894
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -24,37 +24,53 @@
   /**
    * Produced scorers will allow visiting all matches and get their score.
    */
-  COMPLETE {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  },
+  COMPLETE(true, true),
 
   /**
    * Produced scorers will allow visiting all matches but scores won't be
    * available.
    */
-  COMPLETE_NO_SCORES {
-    @Override
-    public boolean needsScores() {
-      return false;
-    }
-  },
+  COMPLETE_NO_SCORES(true, false),
 
   /**
    * Produced scorers will optionally allow skipping over non-competitive
    * hits using the {@link Scorer#setMinCompetitiveScore(float)} API.
    */
-  TOP_SCORES {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  };
+  TOP_SCORES(false, true),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS(false, false),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES(false, true);
 
 Review comment:
   We need both because the first one sorts by score first and should use e.g. block-max WAND while the latter sorts by field so block-max WAND isn't relevant, but we still need to disable bulk scoring. `needsScores` and `isExhaustive` are not complete descriptions of these enum constants.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408712107
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,16 @@
    */
   void collect(int doc) throws IOException;
 
+  /**
+   * Optionally creates a view of the scorerIterator where only competitive documents
+   * in the scorerIterator are kept and non-competitive are skipped.
+   *
+   * Collectors should delegate this method to their comparators if
+   * their comparators provide the skipping functionality over non-competitive docs.
+   * The default is to return the same iterator which is interpreted as the collector doesn't filter any documents.
+   */
+  default DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return scorerIterator;
+  }
 
 Review comment:
   This allows for some hacks like returning an iterator that matches more hacks than the scorer. I liked the previous approach that returned an iterator better.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-605327672
 
 
   Update: these are wrong results.  Please disregard them
   
   
   @msokolov Thank for suggesting additional benchmarks that we can use.
   Below are the results on the dataset `wikimedium10m`.
   
   First I will repeat the results from the previous round of benchmarking:
   
   topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   
   
   ---
     topN=10, **taskRepeatCount = 500**, concurrentSearchers = False
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       184.60 |    (8.2%) |                 3046.19 |    (4.4%) |
   | HighTermMonthSort     |       209.43 |    (6.5%) |                  253.90 |   (10.5%) |
   | HighTermDayOfYearSort |       130.97 |    (5.8%) |                   73.25 |   (11.8%) |
   
   This seemed to speed up all operations, and here the speedups for `TermDTSort` even bigger: 16.5x times. There is also seems to be more regression for `HighTermDayOfYearSort`.
   
   ---
     **topN=500**,  taskRepeatCount = 20, concurrentSearchers = False
   
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       210.24 |    (9.7%) |                  537.65 |    (6.7%) |
   | HighTermMonthSort     |       116.02 |    (8.9%) |                  189.96 |   (13.5%) |
   | HighTermDayOfYearSort |        42.33 |    (7.6%) |                   67.93 |    (9.3%) |
   
   With increased `topN` the sort optimization has less speedups up to 2x, as it is expected as it will be possible to run it only after collecting `topN` docs.
   
   ---
   topN=10, taskRepeatCount = 20, **concurrentSearchers = True**
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       132.09 |   (14.3%) |                  287.93 |   (11.8%) |
   | HighTermMonthSort     |       211.01 |   (12.2%) |                  116.46 |    (7.1%) |
   | HighTermDayOfYearSort |        72.28 |    (6.1%) |                   68.21 |   (11.4%) |
   
   With the concurrent searchers the speedups are also smaller up to 2x. This is expected as now segments are spread between several TopFieldCollects/Comparators and they don't exchange bottom values.  As a follow-up on this PR, we can think how we can have a global bottom value similar how `MaxScoreAccumulator` is used to set up a global competitive min score. 
   
   ---
   with **indexSort='lastModNDV:long'**  topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       321.75 |   (11.5%) |                  364.83 |    (7.8%) |
   | HighTermMonthSort     |       205.20 |    (5.7%) |                  178.16 |    (7.8%) |
   | HighTermDayOfYearSort |        66.07 |   (12.0%) |                   58.84 |    (9.3%) |

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-614931785
 
 
   I have run another round of benchmarks, this time comparing the performance of this PR VS master as we don't need any special sort field.  [Here](https://github.com/mayya-sharipova/luceneutil/commit/c3166e4fc44e7fcddcd1672112c96364d9f464e5) are the changes made to luceneutil.
   
   
   **wikimedium10m**
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
      HighTermDayOfYearSort       50.93      (5.6%)       49.31     (10.9%)   -3.2% ( -18% -   14%)
                 TermDTSort       83.37      (5.9%)      129.95     (41.2%)   55.9% (   8% -  109%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 541957 vs 541957+
   WARNING: cat=TermDTSort: hit counts differ: 506054 vs 1861+
   ```
   Here we have two sorts:
   -  Int sort on a day of year. Slight decrease of performance: -3.2%. There was an attempt to do the optimization, but the optimization was eventually not run as every time [estimatedNumberOfMatches](https://github.com/apache/lucene-solr/pull/1351/files#diff-aff67e212aa0edd675ec31c068cb642bR268) was not selective enough. The reason for that the data here a day of the year in the range [1, 366], and all segments contain various values through a segment.
   
   - Long sort on date field (msecSinceEpoch).  Speedups: 55.9%.   
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I will keep thinking how to organize it better
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407523929
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.io.IOException;
+
+public class TestSortOptimization extends LuceneTestCase {
+
+  public void testLongSortOptimization() throws IOException {
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    final int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      final Document doc = new Document();
+      doc.add(new NumericDocValuesField("my_field", i));
+      doc.add(new LongPoint("my_field", i));
+      writer.addDocument(doc);
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    final SortField sortField = new SortField("my_field", SortField.Type.LONG);
+    sortField.allowFilterNonCompetitveDocs();
 
 Review comment:
   I'm a bit wary of this, as I don't think we should be mutating objects passed in to the API that may well be shared (it's already a bit sketchy that SortField doesn't have final fields, mainly due to how the constructors are implemented - also the case for `Sort`, which I think we should make immutable as a follow-up).  There are several specialised implementations of SortField elsewhere in the codebase, so I don't think it should be too hard to create a new SkippingSortField that takes another SortField in its constructor and copies the relevant field, type, etc data from that?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-608437953
 
 
   I like the idea of wrapping things up, and I think we may be able to take this further by pushing more of the logic into the comparator:
   * add a `wrapDocIdSetIterator(DocIdSetIterator in)` method to `LeafCollector` that by default returns the passed-in iterator.  This gets called in `DefaultBulkScorer#score` to wrap the iterator for a query.
   * add a `wrapDocIdSetIterator(DocIdSetIterator in)` method to `FieldComparator` that by default returns the passed-in iterator.  `TopFieldCollector` delegates its `wrapDocIdSetIterator` method to this method on its first comparator.  This allows us to completely contain the logic that combines a query's iterator with sorting shortcuts to the `SortField` and associated `FieldComparator` implementation.
   * Move the logic that checks whether or not to update the iterator into `setBottom` on the leaf comparator.  I know this involves passing the `HitsThresholdChecker` into the leaf comparator constructor, but I think that's reasonable if the point of this API change is to make it possible for comparators to skip hits

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-606259561
 
 
   @msokolov  Sorry again for reporting incorrect benchmarking results. Below are are my latest results, and I feel  quite confident in their correctness.
   
   First about the benchmarking setup.
   1. [Here](https://github.com/mayya-sharipova/luceneutil/commit/e0d86b24053cc8a68796abd9f0fd08dbac899779)  are the changes made to `luceneutil`
   2.  `patch` folder is checkout as this PR
   3. `trunk` folder is checkout as this PR as well with a modification.  As there is no `LongDocValuesPointSortField` in master, I can't benchmark [sorting using this field](https://github.com/mayya-sharipova/luceneutil/commit/e0d86b24053cc8a68796abd9f0fd08dbac899779#diff-58e50bb4a8f0be480df656bcd84d5b77R76) on master. What I did is just is on `trunk` 
   folder delegated sorting to the traditional sorting on a long field like this:
   ```java
   public class LongDocValuesPointSortField extends SortField {
       public LongDocValuesPointSortField(String field) {
           super(field, SortField.Type.LONG);
       }
       public LongDocValuesPointSortField(String field, boolean reverse) {
           super(field, SortField.Type.LONG, reverse);
       }
   }
   ```
   So basically I was benchmarking a traditional long sort VS a long sort using a new field `LongDocValuesPointSortField`.
   
   
   wikimedium10m: 10 millon docs, up to 2x speedups
   
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
                TermDTSort       64.53      (6.4%)      155.29     (42.3%)  140.7% (  86% -  202%)
     HighTermDayOfYearSort       47.63      (5.4%)       50.47      (6.8%)    6.0% (  -5% -   19%)
          HighTermMonthSort      110.07     (7.3%)      121.13      (6.8%)   10.0% (  -3% -   26%)
   WARNING: cat=TermDTSort: hit counts differ: 754451 vs 1669+
   ```
   
   wikimediumall: about 33 million docs, up to 3.5 x speedups
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
                 TermDTSort       28.96      (4.3%)      108.45     (56.9%)  274.5% ( 204% -  350%)
      HighTermDayOfYearSort        9.69      (5.1%)        9.56      (6.1%)   -1.3% ( -11% -   10%)
          HighTermMonthSort       39.41      (4.7%)       47.99     (10.0%)   21.8% (   6% -   38%)
   WARNING: cat=TermDTSort: hit counts differ: 1474717 vs 1070+
   ```
   
   Please let me know if these results and methodology make sense.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407525287
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -30,6 +30,7 @@
 import org.apache.lucene.search.MaxScoreAccumulator.DocAndScore;
 import org.apache.lucene.search.TotalHits.Relation;
 
+
 
 Review comment:
   nit: extra line

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-614931785
 
 
   I have run another round of benchmarks, this time comparing the performance of this PR VS master as we don't need any special sort field.  [Here](https://github.com/mayya-sharipova/luceneutil/commit/c3166e4fc44e7fcddcd1672112c96364d9f464e5) are the changes made to luceneutil.
   
   
   **wikimedium10m**: 10 millon docs
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
      HighTermDayOfYearSort       50.93      (5.6%)       49.31     (10.9%)   -3.2% ( -18% -   14%)
                 TermDTSort       83.37      (5.9%)      129.95     (41.2%)   55.9% (   8% -  109%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 541957 vs 541957+
   WARNING: cat=TermDTSort: hit counts differ: 506054 vs 1861+
   ```
   
   **wikimediumall**: about 33 million docs
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
      HighTermDayOfYearSort       23.37      (4.4%)       21.76      (8.8%)   -6.9% ( -19% -    6%)
                 TermDTSort       31.86      (3.5%)      108.33     (49.6%)  240.0% ( 180% -  303%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 1275574 vs 1275574+
   WARNING: cat=TermDTSort: hit counts differ: 1474717 vs 1070+
   ```
   
   Here we have two sorts:
   -  Int sort on a day of year. Slight decrease of performance: **-6.9% – -3.2%,**. There was an attempt to do the optimization, but the optimization was eventually not run as every time [estimatedNumberOfMatches](https://github.com/apache/lucene-solr/pull/1351/files#diff-aff67e212aa0edd675ec31c068cb642bR268) was not selective enough. The reason for that the data here a day of the year in the range [1, 366], and all segments contain various values through a segment, so this data is not really a target for optimization.
   
   - Long sort on date field (msecSinceEpoch).  Speedups: **55.9% – 240.0%**.   
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394655989
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
 
 Review comment:
   Addressed in 6384b15

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407108607
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.io.IOException;
+
+public class TestSortOptimization extends LuceneTestCase {
+
+  public void testLongSortOptimization() throws IOException {
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    final int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      final Document doc = new Document();
+      doc.add(new NumericDocValuesField("my_field", i));
+      doc.add(new LongPoint("my_field", i));
+      writer.addDocument(doc);
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    final SortField sortField = new SortField("my_field", SortField.Type.LONG);
+    sortField.allowFilterNonCompetitveDocs();
 
 Review comment:
   @romseygeek  Thanks for the suggestion.  It is difficult to wrap `SortField` into another class that extends `SortField`, as `SortField` is a concrete class with several constructors. 
   
   Instead, I  am setting `sort.fields[0].allowFilterNonCompetitveDocs()` when we create a `TopFieldCollector` with a set hitsTreshold.   
   
    Partially addressed in 719882e

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410692812
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
+
+    public FilteringFieldComparator(FieldComparator<T> in) {
+        this.in = in;
+    }
+
+    protected abstract void setCanUpdateIterator() throws IOException;
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return in.compare(slot1, slot2);
+    }
+
+    @Override
+    public T value(int slot) {
+        return in.value(slot);
+    }
+
+    @Override
+    public void setTopValue(T value) {
+        in.setTopValue(value);
+    }
+
+    @Override
+    public int compareValues(T first, T second) {
+        return in.compareValues(first, second);
+    }
+
+    /**
+     * Returns an iterator over competitive documents
+     */
+    public DocIdSetIterator competitiveIterator() {
+        if (iterator == null) return null;
+        return new DocIdSetIterator() {
+            private int doc;
+            @Override
+            public int nextDoc() throws IOException {
+                return doc = iterator.nextDoc();
+            }
+
+            @Override
+            public int docID() {
+                return doc;
+            }
+
+            @Override
+            public long cost() {
+                return iterator.cost();
+            }
+
+            @Override
+            public int advance(int target) throws IOException {
+                return doc = iterator.advance(target);
+            }
+        };
+    }
+
+    /**
+     * Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
+     * If for the given comparator the skip functionality is not implemented, return the comparator itself.
+     */
+    public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse) {
+        if (comparator instanceof FieldComparator.LongComparator){
+            return new FilteringFieldComparator.FilteringLongComparator((FieldComparator.LongComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.IntComparator){
+            return new FilteringFieldComparator.FilteringIntComparator((FieldComparator.IntComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.DoubleComparator){
+            return new FilteringFieldComparator.FilteringDoubleComparator((FieldComparator.DoubleComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.FloatComparator){
+            return new FilteringFieldComparator.FilteringFloatComparator((FieldComparator.FloatComparator) comparator, reverse);
+        }
+        return comparator;
+    }
+
+    /**
+     * A wrapper over {@code NumericComparator} that adds a functionality to filter non-competitive docs.
+     */
+    public static abstract class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> implements LeafFieldComparator {
+        private final boolean reverse;
+        private boolean hasTopValue = false;
+        private PointValues pointValues;
+        private final int bytesCount;
+        private final byte[] minValueAsBytes;
+        private final byte[] maxValueAsBytes;
+        private boolean minValueExist = false;
+        private boolean maxValueExist = false;
+        private int maxDoc;
+        private int maxDocVisited;
+        private int updateCounter = 0;
+        private final String field;
+        protected boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
+
+        public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, int bytesCount) {
+            super(in);
+            this.field = in.field;
+            this.bytesCount = bytesCount;
+            this.reverse = reverse;
+            minValueAsBytes = new byte[bytesCount];
+            maxValueAsBytes = new byte[bytesCount];
+            if (reverse) {
+                minValueExist = true;
+            } else {
+                maxValueExist = true;
+            }
+        }
+
+        @Override
+        public void setCanUpdateIterator() throws IOException {
+            this.canUpdateIterator = true;
+            // for the 1st time queue becomes full and hitsThreshold is reached
+            // we can start updating competitive iterator
+            updateCompetitiveIterator();
+        }
+
+        @Override
+        public void setTopValue(T value) {
+            hasTopValue = true;
+            if (reverse) {
+                maxValueExist = true;
+            } else {
+                minValueExist = true;
+            }
+            in.setTopValue(value);
+        }
+
+        @Override
+        public void setBottom(int slot) throws IOException {
+            ((NumericComparator) in).setBottom(slot);
+            updateCompetitiveIterator(); // update an iterator if we set a new bottom
+        }
+
+        @Override
+        public int compareBottom(int doc) throws IOException {
+            return ((NumericComparator) in).compareBottom(doc);
+        }
+
+        @Override
+        public int compareTop(int doc) throws IOException {
+            return ((NumericComparator) in).compareTop(doc);
+        }
+
+        @Override
+        public void copy(int slot, int doc) throws IOException {
+            ((NumericComparator) in).copy(slot, doc);
+        }
+
+        @Override
+        public void setScorer(Scorable scorer) throws IOException {}
+
+        @Override
+        public final LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+            ((NumericComparator) in).doSetNextReader(context);
+            pointValues = context.reader().getPointValues(field);
+            iterator = pointValues == null ? null : ((NumericComparator)in).currentReaderValues;
+            maxDoc = context.reader().maxDoc();
+            maxDocVisited = 0;
+            updateCompetitiveIterator(); // update an iterator if we have a new segment
+            return this;
+        }
+
+        // update its iterator to include possibly only docs that are "stronger" than the current bottom entry
+        public void updateCompetitiveIterator() throws IOException {
+            if (canUpdateIterator == false) return;
+            if (pointValues == null) return;
+            updateCounter++;
+            if (updateCounter > 256 && (updateCounter & 0x1f) != 0x1f) { // Start sampling if we get called too much
+                return;
+            }
+            if (reverse == false) {
+                encodeBottom(maxValueAsBytes);
+                if (hasTopValue) {
+                    encodeTop(minValueAsBytes);
+                }
+            } else {
+                encodeBottom(minValueAsBytes);
+                if (hasTopValue) {
+                    encodeTop(maxValueAsBytes);
+                }
+            };
+            DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
+            PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
+                DocIdSetBuilder.BulkAdder adder;
+                @Override
+                public void grow(int count) {
+                    adder = result.grow(count);
+                }
+
+                @Override
+                public void visit(int docID) {
+                    if (docID <= maxDocVisited) {
+                        return; // Already visited or skipped
+                    }
+                    adder.add(docID);
+                }
+
+                @Override
+                public void visit(int docID, byte[] packedValue) {
+                    if (docID <= maxDocVisited) {
+                        return;  // Already visited or skipped
+                    }
+                    if (maxValueExist) {
+                        // doc's value is too high
+                        if (Arrays.compareUnsigned(packedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount) > 0) return;
+                    }
+                    if (minValueExist) {
+                        // doc's value is too low
+                        if (Arrays.compareUnsigned(packedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount) < 0) return;
+                    }
+                    adder.add(docID); // doc is competitive
+                }
+
+                @Override
+                public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                    if ((maxValueExist && Arrays.compareUnsigned(minPackedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount) > 0) ||
+                            (minValueExist && Arrays.compareUnsigned(maxPackedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount) < 0)) {
+                        return PointValues.Relation.CELL_OUTSIDE_QUERY;
+                    }
+                    if ((maxValueExist && Arrays.compareUnsigned(maxPackedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount) > 0) ||
+                            (minValueExist && Arrays.compareUnsigned(minPackedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount) < 0)) {
+                        return PointValues.Relation.CELL_CROSSES_QUERY;
+                    }
+                    return PointValues.Relation.CELL_INSIDE_QUERY;
+                }
+            };
+            final long threshold = iterator.cost() >>> 3;
+            long estimatedNumberOfMatches = pointValues.estimatePointCount(visitor); // runs in O(log(numPoints))
+            if (estimatedNumberOfMatches >= threshold) {
+                // the new range is not selective enough to be worth materializing, it doesn't reduce number of docs at least 8x
+                return;
+            }
+            pointValues.intersect(visitor);
+            this.iterator = result.build().iterator();
 
 Review comment:
   I believe that doing it is incorrect if a missing value is configured and would be competitive.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r406046648
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -302,20 +312,27 @@ private TopFieldCollector(FieldValueHitQueue<Entry> pq, int numHits,
     this.numHits = numHits;
     this.hitsThresholdChecker = hitsThresholdChecker;
     this.numComparators = pq.getComparators().length;
-    FieldComparator<?> fieldComparator = pq.getComparators()[0];
+    FieldComparator<?> firstComparator = pq.getComparators()[0];
     int reverseMul = pq.reverseMul[0];
-    if (fieldComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
+    if (firstComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
           && reverseMul == 1 // if the natural sort is preserved (sort by descending relevance)
           && hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
-      firstComparator = (FieldComparator.RelevanceComparator) fieldComparator;
+      relevanceComparator = (FieldComparator.RelevanceComparator) firstComparator;
       scoreMode = ScoreMode.TOP_SCORES;
       canSetMinScore = true;
     } else {
-      firstComparator = null;
+      relevanceComparator = null;
       scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
       canSetMinScore = false;
     }
     this.minScoreAcc = minScoreAcc;
+
+    if ((firstComparator instanceof FilteringFieldComparator) && (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE)) {
 
 Review comment:
   Can we merge this with the  `if` statement immediately above it? That way `scoreMode` can stay final

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395284073
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -223,44 +257,23 @@ public int score(LeafCollector collector, Bits acceptDocs, int min, int max) thr
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase,
         Bits acceptDocs, int currentDoc, int end) throws IOException {
-      if (twoPhase == null) {
-        while (currentDoc < end) {
-          if (acceptDocs == null || acceptDocs.get(currentDoc)) {
-            collector.collect(currentDoc);
-          }
-          currentDoc = iterator.nextDoc();
-        }
-        return currentDoc;
-      } else {
-        final DocIdSetIterator approximation = twoPhase.approximation();
-        while (currentDoc < end) {
-          if ((acceptDocs == null || acceptDocs.get(currentDoc)) && twoPhase.matches()) {
-            collector.collect(currentDoc);
-          }
-          currentDoc = approximation.nextDoc();
+      while (currentDoc < end) {
+        if ((acceptDocs == null || acceptDocs.get(currentDoc)) && (twoPhase == null || twoPhase.matches())) {
+          collector.collect(currentDoc);
         }
-        return currentDoc;
+        currentDoc = iterator.nextDoc();
       }
+      return currentDoc;
 
 Review comment:
   Addressed in https://github.com/apache/lucene-solr/pull/1351/commits/d732d7eb9de67a597f67e91c9774104aa055e293

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r396687134
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   @msokolov  Thanks for the suggestion, naming is tough, addressed in 95e1bc1.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615261001
 
 
   Sorry for bringing this up and not finishing, but I thought that is also worth to report the test results on a smaller collection `wikimedium1m`:
   
   ```
    TaskQPS                     baseline   StdDevQPS     patch     StdDev    Pct diff
                 TermDTSort      292.71     (15.1%)       59.60      (4.9%)  -79.6% ( -86% -  -70%)
      HighTermDayOfYearSort       60.01     (44.0%)       33.75     (13.6%)  -43.8% ( -70% -   24%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 65216 vs 65093+
   WARNING: cat=TermDTSort: hit counts differ: 68644 vs 507+
   ```
   
   Here there is a substantial reduction in performance by using the proposed sort optimization.  
   
   As the data in these indexes are not monotonically increasing `setBottom` is called many times.  
   Looks like for smaller indexes (especially with data that is not monotonically increasing) it is faster just to do the conventional sort than the proposed optimization.  
   
   I am not sure how significant is this reduction. 
   - **Should we apply the optimization only for segments over 1 million docs?**
   - **Should we apply the optimization only when the data is diverse enough?**
   
   Or we can follow up on these proposals in subsequent PRs?
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407108607
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.io.IOException;
+
+public class TestSortOptimization extends LuceneTestCase {
+
+  public void testLongSortOptimization() throws IOException {
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    final int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      final Document doc = new Document();
+      doc.add(new NumericDocValuesField("my_field", i));
+      doc.add(new LongPoint("my_field", i));
+      writer.addDocument(doc);
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    final SortField sortField = new SortField("my_field", SortField.Type.LONG);
+    sortField.allowFilterNonCompetitveDocs();
 
 Review comment:
   @romseygeek  Thanks for the suggestion.  It makes sense.
    It is difficult to wrap `SortField` into another class that extends `SortField`, as `SortField` is a concrete class with several constructors. 
   
   Instead, I  am setting `sort.fields[0].allowFilterNonCompetitveDocs()` when we create a `TopFieldCollector` with a set hitsTreshold.  What do you think of this implementation?
   
    Partially addressed in 719882e

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mikemccand commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615484825
 
 
   > I.e., it looks like (for these benchmark queries anyways) the optimization did not wind up skipping any hits (though, it thought it may have, hence the added +)?
   
   OK, I see in some of the other runs above, that the optimization does sometimes reduce the actual number of hits visited, nice!  E.g.:
   
   ```
   WARNING: cat=TermDTSort: hit counts differ: 754451 vs 1669+
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410689886
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
+
+    public FilteringFieldComparator(FieldComparator<T> in) {
+        this.in = in;
+    }
+
+    protected abstract void setCanUpdateIterator() throws IOException;
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return in.compare(slot1, slot2);
+    }
+
+    @Override
+    public T value(int slot) {
+        return in.value(slot);
+    }
+
+    @Override
+    public void setTopValue(T value) {
+        in.setTopValue(value);
+    }
+
+    @Override
+    public int compareValues(T first, T second) {
+        return in.compareValues(first, second);
+    }
+
+    /**
+     * Returns an iterator over competitive documents
+     */
+    public DocIdSetIterator competitiveIterator() {
+        if (iterator == null) return null;
+        return new DocIdSetIterator() {
+            private int doc;
+            @Override
+            public int nextDoc() throws IOException {
+                return doc = iterator.nextDoc();
+            }
+
+            @Override
+            public int docID() {
+                return doc;
+            }
+
+            @Override
+            public long cost() {
+                return iterator.cost();
+            }
+
+            @Override
+            public int advance(int target) throws IOException {
+                return doc = iterator.advance(target);
+            }
+        };
+    }
+
+    /**
+     * Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
+     * If for the given comparator the skip functionality is not implemented, return the comparator itself.
+     */
+    public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse) {
+        if (comparator instanceof FieldComparator.LongComparator){
+            return new FilteringFieldComparator.FilteringLongComparator((FieldComparator.LongComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.IntComparator){
+            return new FilteringFieldComparator.FilteringIntComparator((FieldComparator.IntComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.DoubleComparator){
+            return new FilteringFieldComparator.FilteringDoubleComparator((FieldComparator.DoubleComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.FloatComparator){
+            return new FilteringFieldComparator.FilteringFloatComparator((FieldComparator.FloatComparator) comparator, reverse);
+        }
+        return comparator;
+    }
+
+    /**
+     * A wrapper over {@code NumericComparator} that adds a functionality to filter non-competitive docs.
+     */
+    public static abstract class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> implements LeafFieldComparator {
+        private final boolean reverse;
+        private boolean hasTopValue = false;
+        private PointValues pointValues;
+        private final int bytesCount;
+        private final byte[] minValueAsBytes;
+        private final byte[] maxValueAsBytes;
+        private boolean minValueExist = false;
+        private boolean maxValueExist = false;
+        private int maxDoc;
+        private int maxDocVisited;
+        private int updateCounter = 0;
+        private final String field;
+        protected boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
+
+        public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, int bytesCount) {
+            super(in);
+            this.field = in.field;
+            this.bytesCount = bytesCount;
+            this.reverse = reverse;
+            minValueAsBytes = new byte[bytesCount];
+            maxValueAsBytes = new byte[bytesCount];
+            if (reverse) {
+                minValueExist = true;
+            } else {
+                maxValueExist = true;
+            }
+        }
+
+        @Override
+        public void setCanUpdateIterator() throws IOException {
+            this.canUpdateIterator = true;
+            // for the 1st time queue becomes full and hitsThreshold is reached
+            // we can start updating competitive iterator
+            updateCompetitiveIterator();
+        }
+
+        @Override
+        public void setTopValue(T value) {
+            hasTopValue = true;
+            if (reverse) {
+                maxValueExist = true;
+            } else {
+                minValueExist = true;
+            }
+            in.setTopValue(value);
+        }
+
+        @Override
+        public void setBottom(int slot) throws IOException {
+            ((NumericComparator) in).setBottom(slot);
+            updateCompetitiveIterator(); // update an iterator if we set a new bottom
+        }
+
+        @Override
+        public int compareBottom(int doc) throws IOException {
+            return ((NumericComparator) in).compareBottom(doc);
+        }
+
+        @Override
+        public int compareTop(int doc) throws IOException {
+            return ((NumericComparator) in).compareTop(doc);
+        }
+
+        @Override
+        public void copy(int slot, int doc) throws IOException {
+            ((NumericComparator) in).copy(slot, doc);
+        }
+
+        @Override
+        public void setScorer(Scorable scorer) throws IOException {}
+
+        @Override
+        public final LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+            ((NumericComparator) in).doSetNextReader(context);
+            pointValues = context.reader().getPointValues(field);
+            iterator = pointValues == null ? null : ((NumericComparator)in).currentReaderValues;
+            maxDoc = context.reader().maxDoc();
+            maxDocVisited = 0;
+            updateCompetitiveIterator(); // update an iterator if we have a new segment
+            return this;
+        }
+
+        // update its iterator to include possibly only docs that are "stronger" than the current bottom entry
+        public void updateCompetitiveIterator() throws IOException {
+            if (canUpdateIterator == false) return;
+            if (pointValues == null) return;
 
 Review comment:
   I believe that this `pointValues == null` aims at disabling the optimization when the field has doc values but is not indexed. However it can also be null if the field is indexed but doesn't have values on this segment. It's fine to not optimize this case right now, but let's at least leave a comment about it?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-605327672
 
 
   @msokolov Thank for suggesting additional benchmarks that we can use.
   Below are the results on the dataset `wikimedium10m`.
   
   First I will repeat the results from the previous round of benchmarking:
   
   topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   
   
   ---
     topN=10, **taskRepeatCount = 500**, concurrentSearchers = False
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       184.60 |    (8.2%) |                 3046.19 |    (4.4%) |
   | HighTermMonthSort     |       209.43 |    (6.5%) |                  253.90 |   (10.5%) |
   | HighTermDayOfYearSort |       130.97 |    (5.8%) |                   73.25 |   (11.8%) |
   
   This seemed to speed up all operations, and here the speedups for `TermDTSort` even bigger: 16.5x times. There is also seems to be more regression for `HighTermDayOfYearSort`.
   
   ---
     **topN=500**,  taskRepeatCount = 20, concurrentSearchers = False
   
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       210.24 |    (9.7%) |                  537.65 |    (6.7%) |
   | HighTermMonthSort     |       116.02 |    (8.9%) |                  189.96 |   (13.5%) |
   | HighTermDayOfYearSort |        42.33 |    (7.6%) |                   67.93 |    (9.3%) |
   
   With increased `topN` the sort optimization has less speedups up to 2x, as it is expected as it will be possible to run it only after collecting `topN` docs.
   
   ---
   topN=10, taskRepeatCount = 20, **concurrentSearchers = True**
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       132.09 |   (14.3%) |                  287.93 |   (11.8%) |
   | HighTermMonthSort     |       211.01 |   (12.2%) |                  116.46 |    (7.1%) |
   | HighTermDayOfYearSort |        72.28 |    (6.1%) |                   68.21 |   (11.4%) |
   
   With the concurrent searchers the speedups are also smaller up to 2x. This is expected as now segments are spread between several TopFieldCollects/Comparators and they don't exchange bottom values.  As a follow-up on this PR, we can think how we can have a global bottom value similar how `MaxScoreAccumulator` is used to set up a global competitive min score. 
   
   ---
   with **indexSort='lastModNDV:long'**  topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       321.75 |   (11.5%) |                  364.83 |    (7.8%) |
   | HighTermMonthSort     |       205.20 |    (5.7%) |                  178.16 |    (7.8%) |
   | HighTermDayOfYearSort |        66.07 |   (12.0%) |                   58.84 |    (9.3%) |

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
msokolov commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-606047977
 
 
   @mayya-sharipova sounds good - I'd also encourage you to post a PR with your modifications to luceneutil

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410688678
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -432,7 +461,9 @@ static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
       throw new IllegalArgumentException("hitsThresholdChecker should not be null");
     }
 
-    FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
+    // here we assume that if hitsThreshold was set, we let a comparator to skip non-competitive docs
+    boolean filterNonCompetitveDocs = hitsThresholdChecker.getHitsThreshold() == Integer.MAX_VALUE ? false : true;
 
 Review comment:
   ```suggestion
       boolean filterNonCompetitiveDocs = hitsThresholdChecker.getHitsThreshold() == Integer.MAX_VALUE ? false : true;
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
msokolov commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-604683807
 
 
   That 3x speedup is very nice! My experience with these benchmarks is they can be pretty noisy, maybe accounting for the regressions? I tend to increase comp.taskRepeatCount = 500. I'd also be interested to see how this optimization fares for higher values of topN - I think the default is 10, but you can edit in benchUtil.py. You did not sort the index right (eg: comp.newIndex('baseline', sourceData, facets=facets, indexSort='lastModNDV:long', addDVFields=True)? It would be interesting to see if this has the same impact for sorted index, large N, especially running with an executor (.competitor(...concurrentSearchers = True ). 

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r400741063
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
+   * optionally returns an iterator over competitive documents
 
 Review comment:
   Can you document that the default is to return `null` which Lucene interprets as the collector doesn't filter any documents. It's probably worth making explicit as null iterators are elsewhere interpreted as matching no documents.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407108032
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -51,10 +59,51 @@ public boolean needsScores() {
     public boolean needsScores() {
       return true;
     }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
+  },
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS {
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
+  },
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES {
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
   };
 
   /**
    * Whether this {@link ScoreMode} needs to compute scores.
    */
   public abstract boolean needsScores();
+
+  /**
+   * Returns {@code true} if for this {@link ScoreMode} it is necessary to process all documents,
+   * or {@code false} if is enough to go through top documents only.
+   */
+  public abstract boolean isExhaustive();
 
 Review comment:
   @romseygeek Thanks, a very good suggestion, this indeed made the code more clear and less verbose.  Addressed in 719882e

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-606889736
 
 
   @jpountz  Thank you for the review.
   
   > I wonder whether we could make it easier to write implementations. I haven't spent much time thinking about it, but for instance would it be possible to wrap existing comparators to add the skipping functionality? Alternatively we could add the skipping logic to the existing comparators, but the fact that Lucene doesn't require that the same data be stored in indexes and doc values makes me a bit nervous about enabling it by default, and I'd like to avoid adding a new constructor argument.
   
   Would it make sense  for each numeric FieldComparator to add an extra class that would wrap a numeric comparator and provide additional methods for skipping logic (getting an iterator and updating an iterator)? 

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-611260306
 
 
   @romseygeek  I have tried to address your outstanding feedback in 4448499f0f.  Can you please continue the review when you have time?
   
   > Move the logic that checks whether or not to update the iterator into setBottom on the leaf comparator.
   
   In the new `FilteringFieldComparator` class, the iterator is updated in
   - setBottom
   - when we change a segment in `getLeafComparator`, so that we can also update iterators of subsequent segments. 
   - and also when for the first time queue becomes full and hitsThreshold is reached in `setCanUpdateIterator`, this method is called from a collector.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394663323
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -249,6 +249,10 @@ static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPha
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase, Bits acceptDocs) throws IOException {
       if (twoPhase == null) {
+        if (collector.iterator() != null) {
 
 Review comment:
   @jimczi Thanks for the initial review.   
   
   >  I wonder how this would look like if we build a conjunction from the main query and the collector iterator directly when building the weight (in IndexSearcher) 
   
   I could not think of any clever way  to do this in `IndexSearcher`,  I would appreciate your help if you can suggest any such way.
   
   I looked at classes that override `BulkScorer` and many of them still refer to a default `BulkScorer`, and for those that don't such as `BooleanScorer` I found its logic to be too complex to combine with  a collector's iterator. 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-608059291
 
 
   @jpountz  What do you think of this design in eeb23c11?
   
   1. `IterableFieldComparator` wraps an `FieldComparator` to provide skipping functionality. All numeric  comparators are wrapped in corresponding iterable comparators.
   2.  `SortField` has a new method  `allowSkipNonCompetitveDocs`, that if set will use a comparator that provided skipping functionality.
   
   In this case, we would not need other classes that I previously introduced `LongDocValuesPointComparator` and  `LongDocValuesPointSortField`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mikemccand commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mikemccand commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615483497
 
 
   The `wikimedium1m` corpus is really too small to draw strong conclusions -- I would use it to run a quick performance test, e.g. to see that it can run to completion, not dying with an exception, but then run the real test on `wikimediumall`.
   
   Do you know why you are seeing these warnings?
   
   ```
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 541658 vs 541658+
   WARNING: cat=TermDTSort: hit counts differ: 68644 vs 68644+
   ```
   
   Oh, I guess this is expected, since the optimization may skip some non-competitive matches by skipping forward to find possibly competitive hits based on current PQ bottom?  But then, why is the warning not something more like:
   
   ```
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 541658 vs 5000+
   ```
   
   I.e., it looks like (for these benchmark queries anyways) the optimization did not wind up skipping any hits (though, it thought it may have, hence the added `+`)?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615263686
 
 
   I thought I also report benchmarking results if we apply the optimization only on segments over 1 million docs .  Here we don't have any significant reductions, but also able to achieve speedups.
   
   wikimedium1m
   ```
     TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff
                 TermDTSort      395.99      (8.0%)      360.72     (11.5%)   -8.9% ( -26% -   11%)
      HighTermDayOfYearSort       49.51     (19.8%)       51.95     (14.0%)    4.9% ( -24% -   48%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 541658 vs 541658+
   WARNING: cat=TermDTSort: hit counts differ: 68644 vs 68644+
   ```
   
   wikimedium10m
   ```
     TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff
                 TermDTSort       83.37      (5.1%)      111.73     (30.4%)   34.0% (  -1% -   73%)
      HighTermDayOfYearSort       52.46      (6.9%)       46.76     (12.4%)  -10.9% ( -28% -    9%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 496079 vs 496079+
   WARNING: cat=TermDTSort: hit counts differ: 506054 vs 44560+
   ```
   
   wikimediumall
   ```
     TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff
                 TermDTSort       32.23      (3.2%)       85.28     (26.2%)  164.6% ( 131% -  200%)
      HighTermDayOfYearSort       14.46      (5.0%)       13.93      (6.6%)   -3.7% ( -14% -    8%)
   WARNING: cat=HighTermDayOfYearSort: hit counts differ: 2485178 vs 2485178+
   WARNING: cat=TermDTSort: hit counts differ: 1474717 vs 106400+
   ```
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393627923
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -249,6 +249,10 @@ static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPha
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase, Bits acceptDocs) throws IOException {
       if (twoPhase == null) {
+        if (collector.iterator() != null) {
 
 Review comment:
   I think we should try to handle the collector iterator early in the chain. The default BulkScorer can be overridden so I wonder how this would look like if we build a conjunction from the main query and the collector iterator directly when building the weight (in IndexSearcher) ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-598953853
 
 
   @jimczi I have created a draft PR for comparators and collectors to skip non-competitive docs.  Can you please have a look at it and see if we are happy with this approach. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r401227871
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
+   * optionally returns an iterator over competitive documents
+   */
+  default DocIdSetIterator iterator() {
 
 Review comment:
   Would `competitiveIterator` make sense?  If yes, addressed in 6c628f7

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394656108
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
+        final byte[] maxValueAsBytes = new byte[Long.BYTES];
+        final byte[] minValueAsBytes = new byte[Long.BYTES];
+        if (reverse == false) {
+            LongPoint.encodeDimension(bottom, maxValueAsBytes, 0);
+        } else {
+            LongPoint.encodeDimension(bottom, minValueAsBytes, 0);
+        };
+
 
 Review comment:
   Addressed in 6384b15

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-606042901
 
 
   @msokolov Thank you for an additional  review.  I realized I ran benchmarks incorrectly, not indexing documents with docValues. Sorry, I am still learning lucene benchmarking tool.  Please disregard the previous benchmarking results, I will be rerunning them.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-610078508
 
 
   @romseygeek Thanks for the feedback. I have addressed your comments 1 and 2 in 89d241e.   Indeed, the APIs look simpler, I like them more now.  I just renamed `wrapDocIdSetIterator` to `filterIterator`. 
   
   The comment 3 is challenging to address.  I have already tried to do this in d732d7eb9 as a response to @jimczi  feedback, but I reverted this commit because of those challenges.  `TopFieldCollector` has a lot of subtle logic that makes it difficult to reason and imitate in other classes.  The challenges are following:
   
   1. `HitsThresholdChecker`. First we are passing a not strictly related class `hitsThresholdChecker` to `LeafComparator`.  Secondly, it turned out that we can't use `hitsThresholdChecker.isThresholdReached` method in `setBottom` as it starts to return `true` only after we have already collected  hits  more than `numHits`, but in `setBottom` we need to update an iterator as as soon as we have collected `numHits`, because if there are no competitive docs later `setBottom` will never be called again.
   
   2. `TotalHitsRelation`.  If we end up updating the iterator, we need to set it to `TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO` and it is not clear to me when this should be set.
   
   3. If we have a parallel collector and would like to update a global bottom, it is not clear to me how to do this with this model as well.
   
   I guess I need to think more about it. 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395283880
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -201,20 +201,54 @@ public long cost() {
     @Override
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
+      DocIdSetIterator scorerIterator = twoPhase == null? iterator: twoPhase.approximation();
+      DocIdSetIterator combinedIterator = collector.iterator() == null ? scorerIterator: combineScorerAndCollectorIterators(scorerIterator, collector);
       if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
-        scoreAll(collector, iterator, twoPhase, acceptDocs);
+        scoreAll(collector, combinedIterator, twoPhase, acceptDocs);
         return DocIdSetIterator.NO_MORE_DOCS;
       } else {
         int doc = scorer.docID();
-        if (doc < min) {
-          if (twoPhase == null) {
-            doc = iterator.advance(min);
-          } else {
-            doc = twoPhase.approximation().advance(min);
+        if (doc < min) scorerIterator.advance(min);
 
 Review comment:
   Addressed in https://github.com/apache/lucene-solr/pull/1351/commits/d732d7eb9de67a597f67e91c9774104aa055e293

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615271854
 
 
   Have you tried increasing the sampling rate?  256 might be too low, what about 1024 instead?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-614988778
 
 
   @msokolov  @jimczi @jpountz  I was wondering if you have any other additional comments for this change?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407526518
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -432,6 +462,12 @@ static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
       throw new IllegalArgumentException("hitsThresholdChecker should not be null");
     }
 
+    // here we assume that if hitsThreshold was set, we let the corresponding comparator to skip non-competitive docs
+    // It is beneficial for the 1st field only to skip non-competitive docs
+    if (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
+      sort.fields[0].allowFilterNonCompetitveDocs();
+    }
+
 
 Review comment:
   I think this should be done in `FieldValueHitQueue.create()` below - pass in a boolean to indicate that we want to skip hits if we can, and then do the wrapping/rewriting/whatever it ends up being as a part of building the sort queue

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r400736631
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
 ##########
 @@ -115,7 +115,7 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
       return new ConstantScoreWeight(this, boost) {
         @Override
         public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-          if (scoreMode == ScoreMode.TOP_SCORES) {
+          if (scoreMode == ScoreMode.TOP_SCORES || scoreMode == ScoreMode.TOP_DOCS || scoreMode == ScoreMode.TOP_DOCS_WITH_SCORES) {
 
 Review comment:
   maybe add a `isExhaustive()` method on the enum to avoid these large conditions?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-605327672
 
 
   @msokolov Thank for suggesting additional benchmarks that we can use.
   Below are the results on the dataset `wikimedium10m`.
   
   First I will repeat the results from the previous round of benchmarking:
   
   topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   
   
   ---
     topN=10, **taskRepeatCount = 500**, concurrentSearchers = False
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       184.60 |    (8.2%) |                 3046.19 |    (4.4%) |
   | HighTermMonthSort     |       209.43 |    (6.5%) |                  253.90 |   (10.5%) |
   | HighTermDayOfYearSort |       130.97 |    (5.8%) |                   73.25 |   (11.8%) |
   
   This seemed to speed up all operations, and here the speedups for `TermDTSort` even bigger: 16.5x times. There is also seems to be more regression for `HighTermDayOfYearSort`.
   
   ---
     **topN=500**,  taskRepeatCount = 20, concurrentSearchers = False
   
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       210.24 |    (9.7%) |                  537.65 |    (6.7%) |
   | HighTermMonthSort     |       116.02 |    (8.9%) |                  189.96 |   (13.5%) |
   | HighTermDayOfYearSort |        42.33 |    (7.6%) |                   67.93 |    (9.3%) |
   
   With increased `topN` the sort optimization has less speedups up to 2x, as it is expected as it will be possible to run it only after collecting `topN` docs.
   
   ---
   topN=10, taskRepeatCount = 20, **concurrentSearchers = True**
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       132.09 |   (14.3%) |                  287.93 |   (11.8%) |
   | HighTermMonthSort     |       211.01 |   (12.2%) |                  116.46 |    (7.1%) |
   | HighTermDayOfYearSort |        72.28 |    (6.1%) |                   68.21 |   (11.4%) |
   
   With the concurrent searchers the speedups are also smaller up to 2x. This is expected as now segments are spread between several TopFieldCollects/Comparators and they don't exchange bottom values.  As a follow-up on this PR, we can think how we can have a global bottom value similar how `MaxScoreAccumulator` is used to set up a global competitive min score. 
   
   ---
   with **indexSort='lastModNDV:long'**  topN=10, taskRepeatCount = 20, concurrentSearchers = False
   
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       314.78 |   (11.6%) |                  111.80 |   (13.3%) |
   | HighTermMonthSort     |       114.77 |   (13.1%) |                   78.22 |    (7.5%) |
   | HighTermDayOfYearSort |        46.82 |    (5.7%) |                   33.68 |    (6.1%) |

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394676728
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -223,44 +257,23 @@ public int score(LeafCollector collector, Bits acceptDocs, int min, int max) thr
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static int scoreRange(LeafCollector collector, DocIdSetIterator iterator, TwoPhaseIterator twoPhase,
         Bits acceptDocs, int currentDoc, int end) throws IOException {
-      if (twoPhase == null) {
-        while (currentDoc < end) {
-          if (acceptDocs == null || acceptDocs.get(currentDoc)) {
-            collector.collect(currentDoc);
-          }
-          currentDoc = iterator.nextDoc();
-        }
-        return currentDoc;
-      } else {
-        final DocIdSetIterator approximation = twoPhase.approximation();
-        while (currentDoc < end) {
-          if ((acceptDocs == null || acceptDocs.get(currentDoc)) && twoPhase.matches()) {
-            collector.collect(currentDoc);
-          }
-          currentDoc = approximation.nextDoc();
+      while (currentDoc < end) {
+        if ((acceptDocs == null || acceptDocs.get(currentDoc)) && (twoPhase == null || twoPhase.matches())) {
+          collector.collect(currentDoc);
         }
-        return currentDoc;
+        currentDoc = iterator.nextDoc();
       }
+      return currentDoc;
 
 Review comment:
   this change is not required ? I see hotspot in the javadoc comment above so  we shouldn't touch it if it's not  required ;).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r403471265
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
 ##########
 @@ -91,6 +91,7 @@
   private String field;
   private Type type;  // defaults to determining type dynamically
   boolean reverse = false;  // defaults to natural order
+  private boolean skipNonCompetitiveDocs = false; // if true, sortField will use a comparator that can skip non-competitive docs
 
 Review comment:
   I'd rather not have this on SortField for now. This is an old API that never required fields to be indexed. I'd rather have new SortField implementations for now, and later look at how we can enable this in SortField.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410690396
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
+
+    public FilteringFieldComparator(FieldComparator<T> in) {
+        this.in = in;
+    }
+
+    protected abstract void setCanUpdateIterator() throws IOException;
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return in.compare(slot1, slot2);
+    }
+
+    @Override
+    public T value(int slot) {
+        return in.value(slot);
+    }
+
+    @Override
+    public void setTopValue(T value) {
+        in.setTopValue(value);
+    }
+
+    @Override
+    public int compareValues(T first, T second) {
+        return in.compareValues(first, second);
+    }
+
+    /**
+     * Returns an iterator over competitive documents
+     */
+    public DocIdSetIterator competitiveIterator() {
+        if (iterator == null) return null;
+        return new DocIdSetIterator() {
+            private int doc;
+            @Override
+            public int nextDoc() throws IOException {
+                return doc = iterator.nextDoc();
+            }
+
+            @Override
+            public int docID() {
+                return doc;
+            }
+
+            @Override
+            public long cost() {
+                return iterator.cost();
+            }
+
+            @Override
+            public int advance(int target) throws IOException {
+                return doc = iterator.advance(target);
+            }
+        };
+    }
+
+    /**
+     * Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
+     * If for the given comparator the skip functionality is not implemented, return the comparator itself.
+     */
+    public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse) {
+        if (comparator instanceof FieldComparator.LongComparator){
+            return new FilteringFieldComparator.FilteringLongComparator((FieldComparator.LongComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.IntComparator){
+            return new FilteringFieldComparator.FilteringIntComparator((FieldComparator.IntComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.DoubleComparator){
+            return new FilteringFieldComparator.FilteringDoubleComparator((FieldComparator.DoubleComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.FloatComparator){
+            return new FilteringFieldComparator.FilteringFloatComparator((FieldComparator.FloatComparator) comparator, reverse);
+        }
+        return comparator;
+    }
+
+    /**
+     * A wrapper over {@code NumericComparator} that adds a functionality to filter non-competitive docs.
+     */
+    public static abstract class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> implements LeafFieldComparator {
+        private final boolean reverse;
+        private boolean hasTopValue = false;
+        private PointValues pointValues;
+        private final int bytesCount;
+        private final byte[] minValueAsBytes;
+        private final byte[] maxValueAsBytes;
+        private boolean minValueExist = false;
+        private boolean maxValueExist = false;
+        private int maxDoc;
+        private int maxDocVisited;
+        private int updateCounter = 0;
+        private final String field;
+        protected boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
+
+        public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, int bytesCount) {
+            super(in);
+            this.field = in.field;
+            this.bytesCount = bytesCount;
+            this.reverse = reverse;
+            minValueAsBytes = new byte[bytesCount];
+            maxValueAsBytes = new byte[bytesCount];
+            if (reverse) {
+                minValueExist = true;
+            } else {
+                maxValueExist = true;
+            }
+        }
+
+        @Override
+        public void setCanUpdateIterator() throws IOException {
+            this.canUpdateIterator = true;
+            // for the 1st time queue becomes full and hitsThreshold is reached
+            // we can start updating competitive iterator
+            updateCompetitiveIterator();
+        }
+
+        @Override
+        public void setTopValue(T value) {
+            hasTopValue = true;
+            if (reverse) {
+                maxValueExist = true;
+            } else {
+                minValueExist = true;
+            }
+            in.setTopValue(value);
+        }
+
+        @Override
+        public void setBottom(int slot) throws IOException {
+            ((NumericComparator) in).setBottom(slot);
+            updateCompetitiveIterator(); // update an iterator if we set a new bottom
+        }
+
+        @Override
+        public int compareBottom(int doc) throws IOException {
+            return ((NumericComparator) in).compareBottom(doc);
+        }
+
+        @Override
+        public int compareTop(int doc) throws IOException {
+            return ((NumericComparator) in).compareTop(doc);
+        }
+
+        @Override
+        public void copy(int slot, int doc) throws IOException {
+            ((NumericComparator) in).copy(slot, doc);
+        }
+
+        @Override
+        public void setScorer(Scorable scorer) throws IOException {}
+
+        @Override
+        public final LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+            ((NumericComparator) in).doSetNextReader(context);
+            pointValues = context.reader().getPointValues(field);
+            iterator = pointValues == null ? null : ((NumericComparator)in).currentReaderValues;
 
 Review comment:
   I don't think we should steal the iterator of the wrapped comparator, it's a hacky and I believe that it could cause bugs due to the iterator being advanced on doc IDs out-of-order since the same iterator has two consumers that don't know about each other.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408887800
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -24,37 +24,53 @@
   /**
    * Produced scorers will allow visiting all matches and get their score.
    */
-  COMPLETE {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  },
+  COMPLETE(true, true),
 
   /**
    * Produced scorers will allow visiting all matches but scores won't be
    * available.
    */
-  COMPLETE_NO_SCORES {
-    @Override
-    public boolean needsScores() {
-      return false;
-    }
-  },
+  COMPLETE_NO_SCORES(true, false),
 
   /**
    * Produced scorers will optionally allow skipping over non-competitive
    * hits using the {@link Scorer#setMinCompetitiveScore(float)} API.
    */
-  TOP_SCORES {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  };
+  TOP_SCORES(false, true),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS(false, false),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES(false, true);
 
 Review comment:
   But `TOP_SCORES` and `TOP_DOCS_WITH_SCORES` have identical `needsScores()` and `isExhaustive()` values, so I'm not sure why we need both?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r406054398
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -51,10 +59,51 @@ public boolean needsScores() {
     public boolean needsScores() {
       return true;
     }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
+  },
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS {
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
+  },
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES {
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+    @Override
+    public boolean isExhaustive() {
+      return false;
+    }
   };
 
   /**
    * Whether this {@link ScoreMode} needs to compute scores.
    */
   public abstract boolean needsScores();
+
+  /**
+   * Returns {@code true} if for this {@link ScoreMode} it is necessary to process all documents,
+   * or {@code false} if is enough to go through top documents only.
+   */
+  public abstract boolean isExhaustive();
 
 Review comment:
   Rather than defining these as abstract, maybe have two final fields set in a constructor?  Then the definitions become much less verbose: `TOP_DOCS_WITH_SCORES(true, false);`

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r409079589
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,16 @@
    */
   void collect(int doc) throws IOException;
 
+  /**
+   * Optionally creates a view of the scorerIterator where only competitive documents
+   * in the scorerIterator are kept and non-competitive are skipped.
+   *
+   * Collectors should delegate this method to their comparators if
+   * their comparators provide the skipping functionality over non-competitive docs.
+   * The default is to return the same iterator which is interpreted as the collector doesn't filter any documents.
+   */
+  default DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return scorerIterator;
+  }
 
 Review comment:
   Thanks, makes sense. Addressed in d7ef9b6

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408522075
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/ScoreMode.java
 ##########
 @@ -24,37 +24,53 @@
   /**
    * Produced scorers will allow visiting all matches and get their score.
    */
-  COMPLETE {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  },
+  COMPLETE(true, true),
 
   /**
    * Produced scorers will allow visiting all matches but scores won't be
    * available.
    */
-  COMPLETE_NO_SCORES {
-    @Override
-    public boolean needsScores() {
-      return false;
-    }
-  },
+  COMPLETE_NO_SCORES(true, false),
 
   /**
    * Produced scorers will optionally allow skipping over non-competitive
    * hits using the {@link Scorer#setMinCompetitiveScore(float)} API.
    */
-  TOP_SCORES {
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-  };
+  TOP_SCORES(false, true),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs
+   */
+  TOP_DOCS(false, false),
+
+  /**
+   * ScoreMode for top field collectors that can provide their own iterators,
+   * to optionally allow to skip for non-competitive docs.
+   * This mode is used when there is a secondary sort by _score.
+   */
+  TOP_DOCS_WITH_SCORES(false, true);
 
 Review comment:
   Yes, I think this distinction is necessary, if there is an additional sort field on `_score`.
   `Weight` and `Scorer` classes check collector's scoreMode for `needsScores()`, and will not calculate scores if `needsScores` is false.  I have added [an additional test](https://github.com/apache/lucene-solr/pull/1351/commits/c84fe5ed90b866dcbe41979dc6ba2ce2a874d59b#diff-cade050face1b04d4f21171fb26a319eR84)  to check that scores computed .

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
romseygeek commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r406053016
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.io.IOException;
+
+public class TestSortOptimization extends LuceneTestCase {
+
+  public void testLongSortOptimization() throws IOException {
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    final int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      final Document doc = new Document();
+      doc.add(new NumericDocValuesField("my_field", i));
+      doc.add(new LongPoint("my_field", i));
+      writer.addDocument(doc);
+    }
+    final IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    final SortField sortField = new SortField("my_field", SortField.Type.LONG);
+    sortField.allowFilterNonCompetitveDocs();
 
 Review comment:
   Can we avoid having to set this explicitly? The fact that we're setting a totalHitsThreshold on the collector should be enough to tell the comparator that we're enabling skipping.  Maybe have a `SkippableSortField` implementation that wraps an existing SortField and overrides `getComparator()`; then in `TopFieldCollector.create()` we check if totalHitsThreshold is set and if so rebuild the sort

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I think it is better to have a previous implementation with a dedicate `updateIterator` function called from `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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393624154
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
 
 Review comment:
   We should throttle the checks here (if the bottom value changes frequently). In the distance feature query we start throttling after 256 calls, we should replicate here ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410688437
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
+
+    public FilteringFieldComparator(FieldComparator<T> in) {
+        this.in = in;
+    }
+
+    protected abstract void setCanUpdateIterator() throws IOException;
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return in.compare(slot1, slot2);
+    }
+
+    @Override
+    public T value(int slot) {
+        return in.value(slot);
+    }
+
+    @Override
+    public void setTopValue(T value) {
+        in.setTopValue(value);
+    }
+
+    @Override
+    public int compareValues(T first, T second) {
+        return in.compareValues(first, second);
+    }
+
+    /**
+     * Returns an iterator over competitive documents
+     */
+    public DocIdSetIterator competitiveIterator() {
+        if (iterator == null) return null;
+        return new DocIdSetIterator() {
+            private int doc;
+            @Override
+            public int nextDoc() throws IOException {
+                return doc = iterator.nextDoc();
+            }
+
+            @Override
+            public int docID() {
+                return doc;
+            }
+
+            @Override
+            public long cost() {
+                return iterator.cost();
+            }
+
+            @Override
+            public int advance(int target) throws IOException {
+                return doc = iterator.advance(target);
+            }
+        };
+    }
+
+    /**
+     * Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
+     * If for the given comparator the skip functionality is not implemented, return the comparator itself.
+     */
+    public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse) {
+        if (comparator instanceof FieldComparator.LongComparator){
+            return new FilteringFieldComparator.FilteringLongComparator((FieldComparator.LongComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.IntComparator){
+            return new FilteringFieldComparator.FilteringIntComparator((FieldComparator.IntComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.DoubleComparator){
+            return new FilteringFieldComparator.FilteringDoubleComparator((FieldComparator.DoubleComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.FloatComparator){
+            return new FilteringFieldComparator.FilteringFloatComparator((FieldComparator.FloatComparator) comparator, reverse);
+        }
+        return comparator;
+    }
+
+    /**
+     * A wrapper over {@code NumericComparator} that adds a functionality to filter non-competitive docs.
+     */
+    public static abstract class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> implements LeafFieldComparator {
 
 Review comment:
   I know some existing comparators do this but it'd be cleaner to not implement FieldComparator and LeafFieldComparator in the same 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394656220
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
+        final byte[] maxValueAsBytes = new byte[Long.BYTES];
+        final byte[] minValueAsBytes = new byte[Long.BYTES];
+        if (reverse == false) {
+            LongPoint.encodeDimension(bottom, maxValueAsBytes, 0);
+        } else {
+            LongPoint.encodeDimension(bottom, minValueAsBytes, 0);
+        };
+
+        DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
+        PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
+            DocIdSetBuilder.BulkAdder adder;
+            @Override
+            public void grow(int count) {
+                adder = result.grow(count);
+            }
+
+            @Override
+            public void visit(int docID) {
+                if (docID <= maxDocVisited) {
+                    return; // Already visited or skipped
+                }
+                adder.add(docID);
+            }
+
+            @Override
+            public void visit(int docID, byte[] packedValue) {
+                if (docID <= maxDocVisited) {
+                    return;  // Already visited or skipped
+                }
+                if ((reverse == false) && (Arrays.compareUnsigned(packedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0)) {
+                    return; // Doc's value is too high
+                }
+                if ((reverse == true) && (Arrays.compareUnsigned(packedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0)) {
+                    return;  // Doc's value is too low,
+                }
+                adder.add(docID); // doc is competitive
+            }
+
+            @Override
+            public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                if ((reverse == false) && (Arrays.compareUnsigned(minPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0)) {
+                   return PointValues.Relation.CELL_OUTSIDE_QUERY;
+                }
+                if ((reverse == true) && (Arrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0)) {
+                    return PointValues.Relation.CELL_OUTSIDE_QUERY;
+                }
+                return PointValues.Relation.CELL_CROSSES_QUERY;
+            }
+        };
+        pointValues.intersect(visitor);
 
 Review comment:
   Addressed in 6384b15

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393621549
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   Why do we need this ? We could update the iterator every time a bottom value is set ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r399829998
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IterableComparator;
+
+/**
+ * Expert: a FieldComparator class for long types corresponding to
+ * {@link LongDocValuesPointSortField}.
+ * This comparator provides {@code iterator} over competitive documents,
+ * that are stronger than the current {@code bottom} value.
+ */
+public class LongDocValuesPointComparator extends IterableComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+    private byte[] cmaxValueAsBytes = null;
 
 Review comment:
   Can these be final, and allocated only in the constructor? I think it might be clearer to add a boolean "hasTopValues" and set that in setTopValue, rather than use the existence of these byte[]? Then you could make these final and eliminate the local variables where they get copied below 

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   Do you know the reason why `hitsThresholdChecker.isThresholdReached`  checks for greater than numHits and not greater or equal numHits?
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I think it is better to have a previous implementation with a dedicated `updateIterator` function called from `TopFieldCollector`. WDYT?
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I will keep thinking how to organize it better
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410690540
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
+
+    public FilteringFieldComparator(FieldComparator<T> in) {
+        this.in = in;
+    }
+
+    protected abstract void setCanUpdateIterator() throws IOException;
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return in.compare(slot1, slot2);
+    }
+
+    @Override
+    public T value(int slot) {
+        return in.value(slot);
+    }
+
+    @Override
+    public void setTopValue(T value) {
+        in.setTopValue(value);
+    }
+
+    @Override
+    public int compareValues(T first, T second) {
+        return in.compareValues(first, second);
+    }
+
+    /**
+     * Returns an iterator over competitive documents
+     */
+    public DocIdSetIterator competitiveIterator() {
+        if (iterator == null) return null;
+        return new DocIdSetIterator() {
+            private int doc;
+            @Override
+            public int nextDoc() throws IOException {
+                return doc = iterator.nextDoc();
+            }
+
+            @Override
+            public int docID() {
+                return doc;
+            }
+
+            @Override
+            public long cost() {
+                return iterator.cost();
+            }
+
+            @Override
+            public int advance(int target) throws IOException {
+                return doc = iterator.advance(target);
+            }
+        };
+    }
+
+    /**
+     * Try to wrap a given field comparator to add to it a functionality to skip over non-competitive docs.
+     * If for the given comparator the skip functionality is not implemented, return the comparator itself.
+     */
+    public static FieldComparator<?> wrapToFilteringComparator(FieldComparator<?> comparator, boolean reverse) {
+        if (comparator instanceof FieldComparator.LongComparator){
+            return new FilteringFieldComparator.FilteringLongComparator((FieldComparator.LongComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.IntComparator){
+            return new FilteringFieldComparator.FilteringIntComparator((FieldComparator.IntComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.DoubleComparator){
+            return new FilteringFieldComparator.FilteringDoubleComparator((FieldComparator.DoubleComparator) comparator, reverse);
+        }
+        if (comparator instanceof FieldComparator.FloatComparator){
+            return new FilteringFieldComparator.FilteringFloatComparator((FieldComparator.FloatComparator) comparator, reverse);
+        }
+        return comparator;
+    }
+
+    /**
+     * A wrapper over {@code NumericComparator} that adds a functionality to filter non-competitive docs.
+     */
+    public static abstract class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> implements LeafFieldComparator {
+        private final boolean reverse;
+        private boolean hasTopValue = false;
+        private PointValues pointValues;
+        private final int bytesCount;
+        private final byte[] minValueAsBytes;
+        private final byte[] maxValueAsBytes;
+        private boolean minValueExist = false;
+        private boolean maxValueExist = false;
+        private int maxDoc;
+        private int maxDocVisited;
+        private int updateCounter = 0;
+        private final String field;
+        protected boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
+
+        public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, int bytesCount) {
+            super(in);
+            this.field = in.field;
+            this.bytesCount = bytesCount;
+            this.reverse = reverse;
+            minValueAsBytes = new byte[bytesCount];
+            maxValueAsBytes = new byte[bytesCount];
+            if (reverse) {
+                minValueExist = true;
+            } else {
+                maxValueExist = true;
+            }
+        }
+
+        @Override
+        public void setCanUpdateIterator() throws IOException {
+            this.canUpdateIterator = true;
+            // for the 1st time queue becomes full and hitsThreshold is reached
+            // we can start updating competitive iterator
+            updateCompetitiveIterator();
+        }
+
+        @Override
+        public void setTopValue(T value) {
+            hasTopValue = true;
+            if (reverse) {
+                maxValueExist = true;
+            } else {
+                minValueExist = true;
+            }
+            in.setTopValue(value);
+        }
+
+        @Override
+        public void setBottom(int slot) throws IOException {
+            ((NumericComparator) in).setBottom(slot);
+            updateCompetitiveIterator(); // update an iterator if we set a new bottom
+        }
+
+        @Override
+        public int compareBottom(int doc) throws IOException {
+            return ((NumericComparator) in).compareBottom(doc);
+        }
+
+        @Override
+        public int compareTop(int doc) throws IOException {
+            return ((NumericComparator) in).compareTop(doc);
+        }
+
+        @Override
+        public void copy(int slot, int doc) throws IOException {
+            ((NumericComparator) in).copy(slot, doc);
+        }
+
+        @Override
+        public void setScorer(Scorable scorer) throws IOException {}
+
+        @Override
+        public final LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+            ((NumericComparator) in).doSetNextReader(context);
+            pointValues = context.reader().getPointValues(field);
+            iterator = pointValues == null ? null : ((NumericComparator)in).currentReaderValues;
 
 Review comment:
   Furthermore we use `iterator.cost() >>> 3` below as a condition. I think that the first time we should instead use the cost of the scorer otherwise this approach is going to slow down selective queries.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394659789
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
 ##########
 @@ -928,4 +928,9 @@ public int compareTop(int doc) throws IOException {
     @Override
     public void setScorer(Scorable scorer) {}
   }
+
+  public static abstract class IteratorSupplierComparator<T> extends FieldComparator<T> implements LeafFieldComparator {
+    abstract DocIdSetIterator iterator();
+    abstract void updateIterator() throws IOException;
 
 Review comment:
   Indeed it is more straightforward to just update an iterator in `setBottom` function of a comparator.
   
   But I was thinking it is better to have a special function for two reasons:
   1)  After updating an iterator, in `TopFieldCollector` we need to change 
   `totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;`
   
   2)  we also need to check `hitsThresholdChecker.isThresholdReached()`, and passing not strictly related object `hitsThresholdChecker` to a comparator's constructor doesn't look nice to me. 
   Please let me know if you think otherwise

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395284149
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -201,20 +201,54 @@ public long cost() {
     @Override
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
+      DocIdSetIterator scorerIterator = twoPhase == null? iterator: twoPhase.approximation();
+      DocIdSetIterator combinedIterator = collector.iterator() == null ? scorerIterator: combineScorerAndCollectorIterators(scorerIterator, collector);
       if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
-        scoreAll(collector, iterator, twoPhase, acceptDocs);
+        scoreAll(collector, combinedIterator, twoPhase, acceptDocs);
         return DocIdSetIterator.NO_MORE_DOCS;
       } else {
         int doc = scorer.docID();
-        if (doc < min) {
-          if (twoPhase == null) {
-            doc = iterator.advance(min);
-          } else {
-            doc = twoPhase.approximation().advance(min);
+        if (doc < min) scorerIterator.advance(min);
+        return scoreRange(collector, combinedIterator, twoPhase, acceptDocs, doc, max);
+      }
+    }
+
+    // conjunction iterator between scorer's iterator and collector's iterator
 
 Review comment:
   https://github.com/apache/lucene-solr/pull/1351/commits/d732d7eb9de67a597f67e91c9774104aa055e293

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova edited a comment on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-604173071
 
 
   I have run some benchmarking using `luceneutil`.
   As the new sort optimization uses a new `LongDocValuesPointSortField` that is not present in `luceneutil`, I had to hack `luceneutil` as follows:
   
   1. I added a  sort task on a long field `TermDateTimeSort`  to `wikimedium.1M.nostopwords.tasks` . This task was present in `wikinightly.tasks` , but was not able for wikimedium 1M and 10M tasks
   2. I indexed the corresponding field `lastModNDV` as `LongPoint` as well. It was only indexed as `NumericDocValuesField` before, but for the sort optimization we need long values to be indexed both as docValues and as points.
   3. I modified `SearchTask.java` to have `TopFieldCollector` with `totalHitsThreshold` set to `topK`: `final TopFieldCollector c = TopFieldCollector.create(s, topN, null, topN);`   Sort optimization only works when we set total hits threshold.
   4. For the patch version , I modified sort in `TaskParser.java`. Instead of `lastModNDVSort = new Sort(new SortField("lastModNDV", SortField.Type.LONG));`  I useed the optimized sort: `lastModNDVSort = new Sort(new LongDocValuesPointSortField("lastModNDV"));`
   
   Here the main point of comparison is `TermDTSort` as it is the only sort on long field. Other sorts are presented to demonstrate a possible regression or absence on them.
   
   ---
   wikimedium1m
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       507.20 |   (11.2%) |                  550.02 |   (16.1%) |
   | HighTermMonthSort     |       550.06 |   (10.4%) |                  443.69 |   (16.1%) |
   | HighTermDayOfYearSort |       105.62 |   (24.9%) |                   91.93 |   (22.1%) |
   ---
   wikimedium10m
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   For wikimedium1m  TermDTSort using `LongDocValuesPointSortField` doesn't seem to have much effect. As probably in this index segments are smaller, and probably optimization was completely skipped on those segments.
   For wikimedium10m TermDTSort using `LongDocValuesPointSortField`  instead of usual `SortField.Type.LONG` **brings about 3x speedups**.
   There is some regression/speedups for the sort tasks of HighTermMonthSort and HighTermDayOfYearSort, which I don't know the reason why, as they should not be effected. 
   
   
   
   
   
   
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r400739533
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
+   * optionally returns an iterator over competitive documents
+   */
+  default DocIdSetIterator iterator() {
 
 Review comment:
   maybe give it a more descriptive name, e.g. `competitiveFilter`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r410688181
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/FilteringFieldComparator.java
 ##########
 @@ -0,0 +1,350 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Decorates a wrapped FieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringFieldComparator provides two additional functions for a FieldComparator:
+ * 1) {@code competitiveIterator()} that returns an iterator over
+ *      competitive docs that are stronger than already collected docs.
+ * 2) {@code setCanUpdateIterator()} that notifies the comparator when it is ok to start updating its internal iterator.
+ *  This method is called from a collector to inform the comparator to start updating its iterator.
+ */
+public abstract class FilteringFieldComparator<T> extends FieldComparator<T> {
+    final FieldComparator<T> in;
+    protected DocIdSetIterator iterator = null;
 
 Review comment:
   It feels wrong to have an iterator - which is a per-segment object - on a FieldComparator - which is a top-level object. Can we only have it on the LeafComparator?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393625750
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    public void updateIterator() throws IOException {
+        final byte[] maxValueAsBytes = new byte[Long.BYTES];
+        final byte[] minValueAsBytes = new byte[Long.BYTES];
+        if (reverse == false) {
+            LongPoint.encodeDimension(bottom, maxValueAsBytes, 0);
+        } else {
+            LongPoint.encodeDimension(bottom, minValueAsBytes, 0);
+        };
+
 
 Review comment:
   you should also take the topValue into  account here (searchAfter) ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r396690116
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,195 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    // update its iterator to include possibly only docs that are "stronger" than the current bottom entry
+    public void updateIterator() throws IOException {
+        updateCounter++;
+        if (updateCounter > 256 && (updateCounter & 0x1f) != 0x1f) { // Start sampling if we get called too much
+            return;
+        }
+
+        final byte[] maxValueAsBytes = reverse == false ? new byte[Long.BYTES] : hasTopValue ? new byte[Long.BYTES]: null;
 
 Review comment:
   @msokolov  Thanks for the suggestion, indeed these values can be initialized in the comparator's constructor. As each topfieldcollector has its own comparator and processes segments sequentially, these values should be shareable.  Addressed in 95e1bc1

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-615280419
 
 
   @romseygeek  Are you suggesting to do 
   ```java
   if (updateCounter > 1024 && (updateCounter & 0x1f) != 0x1f) {
   ```
   but this will run optimization even more often which we want to avoid, no?
   
   It the wikimedium1m TermDTSort  case,  `updateCounter` doesn't even reach 20 (so the optimization doesn't called that many times), but enough to make it slower than the traditional sort.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r394677039
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/Weight.java
 ##########
 @@ -201,20 +201,54 @@ public long cost() {
     @Override
     public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
       collector.setScorer(scorer);
+      DocIdSetIterator scorerIterator = twoPhase == null? iterator: twoPhase.approximation();
+      DocIdSetIterator combinedIterator = collector.iterator() == null ? scorerIterator: combineScorerAndCollectorIterators(scorerIterator, collector);
       if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
-        scoreAll(collector, iterator, twoPhase, acceptDocs);
+        scoreAll(collector, combinedIterator, twoPhase, acceptDocs);
         return DocIdSetIterator.NO_MORE_DOCS;
       } else {
         int doc = scorer.docID();
-        if (doc < min) {
-          if (twoPhase == null) {
-            doc = iterator.advance(min);
-          } else {
-            doc = twoPhase.approximation().advance(min);
+        if (doc < min) scorerIterator.advance(min);
 
 Review comment:
   ```suggestion
           if (doc < min) {
             doc = combinedIterator.advance(min);
           }
   ```
   ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on issue #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#issuecomment-604173071
 
 
   I have run some benchmarking using `luceneutil`.
   As the new sort optimization uses a new `LongDocValuesPointSortField` that is not present in `luceneutil`, I had to hack `luceneutil` as follows:
   
   1. I added a  sort task on a long field `TermDateTimeSort`  to `wikimedium.1M.nostopwords.tasks` . This task was present in `wikinightly.tasks` , but was not able for wikimedium 1M and 10M tasks
   2. I indexed the corresponding field `lastModNDV` as `LongPoint` as well. It was only indexed as `NumericDocValuesField` before, but for the sort optimization we need long values to be indexed both as docValues and as points.
   3. I modified `SearchTask.java` to have `TopFieldCollector` with `totalHitsThreshold` set to `topK`: `final TopFieldCollector c = TopFieldCollector.create(s, topN, null, topN);`   Sort optimization only works when we set total hits threshold.
   4. For the patch version , I modified sort in `TaskParser.java`. Instead of `lastModNDVSort = new Sort(new SortField("lastModNDV", SortField.Type.LONG));`  I useed the optimized sort: `lastModNDVSort = new Sort(new LongDocValuesPointSortField("lastModNDV"));`
   
   Here the main point of comparison is `TermDTSort` as it is the only sort on long field. Other sorts are presented to demonstrate a possible regression or absence on them.
   
   ---
   wikimedium1m
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       507.20 |   (11.2%) |                  550.02 |   (16.1%) |
   | HighTermMonthSort     |       550.06 |   (10.4%) |                  443.69 |   (16.1%) |
   | HighTermDayOfYearSort |       105.62 |   (24.9%) |                   91.93 |   (22.1%) |
   ---
   wikimedium10m
   | TaskQPS               | baseline QPS | StdDevQPS | my_modified_version QPS | StdDevQPS |
   | --------------------- | -----------: | --------: | ----------------------: | --------: |
   | **TermDTSort**        |       147.64 |   (11.5%) |                  547.80 |    (6.6%) |
   | HighTermMonthSort     |       147.85 |   (12.2%) |                  239.28 |    (7.3%) |
   | HighTermDayOfYearSort |        74.44 |    (7.7%) |                   42.56 |   (12.1%) |
   
   For wikimedium1m using `LongDocValuesPointSortField` doesn't seem to have much effect. As probably in this index segments are smaller, and probably optimization was completely skipped on those segments.
   For wikimedium10m using `LongDocValuesPointSortField`  instead of usual `SortField.Type.LONG` **brings about 3x speedups**.
   There is so regression/speedups for the sort tasks of HighTermMonthSort and HighTermDayOfYearSort, which I don't know the reason why, as they should not be effected. 
   
   
   
   
   
   
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r400739959
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
 ##########
 @@ -93,4 +93,11 @@
    */
   void collect(int doc) throws IOException;
 
+  /*
 
 Review comment:
   Let's make it proper javadocs, not comments?
   ```suggestion
     /**
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r407107952
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -302,20 +312,27 @@ private TopFieldCollector(FieldValueHitQueue<Entry> pq, int numHits,
     this.numHits = numHits;
     this.hitsThresholdChecker = hitsThresholdChecker;
     this.numComparators = pq.getComparators().length;
-    FieldComparator<?> fieldComparator = pq.getComparators()[0];
+    FieldComparator<?> firstComparator = pq.getComparators()[0];
     int reverseMul = pq.reverseMul[0];
-    if (fieldComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
+    if (firstComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
           && reverseMul == 1 // if the natural sort is preserved (sort by descending relevance)
           && hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
-      firstComparator = (FieldComparator.RelevanceComparator) fieldComparator;
+      relevanceComparator = (FieldComparator.RelevanceComparator) firstComparator;
       scoreMode = ScoreMode.TOP_SCORES;
       canSetMinScore = true;
     } else {
-      firstComparator = null;
+      relevanceComparator = null;
       scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
       canSetMinScore = false;
     }
     this.minScoreAcc = minScoreAcc;
+
+    if ((firstComparator instanceof FilteringFieldComparator) && (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE)) {
 
 Review comment:
   @romseygeek  Addressed in 719882e

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r396128019
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,195 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+    }
+
+    @Override
+    public int compareBottom(int doc) throws IOException {
+        return Long.compare(bottom, getValueForDoc(doc));
+    }
+
+    @Override
+    public int compareTop(int doc) throws IOException {
+        return Long.compare(topValue, getValueForDoc(doc));
+    }
+
+    @Override
+    public void copy(int slot, int doc) throws IOException {
+        maxDocVisited = doc;
+        values[slot] = getValueForDoc(doc);
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {}
+
+    public DocIdSetIterator iterator() {
+        return iterator;
+    }
+
+    // update its iterator to include possibly only docs that are "stronger" than the current bottom entry
+    public void updateIterator() throws IOException {
+        updateCounter++;
+        if (updateCounter > 256 && (updateCounter & 0x1f) != 0x1f) { // Start sampling if we get called too much
+            return;
+        }
+
+        final byte[] maxValueAsBytes = reverse == false ? new byte[Long.BYTES] : hasTopValue ? new byte[Long.BYTES]: null;
 
 Review comment:
   Can we move this initialization into the constructor, or is this not shareable and must be local storage? I think we call updateIterator in collect() right? If we can avoid object creation in an inner loop, that would be good. We could create both arrays unconditionally I think and set a boolean here to be used below?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
jimczi commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r393625098
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final boolean reverse;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private int maxDoc;
+    private int maxDocVisited;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse) {
+        this.field = field;
+        this.reverse = reverse;
+        this.values = new long[numHits];
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return 0L; // TODO: missing value
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) {
+        this.bottom = values[slot];
 
 Review comment:
   Can you update the iterator here ? We would need to check the total hits threshold so maybe pass the `HitsThresholdChecker` in the ctr somehow ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r395291803
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/LongDocValuesPointComparator.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.apache.lucene.search.FieldComparator.IteratorSupplierComparator;
+
+public class LongDocValuesPointComparator extends IteratorSupplierComparator<Long> {
+    private final String field;
+    private final int numHits;
+    private final boolean reverse;
+    private final long missingValue;
+    private final long[] values;
+    private long bottom;
+    private long topValue;
+    boolean hasTopValue = false; // indicates that topValue for searchAfter is set
+    protected NumericDocValues docValues;
+    private DocIdSetIterator iterator;
+    private PointValues pointValues;
+    private HitsThresholdChecker hitsThresholdChecker = null;
+    private int maxDoc;
+    private int maxDocVisited;
+    private int updateCounter = 0;
+
+    public LongDocValuesPointComparator(String field, int numHits, boolean reverse, Long missingValue) {
+        this.field = field;
+        this.numHits = numHits;
+        this.reverse = reverse;
+        this.missingValue = missingValue != null ? missingValue : 0L;
+        this.values = new long[numHits];
+    }
+
+    @Override
+    void setHitsThresholdChecker(HitsThresholdChecker hitsThresholdChecker) {
+        this.hitsThresholdChecker = hitsThresholdChecker;
+    }
+
+    private long getValueForDoc(int doc) throws IOException {
+        if (docValues.advanceExact(doc)) {
+            return docValues.longValue();
+        } else {
+            return missingValue;
+        }
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+        return Long.compare(values[slot1], values[slot2]);
+    }
+
+    @Override
+    public void setTopValue(Long value) {
+        topValue = value;
+        hasTopValue = true;
+    }
+
+    @Override
+    public Long value(int slot) {
+        return Long.valueOf(values[slot]);
+    }
+
+    @Override
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+        docValues = DocValues.getNumeric(context.reader(), field);
+        iterator = docValues;
+        pointValues = context.reader().getPointValues(field);
+        maxDoc = context.reader().maxDoc();
+        maxDocVisited = 0;
+        return this;
+    }
+
+    @Override
+    public void setBottom(int slot) throws IOException {
+        this.bottom = values[slot];
+        // can't use hitsThresholdChecker.isThresholdReached() as it uses > numHits,
+        // while we want to update iterator as soon as threshold reaches numHits
+        if (hitsThresholdChecker != null && (hitsThresholdChecker.getHitsThreshold() >= numHits)) {
 
 Review comment:
   @jimczi  I am not very happy about this change because of 2 reasons: 
   1) We  can't use `hitsThresholdChecker.isThresholdReached` as it checks for greater than numHits, but we need to check starting with equal, as if there are no competitive docs later `setBottom` will not be called.
   Do you know the reason why `hitsThresholdChecker.isThresholdReached`  checks for greater than numHits and not greater or equal numHits?
   2) totalHitsRelation may not end up to be set to TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, as we set it only when we have  later competitive hits. 
   
   I think it is better to have a previous implementation with a dedicated `updateIterator` function called from `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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408521182
 
 

 ##########
 File path: lucene/test-framework/src/java/org/apache/lucene/search/AssertingLeafCollector.java
 ##########
 @@ -50,5 +50,9 @@ public void collect(int doc) throws IOException {
     lastCollected = doc;
   }
 
+  @Override
+  public DocIdSetIterator filterIterator(DocIdSetIterator scorerIterator) {
+    return super.filterIterator(scorerIterator);
 
 Review comment:
   @romseygeek Good notice, not it is not necessary. Removed in c84fe5e

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #1351: LUCENE-9280: Collectors to skip noncompetitive documents

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r408522716
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -432,6 +462,12 @@ static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
       throw new IllegalArgumentException("hitsThresholdChecker should not be null");
     }
 
+    // here we assume that if hitsThreshold was set, we let the corresponding comparator to skip non-competitive docs
+    // It is beneficial for the 1st field only to skip non-competitive docs
+    if (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
+      sort.fields[0].allowFilterNonCompetitveDocs();
+    }
+
 
 Review comment:
   @romseygeek  Great suggestion to move wrapping into `FieldValueHitQueue`. I like the APIs more and more now.  Addressed in c84fe5e.
   And also looks like we don't even need any changes to `SortField`, we can just wrap comparators.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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