atris 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_r307656294
##########
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:
> > So, we will need a thread safe implementation of PriorityQueue to work
in a shared manner across all the collectors belong to
EarlyTerminatingCollectorManager?
>
> I think it's an idea worth playing with indeed. I'm curious whether
@msokolov has thoughts on this, since it has some intersection with pro-rated
collection given that it helps avoid extra work when collecting multiple slices
in parallel.
I was thinking if it is worth collecting in individual PQs until all
collectors have collected numHits, then perform a merge operation and populate
a global shared PQ with the top hits across all of the numHits hits. Post that,
the second phase of collection can operate on the shared queue.
This can help in cases where number of relevant hits < numHits, and for
cases where numHits is close to the totalHitsThreshold.
Thoughts?
----------------------------------------------------------------
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:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]