mikemccand commented on a change in pull request #854: Shared PQ Based Early Termination for Concurrent Search URL: https://github.com/apache/lucene-solr/pull/854#discussion_r323176113
########## File path: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java ########## @@ -279,7 +297,130 @@ public void collect(int doc) throws IOException { } - private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0]; + /* + * Collects hits into a local queue until the requested number of hits are collected + * globally. Post that, a global calibration step is performed + */ + public static class EarlyTerminatingFieldCollector extends TopFieldCollector { + + final Sort sort; + final FieldValueHitQueue<Entry> queue; + final EarlyTerminatingFieldCollectorManager earlyTerminatingFieldCollectorManager; + private final AtomicInteger globalNumberOfHits; + private boolean addedSelfToGlobalQueue; + + //TODO: Refactor this to make an interface only for field collector uses + public EarlyTerminatingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, int totalHitsThreshold, + EarlyTerminatingFieldCollectorManager collectorManager, AtomicInteger globalNumberOfHits) { + super(queue, numHits, totalHitsThreshold, sort.needsScores()); + this.sort = sort; + this.queue = queue; + this.earlyTerminatingFieldCollectorManager = collectorManager; + this.globalNumberOfHits = globalNumberOfHits; + } + + @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 EarlyTerminatingMultiComparatorLeafCollector(comparators, reverseMul, this) { + + boolean collectedAllCompetitiveHits = false; + + @Override + public void setScorer(Scorable scorer) throws IOException { + super.setScorer(scorer); + updateMinCompetitiveScore(scorer); + } + + @Override + public void collect(int doc) throws IOException { + + if (globalNumberOfHits.incrementAndGet() > numHits) { + if (addedSelfToGlobalQueue == false) { + Entry returnedEntry = earlyTerminatingFieldCollectorManager.addCollectorToGlobalQueue(earlyTerminatingFieldCollector, docBase); + + if (returnedEntry != null) { + filterCompetitiveHit(returnedEntry.doc, false, returnedEntry.values); + + if (queue.size() > 0) { + Entry entry = queue.pop(); + + while (entry != null) { + filterCompetitiveHit(entry.doc, false, entry.values); + entry = queue.pop(); + } + } + } + addedSelfToGlobalQueue = true; Review comment: I wish there were some way to swap in a different collector implementation once we absorb ourselves into the global queue ... the behavior of this collector is so modal depending on 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: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org