Update SimpleStreamSearcher.java
This commit is contained in:
parent
918ff71091
commit
20069e7e06
@ -32,30 +32,44 @@ public class SimpleStreamSearcher implements LuceneStreamSearcher {
|
||||
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
|
||||
1000
|
||||
);
|
||||
var firstTopDocs = searcher.getTopDocs(0, 1);
|
||||
long totalHitsCount = firstTopDocs.totalHits.value;
|
||||
return new LuceneSearchInstance() {
|
||||
@Override
|
||||
public long getTotalHitsCount() throws IOException {
|
||||
return searcher.getTopDocs(0, 1).totalHits.value;
|
||||
public long getTotalHitsCount() {
|
||||
return totalHitsCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getResults(ResultItemConsumer resultsConsumer) throws IOException {
|
||||
ObjectArrayList<ScoreDoc> hits = ObjectArrayList.wrap(searcher.getTopDocs(offset, limit).scoreDocs);
|
||||
for (ScoreDoc hit : hits) {
|
||||
int docId = hit.doc;
|
||||
float score = hit.score;
|
||||
if (LuceneUtils.collectTopDoc(logger,
|
||||
docId,
|
||||
score,
|
||||
minCompetitiveScore,
|
||||
indexSearcher,
|
||||
keyFieldName,
|
||||
resultsConsumer
|
||||
) == HandleResult.HALT) {
|
||||
return;
|
||||
if (firstTopDocs.scoreDocs.length > 0) {
|
||||
{
|
||||
var hit = firstTopDocs.scoreDocs[0];
|
||||
if (publishHit(hit, resultsConsumer) == HandleResult.HALT) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
ObjectArrayList<ScoreDoc> hits = ObjectArrayList.wrap(searcher.getTopDocs(offset, limit - 1).scoreDocs);
|
||||
for (ScoreDoc hit : hits) {
|
||||
if (publishHit(hit, resultsConsumer) == HandleResult.HALT) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private HandleResult publishHit(ScoreDoc hit, ResultItemConsumer resultsConsumer) throws IOException {
|
||||
int docId = hit.doc;
|
||||
float score = hit.score;
|
||||
return LuceneUtils.collectTopDoc(logger,
|
||||
docId,
|
||||
score,
|
||||
minCompetitiveScore,
|
||||
indexSearcher,
|
||||
keyFieldName,
|
||||
resultsConsumer
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user