You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2020/12/04 13:31:29 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9599 Disable sort optim on index sort (#2117)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 7b3dbfe  LUCENE-9599 Disable sort optim on index sort (#2117)
7b3dbfe is described below

commit 7b3dbfedd95135aecaa4dca053cc63aabb422f24
Author: Mayya Sharipova <ma...@elastic.co>
AuthorDate: Fri Dec 4 08:31:10 2020 -0500

    LUCENE-9599 Disable sort optim on index sort (#2117)
    
    Disable sort optimization in comparators on index sort.
    
    Currently, if search sort is equal or a part of the index sort, we have
    an early termination in TopFieldCollector.
    But comparators are not aware of the index sort, and may run
    sort optimization even if the search sort is congruent with
    the index sort.
    
    This patch:
    - adds `disableSkipping` method to `FieldComparator`,
    This method is called by `TopFieldCollector`, when the search sort
    is congruent with the index sort.
    It is also called when we can't use points for sort optimization.
    - disables sort optimization in comparators in these cases.
    
    Relates to #1351
    Backport for #2075
---
 .../org/apache/lucene/search/FieldComparator.java  | 19 +++++++----
 .../apache/lucene/search/FieldValueHitQueue.java   |  7 ++--
 .../apache/lucene/search/TopFieldCollector.java    | 38 +++++++++++++++-------
 .../search/comparators/NumericComparator.java      | 15 ++++-----
 4 files changed, 49 insertions(+), 30 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
index 09cec43..be9b11f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
@@ -143,13 +143,20 @@ public abstract class FieldComparator<T> {
   }
 
   /**
-   * For numeric comparators, setting this value, indicates that
-   * the same numeric data has been indexed with two fields: doc values and points and
-   * that these fields have the same name.
-   * This allows to use sort optimization and skip non-competitive documents.
-   * Not applicable for other comparators.
+   * Informs the comparator that the skipping of documents should be disabled.
+   * This function is called in cases when the skipping functionality
+   * should not be applied or not necessary.
+   *
+   * One example for numeric comparators is when we don't know if the same numeric data has
+   * been indexed with docValues and points if these two fields have the same name.
+   * As the skipping functionality relies on these fields to have the same data
+   * and as we don't know if it is true, we have to disable it.
+   *
+   * Another example could be when search sort is a part of the index sort,
+   * and can be already efficiently handled by TopFieldCollector,
+   * and doing extra work for skipping in the comparator is redundant.
    */
-  public void setCanUsePoints() {
+  public void disableSkipping() {
   }
 
   /** Sorts by descending relevance.  NOTE: if you are
diff --git a/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java b/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
index f00c727..d6d3b60 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
@@ -136,12 +136,11 @@ public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> ext
     for (int i = 0; i < numComparators; ++i) {
       SortField field = fields[i];
       reverseMul[i] = field.reverse ? -1 : 1;
-      //TODO: field.getCanUsePoints()
       comparators[i] = field.getComparator(size, i);
     }
-    if (numComparators > 0 && fields[0].getCanUsePoints()) {
-      // inform a numeric comparator that it can use points for sort optimization
-      comparators[0].setCanUsePoints();
+    if (numComparators > 0 && fields[0].getCanUsePoints() == false) {
+      // disable skipping functionality of a numeric comparator if we can't use points
+      comparators[0].disableSkipping();
     }
     if (numComparators == 1) {
       // inform a comparator that sort is based on this single field
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 3d5e4ba..219f435 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -113,10 +113,17 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       docBase = context.docBase;
 
+      // as all segments are sorted in the same way, enough to check only the 1st segment for indexSort
+      if (searchSortPartOfIndexSort == null) {
+        final Sort indexSort = context.reader().getMetaData().getSort();
+        searchSortPartOfIndexSort = canEarlyTerminate(sort, indexSort);
+        if (searchSortPartOfIndexSort) {
+          firstComparator.disableSkipping();
+        }
+      }
+
       final LeafFieldComparator[] comparators = queue.getComparators(context);
       final int[] reverseMul = queue.getReverseMul();
-      final Sort indexSort = context.reader().getMetaData().getSort();
-      final boolean canEarlyTerminate = canEarlyTerminate(sort, indexSort);
 
       return new MultiComparatorLeafCollector(comparators, reverseMul) {
 
@@ -152,7 +159,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
               // since docs are visited in doc Id order, if compare is 0, it means
               // this document is largest than anything else in the queue, and
               // therefore not competitive.
-              if (canEarlyTerminate) {
+              if (searchSortPartOfIndexSort) {
                 if (hitsThresholdChecker.isThresholdReached()) {
                   totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
                   throw new CollectionTerminatedException();
@@ -226,8 +233,15 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       docBase = context.docBase;
       final int afterDoc = after.doc - docBase;
-      final Sort indexSort = context.reader().getMetaData().getSort();
-      final boolean canEarlyTerminate = canEarlyTerminate(sort, indexSort);
+      // as all segments are sorted in the same way, enough to check only the 1st segment for indexSort
+      if (searchSortPartOfIndexSort == null) {
+        final Sort indexSort = context.reader().getMetaData().getSort();
+        searchSortPartOfIndexSort = canEarlyTerminate(sort, indexSort);
+        if (searchSortPartOfIndexSort) {
+          firstComparator.disableSkipping();
+        }
+      }
+
       return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul()) {
 
         boolean collectedAllCompetitiveHits = false;
@@ -266,7 +280,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
               // since docs are visited in doc Id order, if compare is 0, it means
               // this document is largest than anything else in the queue, and
               // therefore not competitive.
-              if (canEarlyTerminate) {
+              if (searchSortPartOfIndexSort) {
                 if (hitsThresholdChecker.isThresholdReached()) {
                   totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
                   throw new CollectionTerminatedException();
@@ -330,9 +344,11 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
 
   final int numHits;
   final HitsThresholdChecker hitsThresholdChecker;
-  final FieldComparator.RelevanceComparator relevanceComparator;
+  final FieldComparator<?> firstComparator;
   final boolean canSetMinScore;
 
+  // shows if Search Sort if a part of the Index Sort
+  Boolean searchSortPartOfIndexSort = null;
   // an accumulator that maintains the maximum of the segment's minimum competitive scores
   final MaxScoreAccumulator minScoreAcc;
   // the current local minimum competitive score already propagated to the underlying scorer
@@ -358,17 +374,15 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
     this.numHits = numHits;
     this.hitsThresholdChecker = hitsThresholdChecker;
     this.numComparators = pq.getComparators().length;
-    FieldComparator<?> firstComparator = pq.getComparators()[0];
+    this.firstComparator = pq.getComparators()[0];
     int reverseMul = pq.reverseMul[0];
 
     if (firstComparator.getClass().equals(FieldComparator.RelevanceComparator.class)
             && reverseMul == 1 // if the natural sort is preserved (sort by descending relevance)
             && hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
-      relevanceComparator = (FieldComparator.RelevanceComparator) firstComparator;
       scoreMode = ScoreMode.TOP_SCORES;
       canSetMinScore = true;
     } else {
-      relevanceComparator = null;
       canSetMinScore = false;
       if (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) {
         scoreMode = needsScores ? ScoreMode.TOP_DOCS_WITH_SCORES : ScoreMode.TOP_DOCS;
@@ -404,8 +418,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
     if (canSetMinScore
           && queueFull
           && hitsThresholdChecker.isThresholdReached()) {
-      assert bottom != null && relevanceComparator != null;
-      float minScore = relevanceComparator.value(bottom.slot);
+      assert bottom != null;
+      float minScore = (float) firstComparator.value(bottom.slot);
       if (minScore > minCompetitiveScore) {
         scorer.setMinCompetitiveScore(minScore);
         minCompetitiveScore = minScore;
diff --git a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
index 7ed7511..73da1fb 100644
--- a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java
@@ -39,26 +39,25 @@ public abstract class NumericComparator<T extends Number> extends FieldComparato
     protected final T missingValue;
     protected final String field;
     protected final boolean reverse;
-    protected final boolean primarySort;
     private final int bytesCount; // how many bytes are used to encode this number
 
-    private boolean canUsePoints;
     protected boolean topValueSet;
     protected boolean singleSort; // singleSort is true, if sort is based on a single sort field.
     protected boolean hitsThresholdReached;
     protected boolean queueFull;
+    private boolean canSkipDocuments;
 
     protected NumericComparator(String field, T missingValue, boolean reverse, int sortPos, int bytesCount) {
         this.field = field;
         this.missingValue = missingValue;
         this.reverse = reverse;
-        this.primarySort = (sortPos == 0);
+        this.canSkipDocuments = (sortPos == 0); // skipping functionality is only relevant for primary sort
         this.bytesCount = bytesCount;
     }
 
     @Override
-    public void setCanUsePoints() {
-        canUsePoints = true;
+    public void disableSkipping() {
+        canSkipDocuments = false;
     }
 
     @Override
@@ -77,7 +76,7 @@ public abstract class NumericComparator<T extends Number> extends FieldComparato
     public abstract class NumericLeafComparator implements LeafFieldComparator {
         protected final NumericDocValues docValues;
         private final PointValues pointValues;
-        private final boolean enableSkipping; // if skipping functionality should be enabled
+        private final boolean enableSkipping; // if skipping functionality should be enabled on this segment
         private final int maxDoc;
         private final byte[] minValueAsBytes;
         private final byte[] maxValueAsBytes;
@@ -89,9 +88,9 @@ public abstract class NumericComparator<T extends Number> extends FieldComparato
 
         public NumericLeafComparator(LeafReaderContext context) throws IOException {
             this.docValues = getNumericDocValues(context, field);
-            this.pointValues = (primarySort && canUsePoints) ? context.reader().getPointValues(field) : null;
+            this.pointValues = canSkipDocuments ? context.reader().getPointValues(field) : null;
             if (pointValues != null) {
-                this.enableSkipping = true; // skipping is enabled on primarySort and when points are available
+                this.enableSkipping = true; // skipping is enabled when points are available
                 this.maxDoc = context.reader().maxDoc();
                 this.maxValueAsBytes = reverse == false ? new byte[bytesCount] : topValueSet ? new byte[bytesCount] : null;
                 this.minValueAsBytes = reverse ? new byte[bytesCount] : topValueSet ? new byte[bytesCount] : null;