You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ji...@apache.org on 2021/11/09 10:06:42 UTC

[lucene-solr] branch branch_8_11 updated: LUCENE-10208: Ensure that the minimum competitive score does not decrease in concurrent search

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

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


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new f30bfb6  LUCENE-10208: Ensure that the minimum competitive score does not decrease in concurrent search
f30bfb6 is described below

commit f30bfb65ad132582148dfba07093ab1208485f27
Author: jimczi <ji...@apache.org>
AuthorDate: Tue Nov 9 11:05:11 2021 +0100

    LUCENE-10208: Ensure that the minimum competitive score does not decrease in concurrent search
---
 lucene/CHANGES.txt                                 |  2 +
 .../apache/lucene/search/MaxScoreAccumulator.java  | 48 ++++++++++++++++------
 .../apache/lucene/search/TopFieldCollector.java    | 18 ++++----
 .../apache/lucene/search/TopScoreDocCollector.java | 25 ++++++++---
 .../lucene/search/TestMaxScoreAccumulator.java     | 22 ++++++----
 .../apache/lucene/search/TestTopDocsCollector.java | 37 +++++++----------
 .../lucene/search/TestTopFieldCollector.java       | 39 +++++-------------
 7 files changed, 108 insertions(+), 83 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 92d1200..8dd90ae 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -48,6 +48,8 @@ Bug Fixes
 
 * LUCENE-10154: NumericLeafComparator to define getPointValues. (Mayya Sharipova, Adrien Grand)
 
+* LUCENE-10208: Ensure that the minimum competitive score does not decrease in concurrent search. (Jim Ferenczi, Adrien Grand)
+
 Build
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreAccumulator.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreAccumulator.java
index 5265c89..ab7a5e1 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreAccumulator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreAccumulator.java
@@ -27,7 +27,7 @@ final class MaxScoreAccumulator {
   static final int DEFAULT_INTERVAL = 0x3ff;
 
   // scores are always positive
-  final LongAccumulator acc = new LongAccumulator(Long::max, Long.MIN_VALUE);
+  final LongAccumulator acc = new LongAccumulator(MaxScoreAccumulator::maxEncode, Long.MIN_VALUE);
 
   // non-final and visible for tests
   long modInterval;
@@ -36,9 +36,26 @@ final class MaxScoreAccumulator {
     this.modInterval = DEFAULT_INTERVAL;
   }
 
-  void accumulate(int docID, float score) {
-    assert docID >= 0 && score >= 0;
-    long encode = (((long) Float.floatToIntBits(score)) << 32) | docID;
+  /**
+   * Return the max encoded DocAndScore in a way that is consistent with {@link
+   * DocAndScore#compareTo}.
+   */
+  private static long maxEncode(long v1, long v2) {
+    float score1 = Float.intBitsToFloat((int) (v1 >> 32));
+    float score2 = Float.intBitsToFloat((int) (v2 >> 32));
+    int cmp = Float.compare(score1, score2);
+    if (cmp == 0) {
+      // tie-break on the minimum doc base
+      return (int) v1 < (int) v2 ? v1 : v2;
+    } else if (cmp > 0) {
+      return v1;
+    }
+    return v2;
+  }
+
+  void accumulate(int docBase, float score) {
+    assert docBase >= 0 && score >= 0;
+    long encode = (((long) Float.floatToIntBits(score)) << 32) | docBase;
     acc.accumulate(encode);
   }
 
@@ -48,16 +65,16 @@ final class MaxScoreAccumulator {
       return null;
     }
     float score = Float.intBitsToFloat((int) (value >> 32));
-    int docID = (int) value;
-    return new DocAndScore(docID, score);
+    int docBase = (int) value;
+    return new DocAndScore(docBase, score);
   }
 
   static class DocAndScore implements Comparable<DocAndScore> {
-    final int docID;
+    final int docBase;
     final float score;
 
-    DocAndScore(int docID, float score) {
-      this.docID = docID;
+    DocAndScore(int docBase, float score) {
+      this.docBase = docBase;
       this.score = score;
     }
 
@@ -65,7 +82,14 @@ final class MaxScoreAccumulator {
     public int compareTo(DocAndScore o) {
       int cmp = Float.compare(score, o.score);
       if (cmp == 0) {
-        return Integer.compare(docID, o.docID);
+        // tie-break on the minimum doc base
+        // For a given minimum competitive score, we want to know the first segment
+        // where this score occurred, hence the reverse order here.
+        // On segments with a lower docBase, any document whose score is greater
+        // than or equal to this score would be competitive, while on segments with a
+        // higher docBase, documents need to have a strictly greater score to be
+        // competitive since we tie break on doc ID.
+        return Integer.compare(o.docBase, docBase);
       }
       return cmp;
     }
@@ -75,14 +99,14 @@ final class MaxScoreAccumulator {
       if (this == o) return true;
       if (o == null || getClass() != o.getClass()) return false;
       DocAndScore result = (DocAndScore) o;
-      return docID == result.docID &&
+      return docBase == result.docBase &&
           Float.compare(result.score, score) == 0;
     }
 
     @Override
     public String toString() {
       return "DocAndScore{" +
-          "docID=" + docID +
+          "docBase=" + docBase +
           ", score=" + score +
           '}';
     }
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 ec33d7c..ed3ad09 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -111,6 +111,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
 
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      // reset the minimum competitive score
+      minCompetitiveScore = 0f;
       docBase = context.docBase;
 
       // as all segments are sorted in the same way, enough to check only the 1st segment for indexSort
@@ -132,9 +134,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
         @Override
         public void setScorer(Scorable scorer) throws IOException {
           super.setScorer(scorer);
-          minCompetitiveScore = 0f;
-          updateMinCompetitiveScore(scorer);
-          if (minScoreAcc != null) {
+          if (minScoreAcc == null) {
+            updateMinCompetitiveScore(scorer);
+          } else {
             updateGlobalMinCompetitiveScore(scorer);
           }
         }
@@ -231,6 +233,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
 
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      // reset the minimum competitive score
+      minCompetitiveScore = 0f;
       docBase = context.docBase;
       final int afterDoc = after.doc - docBase;
       // as all segments are sorted in the same way, enough to check only the 1st segment for indexSort
@@ -249,9 +253,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
         @Override
         public void setScorer(Scorable scorer) throws IOException {
           super.setScorer(scorer);
-          minCompetitiveScore = 0f;
-          updateMinCompetitiveScore(scorer);
-          if (minScoreAcc != null) {
+          if (minScoreAcc == null) {
+            updateMinCompetitiveScore(scorer);
+          } else {
             updateGlobalMinCompetitiveScore(scorer);
           }
         }
@@ -425,7 +429,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
         minCompetitiveScore = minScore;
         totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
         if (minScoreAcc != null) {
-          minScoreAcc.accumulate(bottom.doc, minScore);
+          minScoreAcc.accumulate(docBase, minScore);
         }
       }
     }
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
index adbbafc..ae06cca 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
@@ -58,15 +58,16 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       // reset the minimum competitive score
+      minCompetitiveScore = 0f;
       docBase = context.docBase;
-      return new ScorerLeafCollector() {
 
+      return new ScorerLeafCollector() {
         @Override
         public void setScorer(Scorable scorer) throws IOException {
           super.setScorer(scorer);
-          minCompetitiveScore = 0f;
-          updateMinCompetitiveScore(scorer);
-          if (minScoreAcc != null) {
+          if (minScoreAcc == null) {
+            updateMinCompetitiveScore(scorer);
+          } else {
             updateGlobalMinCompetitiveScore(scorer);
           }
         }
@@ -133,11 +134,23 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
 
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      // reset the minimum competitive score
+      minCompetitiveScore = 0f;
       docBase = context.docBase;
       final int afterDoc = after.doc - context.docBase;
 
       return new ScorerLeafCollector() {
         @Override
+        public void setScorer(Scorable scorer) throws IOException {
+          super.setScorer(scorer);
+          if (minScoreAcc == null) {
+            updateMinCompetitiveScore(scorer);
+          } else {
+            updateGlobalMinCompetitiveScore(scorer);
+          }
+        }
+
+        @Override
         public void collect(int doc) throws IOException {
           float score = scorer.score();
 
@@ -307,7 +320,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
       // since we tie-break on doc id and collect in doc id order we can require
       // the next float if the global minimum score is set on a document id that is
       // smaller than the ids in the current leaf
-      float score = docBase > maxMinScore.docID ? Math.nextUp(maxMinScore.score) : maxMinScore.score;
+      float score = docBase >= maxMinScore.docBase ? Math.nextUp(maxMinScore.score) : maxMinScore.score;
       if (score > minCompetitiveScore) {
         assert hitsThresholdChecker.isThresholdReached();
         scorer.setMinCompetitiveScore(score);
@@ -332,7 +345,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
           // we don't use the next float but we register the document
           // id so that other leaves can require it if they are after
           // the current maximum
-          minScoreAcc.accumulate(pqTop.doc, pqTop.score);
+          minScoreAcc.accumulate(docBase, pqTop.score);
         }
       }
     }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreAccumulator.java b/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreAccumulator.java
index c656d47..fb0c519 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreAccumulator.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreAccumulator.java
@@ -23,21 +23,29 @@ public class TestMaxScoreAccumulator extends LuceneTestCase {
   public void testSimple() {
     MaxScoreAccumulator acc = new MaxScoreAccumulator();
     acc.accumulate(0, 0f);
+    assertEquals(0f, acc.get().score, 0);
+    assertEquals(0, acc.get().docBase, 0);
     acc.accumulate(10, 0f);
     assertEquals(0f, acc.get().score, 0);
-    assertEquals(10, acc.get().docID, 0);
+    assertEquals(0, acc.get().docBase, 0);
     acc.accumulate(100, 1000f);
     assertEquals(1000f, acc.get().score, 0);
-    assertEquals(100, acc.get().docID, 0);
+    assertEquals(100, acc.get().docBase, 0);
     acc.accumulate(1000, 5f);
     assertEquals(1000f, acc.get().score, 0);
-    assertEquals(100, acc.get().docID, 0);
+    assertEquals(100, acc.get().docBase, 0);
     acc.accumulate(99, 1000f);
     assertEquals(1000f, acc.get().score, 0);
-    assertEquals(100, acc.get().docID, 0);
-    acc.accumulate(0, 1001f);
+    assertEquals(99, acc.get().docBase, 0);
+    acc.accumulate(1000, 1001f);
+    assertEquals(1001f, acc.get().score, 0);
+    assertEquals(1000, acc.get().docBase, 0);
+    acc.accumulate(10, 1001f);
+    assertEquals(1001f, acc.get().score, 0);
+    assertEquals(10, acc.get().docBase, 0);
+    acc.accumulate(100, 1001f);
     assertEquals(1001f, acc.get().score, 0);
-    assertEquals(0, acc.get().docID, 0);
+    assertEquals(10, acc.get().docBase, 0);
   }
 
   public void testRandom() {
@@ -47,7 +55,7 @@ public class TestMaxScoreAccumulator extends LuceneTestCase {
     MaxScoreAccumulator.DocAndScore max = new MaxScoreAccumulator.DocAndScore(-1, -1);
     for (int i = 0; i < numDocs; i++) {
       MaxScoreAccumulator.DocAndScore res = new MaxScoreAccumulator.DocAndScore(random().nextInt(maxDocs), random().nextFloat());
-      acc.accumulate(res.docID, res.score);
+      acc.accumulate(res.docBase, res.score);
       if (res.compareTo(max) > 0) {
         max = res;
       }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
index 544094a..902835e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
@@ -19,10 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -43,7 +39,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.NamedThreadFactory;
 
 public class TestTopDocsCollector extends LuceneTestCase {
 
@@ -113,26 +108,17 @@ public class TestTopDocsCollector extends LuceneTestCase {
   }
 
   private TopDocsCollector<ScoreDoc> doSearchWithThreshold(int numResults, int thresHold, Query q, IndexReader indexReader) throws IOException {
-    IndexSearcher searcher = new IndexSearcher(indexReader);
+    IndexSearcher searcher = newSearcher(indexReader, true, true, false);
     TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(numResults, thresHold);
     searcher.search(q, tdc);
     return tdc;
   }
 
   private TopDocs doConcurrentSearchWithThreshold(int numResults, int threshold, Query q, IndexReader indexReader) throws IOException {
-    ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>(),
-        new NamedThreadFactory("TestTopDocsCollector"));
-    try {
-      IndexSearcher searcher = new IndexSearcher(indexReader, service);
-
-      CollectorManager<TopScoreDocCollector,TopDocs> collectorManager = TopScoreDocCollector.createSharedManager(numResults,
-          null, threshold);
-
-      return searcher.search(q, collectorManager);
-    } finally {
-      service.shutdown();
-    }
+    IndexSearcher searcher = newSearcher(indexReader, true, true, true);
+    CollectorManager<TopScoreDocCollector,TopDocs> collectorManager = TopScoreDocCollector.createSharedManager(numResults,
+            null, threshold);
+    return searcher.search(q, collectorManager);
   }
   
   @Override
@@ -236,8 +222,9 @@ public class TestTopDocsCollector extends LuceneTestCase {
     Float minCompetitiveScore = null;
 
     @Override
-    public void setMinCompetitiveScore(float minCompetitiveScore) {
-      this.minCompetitiveScore = minCompetitiveScore;
+    public void setMinCompetitiveScore(float score) {
+      assert minCompetitiveScore == null || score >= minCompetitiveScore;
+      this.minCompetitiveScore = score;
     }
 
     @Override
@@ -288,9 +275,9 @@ public class TestTopDocsCollector extends LuceneTestCase {
     scorer.doc = 3;
     scorer.score = 0.5f;
     // Make sure we do not call setMinCompetitiveScore for non-competitive hits
-    scorer.minCompetitiveScore = Float.NaN;
+    scorer.minCompetitiveScore = null;
     leafCollector.collect(3);
-    assertTrue(Float.isNaN(scorer.minCompetitiveScore));
+    assertNull(scorer.minCompetitiveScore);
 
     scorer.doc = 4;
     scorer.score = 4;
@@ -536,6 +523,10 @@ public class TestTopDocsCollector extends LuceneTestCase {
     assertEquals(11, topDocs.totalHits.value);
     assertEquals(new TotalHits(11, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), topDocs.totalHits);
 
+    leafCollector.setScorer(scorer);
+    leafCollector2.setScorer(scorer2);
+    leafCollector3.setScorer(scorer3);
+
     reader.close();
     dir.close();
   }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
index 7759891..0717b44 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java
@@ -20,10 +20,6 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -75,28 +71,18 @@ public class TestTopFieldCollector extends LuceneTestCase {
   }
 
   private TopFieldCollector doSearchWithThreshold(int numResults, int thresHold, Query q, Sort sort, IndexReader indexReader) throws IOException {
-    IndexSearcher searcher = new IndexSearcher(indexReader);
+    IndexSearcher searcher = newSearcher(indexReader, true, true, false);
     TopFieldCollector tdc = TopFieldCollector.create(sort, numResults, thresHold);
     searcher.search(q, tdc);
     return tdc;
   }
 
   private TopDocs doConcurrentSearchWithThreshold(int numResults, int threshold, Query q, Sort sort, IndexReader indexReader) throws IOException {
-    ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>(),
-        new NamedThreadFactory("TestTopDocsCollector"));
-    try {
-      IndexSearcher searcher = new IndexSearcher(indexReader, service);
-
-      CollectorManager<TopFieldCollector,TopFieldDocs> collectorManager = TopFieldCollector.createSharedManager(sort, numResults,
-          null, threshold);
-
-      TopDocs tdc = searcher.search(q, collectorManager);
-
-      return tdc;
-    } finally {
-      service.shutdown();
-    }
+    IndexSearcher searcher = newSearcher(indexReader, true, true, true);
+    CollectorManager<TopFieldCollector,TopFieldDocs> collectorManager = TopFieldCollector.createSharedManager(sort, numResults,
+            null, threshold);
+    TopDocs tdc = searcher.search(q, collectorManager);
+    return tdc;
   }
   
   public void testSortWithoutFillFields() throws Exception {
@@ -140,12 +126,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
   }
 
   public void testSharedHitcountCollector() throws Exception {
-
-    ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>(),
-        new NamedThreadFactory("TestTopFieldCollector"));
-
-    IndexSearcher concurrentSearcher = new IndexSearcher(ir, service);
+    IndexSearcher concurrentSearcher = newSearcher(ir, true, true, true);
 
     // Two Sort criteria to instantiate the multi/single comparators.
     Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() };
@@ -166,8 +147,6 @@ public class TestTopFieldCollector extends LuceneTestCase {
 
       CheckHits.checkEqual(q, td.scoreDocs, td2.scoreDocs);
     }
-
-    service.shutdown();
   }
 
   public void testSortWithoutTotalHitTracking() throws Exception {
@@ -646,6 +625,10 @@ public class TestTopFieldCollector extends LuceneTestCase {
     assertEquals(11, topDocs.totalHits.value);
     assertEquals(new TotalHits(11, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), topDocs.totalHits);
 
+    leafCollector.setScorer(scorer);
+    leafCollector2.setScorer(scorer2);
+    leafCollector3.setScorer(scorer3);
+
     reader.close();
     dir.close();
   }