You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by GitBox <gi...@apache.org> on 2019/07/25 18:11:50 UTC

[GitHub] [lucene-solr] jpountz commented on a change in pull request #803: LUCENE-8929: Early Terminating CollectorManager with Global Hitcount

jpountz commented on a change in pull request #803: LUCENE-8929: Early Terminating CollectorManager with Global Hitcount
URL: https://github.com/apache/lucene-solr/pull/803#discussion_r307432751
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
 ##########
 @@ -279,6 +280,125 @@ public void collect(int doc) throws IOException {
 
   }
 
+  /*
+   * Implements a TopFieldCollector that terminates early based on a global
+   * scoreboard which is shared amongst multiple collectors.
+   * NOTE: This should ideally be outside of TopFieldCollector since it does
+   * not have private access, but we keep it here to limit the visibility
+   * of dependent classes
+   */
+  public static class GlobalStateFieldCollector extends TopFieldCollector {
+
+    final Sort sort;
+    final FieldValueHitQueue<Entry> queue;
+    final AtomicInteger globalTotalHits;
+
+    final GlobalStateCollectorManager.GlobalStateCallback callback;
+
+    public GlobalStateFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, int totalHitsThreshold,
+                                     AtomicInteger globalTotalHits, GlobalStateCollectorManager.GlobalStateCallback callback) {
+      super(queue, numHits, totalHitsThreshold, sort.needsScores());
+      this.sort = sort;
+      this.queue = queue;
+      this.globalTotalHits = globalTotalHits;
+      this.callback = callback;
+    }
+
+    @Override
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
+
+      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) {
+
+        boolean collectedAllCompetitiveHits = false;
+
+        @Override
+        public void setScorer(Scorable scorer) throws IOException {
+          super.setScorer(scorer);
+          updateMinCompetitiveScore(scorer);
+        }
+
+        @Override
+        public void collect(int doc) throws IOException {
+          // Increment local hit counter
+          totalHits++;
+
+          if (queueFull) {
+            if (collectedAllCompetitiveHits || !isHitCompetitive(doc, scorer)) {
+              // 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) {
+                // Check the global scoreboard to see total hits accumulated yet
+                if (globalTotalHits.incrementAndGet() > totalHitsThreshold) {
+                  totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
+                  throw new CollectionTerminatedException();
+                } else {
+                  collectedAllCompetitiveHits = true;
+                }
+              } else if (totalHitsRelation == Relation.EQUAL_TO) {
+                // we just reached totalHitsThreshold, we can start setting the min
+                // competitive score now
+                //TODO: Should we also update competitive score globally?
+                updateMinCompetitiveScore(scorer);
+              }
+              return;
+            }
+
+            // This hit is competitive - replace bottom element in queue & adjustTop
+            comparator.copy(bottom.slot, doc);
+            updateBottom(doc);
+            comparator.setBottom(bottom.slot);
+            updateMinCompetitiveScore(scorer);
+            //Increment global hit counter
+            globalTotalHits.incrementAndGet();
+          } else {
+            // Startup transient: queue hasn't gathered numHits yet
+
+            //Increment global hit counter
+            globalTotalHits.incrementAndGet();
+
+            final int slot = totalHits - 1;
+
+            // Copy hit into queue
+            comparator.copy(slot, doc);
+            add(slot, doc);
+            if (queueFull) {
+              comparator.setBottom(bottom.slot);
+              updateMinCompetitiveScore(scorer);
+            }
+          }
+        }
+
+        // Check if hit is competitive and set the global value accordingly
+        private boolean isHitCompetitive(int doc, Scorable scorer) throws IOException {
+          // Check if hit is locally competitive
+          if (reverseMul * comparator.compareBottom(doc) > 0) {
+            // Hit was competitive locally, but was it globally competitive?
+            if (callback.getGlobalMinCompetitiveScore() > scorer.score()) {
+              return false;
+            } else {
+              // Hit was locally and globally competitive, set the right
+              // global minimum competitive score
+              callback.checkAndUpdateMinCompetitiveScore(scorer.score());
+              return true;
+            }
+          }
+
+          // Hit was not locally competitive hence it cannot
+          // be globally competitive
+          return false;
+        }
 
 Review comment:
   I think you got a bit confused between field sort and score sort here. TopFieldCollector is about sorting by field, which can be score with SortField.SCORE, but is generally not the case. This logic of sharing score information between collectors about the min competitive score would be more applicable to TopScoreCollector, or TopFieldCollector when the first sort field is Field.SCORE.
   
   If you want to check whether a hit is globally competitive with TopFieldCollector, you'd need to track hits of all slices in a single hit 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: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org