Add minCompetitiveScore

This commit is contained in:
Andrea Cavalli 2021-02-14 13:46:11 +01:00
parent b3e3f2aca4
commit bb3afd291e
15 changed files with 215 additions and 82 deletions

View File

@ -4,6 +4,7 @@ import it.cavallium.dbengine.database.LLDocument;
import it.cavallium.dbengine.database.LLItem;
import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLScoreMode;
import it.cavallium.dbengine.database.LLSort;
import it.cavallium.dbengine.database.LLTerm;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer;
@ -34,7 +35,14 @@ public class IndicizationExample {
})
)
.then(index.refresh())
.then(index.search(null, Query.exactSearch(TextFieldsAnalyzer.N4GramPartialString,"name", "Mario"), 1, null, LLScoreMode.COMPLETE, "id"))
.then(index.search(null,
Query.exactSearch(TextFieldsAnalyzer.N4GramPartialString, "name", "Mario"),
1,
LLSort.newSortScore(),
LLScoreMode.COMPLETE,
null,
"id"
))
.flatMap(results -> results
.results()
.flatMap(r -> r)
@ -99,7 +107,7 @@ public class IndicizationExample {
))
.then(index.refresh())
.then(index.search(null, Query.exactSearch(TextFieldsAnalyzer.N4GramPartialString,"name", "Mario"), 10, MultiSort.topScore()
.getQuerySort(), LLScoreMode.COMPLETE, "id"))
.getQuerySort(), LLScoreMode.COMPLETE, null, "id"))
.flatMap(results -> LuceneUtils.mergeStream(results
.results(), MultiSort.topScoreRaw(), 10)
.doOnNext(value -> System.out.println("Value: " + value))

View File

@ -120,11 +120,13 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
public Mono<SearchResultKeys<T>> moreLikeThis(@Nullable CompositeSnapshot snapshot,
T key,
U mltDocumentValue,
int limit) {
int limit,
@Nullable Float minCompetitiveScore) {
Flux<Tuple2<String, Set<String>>> mltDocumentFields
= indicizer.getMoreLikeThisDocumentFields(key, mltDocumentValue);
return luceneIndex
.moreLikeThis(resolveSnapshot(snapshot), mltDocumentFields, limit, indicizer.getKeyFieldName())
.moreLikeThis(resolveSnapshot(snapshot), mltDocumentFields, limit,
minCompetitiveScore, indicizer.getKeyFieldName())
.map(llSearchResult -> this.transformLuceneResult(llSearchResult, null, LLScoreMode.TOP_SCORES, limit));
}
@ -140,11 +142,13 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
T key,
U mltDocumentValue,
int limit,
@Nullable Float minCompetitiveScore,
ValueGetter<T, U> valueGetter) {
Flux<Tuple2<String, Set<String>>> mltDocumentFields
= indicizer.getMoreLikeThisDocumentFields(key, mltDocumentValue);
return luceneIndex
.moreLikeThis(resolveSnapshot(snapshot), mltDocumentFields, limit, indicizer.getKeyFieldName())
.moreLikeThis(resolveSnapshot(snapshot), mltDocumentFields, limit,
minCompetitiveScore, indicizer.getKeyFieldName())
.map(llSearchResult ->
this.transformLuceneResultWithValues(llSearchResult, null, LLScoreMode.TOP_SCORES, limit, valueGetter));
}
@ -160,10 +164,12 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
Query query,
int limit,
@Nullable MultiSort<SearchResultKey<T>> sort,
LLScoreMode scoreMode) {
LLScoreMode scoreMode,
@Nullable Float minCompetitiveScore) {
LLSort querySort = sort != null ? sort.getQuerySort() : null;
return luceneIndex
.search(resolveSnapshot(snapshot), query, limit, querySort, scoreMode, indicizer.getKeyFieldName())
.search(resolveSnapshot(snapshot), query, limit, querySort, scoreMode, minCompetitiveScore,
indicizer.getKeyFieldName())
.map(llSearchResult -> this.transformLuceneResult(llSearchResult, sort, scoreMode, limit));
}
@ -179,15 +185,17 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
int limit,
@Nullable MultiSort<SearchResultItem<T, U>> sort,
LLScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
ValueGetter<T, U> valueGetter) {
LLSort querySort = sort != null ? sort.getQuerySort() : null;
return luceneIndex
.search(resolveSnapshot(snapshot), query, limit, querySort, scoreMode, indicizer.getKeyFieldName())
.search(resolveSnapshot(snapshot), query, limit, querySort, scoreMode, minCompetitiveScore,
indicizer.getKeyFieldName())
.map(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, sort, scoreMode, limit, valueGetter));
}
public Mono<Long> count(@Nullable CompositeSnapshot snapshot, Query query) {
return this.search(snapshot, query, 0, null, null)
return this.search(snapshot, query, 0, null, null, null)
.flatMap(SearchResultKeys::totalHitsCount)
.single();
}

View File

@ -34,6 +34,7 @@ public interface LLLuceneIndex extends LLSnapshottable {
Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFields,
int limit,
@Nullable Float minCompetitiveScore,
String keyFieldName);
/**
@ -48,10 +49,11 @@ public interface LLLuceneIndex extends LLSnapshottable {
int limit,
@Nullable LLSort sort,
LLScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName);
default Mono<Long> count(@Nullable LLSnapshot snapshot, Query query) {
return this.search(snapshot, query, 0, null, null, null)
return this.search(snapshot, query, 0, null, null, null, null)
.flatMap(LLSearchResult::totalHitsCount)
.single();
}

View File

@ -1,5 +1,7 @@
package it.cavallium.dbengine.database.disk;
import static it.cavallium.dbengine.lucene.LuceneUtils.checkScoringArgumentsValidity;
import it.cavallium.dbengine.database.LLDocument;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.database.LLLuceneIndex;
@ -336,23 +338,26 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
int limit,
@Nullable Float minCompetitiveScore,
String keyFieldName) {
return moreLikeThis(snapshot, mltDocumentFieldsFlux, limit, keyFieldName, false, 0, 1);
return moreLikeThis(snapshot, mltDocumentFieldsFlux, limit, minCompetitiveScore, keyFieldName, false, 0, 1);
}
public Mono<LLSearchResult> distributedMoreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
int limit,
@Nullable Float minCompetitiveScore,
String keyFieldName,
long actionId,
int scoreDivisor) {
return moreLikeThis(snapshot, mltDocumentFieldsFlux, limit, keyFieldName, false, actionId, scoreDivisor);
return moreLikeThis(snapshot, mltDocumentFieldsFlux, limit, minCompetitiveScore, keyFieldName, false, actionId, scoreDivisor);
}
public Mono<Void> distributedPreMoreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
@Nullable Float minCompetitiveScore,
String keyFieldName, long actionId) {
return moreLikeThis(snapshot, mltDocumentFieldsFlux, -1, keyFieldName, true, actionId, 1)
return moreLikeThis(snapshot, mltDocumentFieldsFlux, -1, minCompetitiveScore, keyFieldName, true, actionId, 1)
.flatMap(LLSearchResult::completion);
}
@ -360,6 +365,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
private Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
int limit,
@Nullable Float minCompetitiveScore,
String keyFieldName,
boolean doDistributedPre,
long actionId,
@ -406,6 +412,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
limit,
null,
ScoreMode.TOP_SCORES,
minCompetitiveScore,
keyFieldName,
keyScore -> {
EmitResult result = topKeysSink.tryEmitNext(fixKeyScore(keyScore, scoreDivisor));
@ -445,29 +452,34 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
@Override
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, int limit,
@Nullable LLSort sort, LLScoreMode scoreMode, String keyFieldName) {
return search(snapshot, query, limit, sort, scoreMode, keyFieldName, false, 0, 1);
@Nullable LLSort sort, LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName) {
return search(snapshot, query, limit, sort, scoreMode, minCompetitiveScore,
keyFieldName, false, 0, 1);
}
public Mono<LLSearchResult> distributedSearch(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, int limit,
@Nullable LLSort sort, LLScoreMode scoreMode, String keyFieldName, long actionId, int scoreDivisor) {
return search(snapshot, query, limit, sort, scoreMode, keyFieldName, false, actionId, scoreDivisor);
@Nullable LLSort sort, LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName, long actionId, int scoreDivisor) {
return search(snapshot, query, limit, sort, scoreMode, minCompetitiveScore,
keyFieldName, false, actionId, scoreDivisor);
}
public Mono<Void> distributedPreSearch(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query,
@Nullable LLSort sort, LLScoreMode scoreMode, String keyFieldName, long actionId) {
return search(snapshot, query, -1, sort, scoreMode, keyFieldName, true, actionId, 1)
@Nullable LLSort sort, LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName, long actionId) {
return this
.search(snapshot, query, -1, sort, scoreMode,
minCompetitiveScore, keyFieldName, true, actionId, 1)
.flatMap(LLSearchResult::completion);
}
@SuppressWarnings("Convert2MethodRef")
private Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
it.cavallium.dbengine.lucene.serializer.Query query, int limit,
@Nullable LLSort sort, LLScoreMode scoreMode, String keyFieldName,
@Nullable LLSort sort, LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName,
boolean doDistributedPre, long actionId, int scoreDivisor) {
return acquireSearcherWrapper(snapshot, doDistributedPre, actionId)
.flatMap(indexSearcher -> Mono
.fromCallable(() -> {
checkScoringArgumentsValidity(sort, scoreMode);
Query luceneQuery = QueryParser.parse(query);
Sort luceneSort = LLUtils.toSort(sort);
org.apache.lucene.search.ScoreMode luceneScoreMode = LLUtils.toScoreMode(scoreMode);
@ -496,6 +508,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
limit,
luceneSort,
luceneScoreMode,
minCompetitiveScore,
keyFieldName,
keyScore -> {
EmitResult result = topKeysSink.tryEmitNext(fixKeyScore(keyScore, scoreDivisor));

View File

@ -204,6 +204,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
Flux<Tuple2<String, Set<String>>> mltDocumentFields,
int limit,
@Nullable Float minCompetitiveScore,
String keyFieldName) {
long actionId;
int scoreDivisor;
@ -222,7 +223,12 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
)
.flatMap(tuple -> tuple
.getT1()
.distributedPreMoreLikeThis(tuple.getT2().orElse(null), mltDocumentFieldsShared, keyFieldName, actionId)
.distributedPreMoreLikeThis(tuple.getT2().orElse(null),
mltDocumentFieldsShared,
minCompetitiveScore,
keyFieldName,
actionId
)
)
.then();
} else {
@ -243,6 +249,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
.distributedMoreLikeThis(tuple.getT2().orElse(null),
mltDocumentFieldsShared,
limit,
minCompetitiveScore,
keyFieldName,
actionId,
scoreDivisor
@ -268,6 +275,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
int limit,
@Nullable LLSort sort,
LLScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) {
long actionId;
int scoreDivisor;
@ -288,7 +296,14 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
)
.flatMap(tuple -> tuple
.getT1()
.distributedPreSearch(tuple.getT2().orElse(null), query, sort, scoreMode, keyFieldName, actionId)
.distributedPreSearch(tuple.getT2().orElse(null),
query,
sort,
scoreMode,
minCompetitiveScore,
keyFieldName,
actionId
)
)
.then();
}
@ -307,6 +322,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
limit,
sort,
scoreMode,
minCompetitiveScore,
keyFieldName,
actionId,
scoreDivisor

View File

@ -8,11 +8,14 @@ import org.apache.lucene.search.Collector;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.jetbrains.annotations.Nullable;
public class LuceneParallelStreamCollector implements Collector, LeafCollector {
private final int base;
private final ScoreMode scoreMode;
@Nullable
private final Float minCompetitiveScore;
private final LuceneParallelStreamConsumer streamConsumer;
private final AtomicBoolean stopped;
private final AtomicLong totalHitsCounter;
@ -20,11 +23,13 @@ public class LuceneParallelStreamCollector implements Collector, LeafCollector {
public LuceneParallelStreamCollector(int base,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
LuceneParallelStreamConsumer streamConsumer,
AtomicBoolean stopped,
AtomicLong totalHitsCounter) {
this.base = base;
this.scoreMode = scoreMode;
this.minCompetitiveScore = minCompetitiveScore;
this.streamConsumer = streamConsumer;
this.stopped = stopped;
this.totalHitsCounter = totalHitsCounter;
@ -34,6 +39,7 @@ public class LuceneParallelStreamCollector implements Collector, LeafCollector {
public final LeafCollector getLeafCollector(LeafReaderContext context) {
return new LuceneParallelStreamCollector(context.docBase,
scoreMode,
minCompetitiveScore,
streamConsumer,
stopped,
totalHitsCounter
@ -41,8 +47,11 @@ public class LuceneParallelStreamCollector implements Collector, LeafCollector {
}
@Override
public void setScorer(Scorable scorer) {
public void setScorer(Scorable scorer) throws IOException {
this.scorer = scorer;
if (minCompetitiveScore != null && !minCompetitiveScore.isNaN() && !minCompetitiveScore.isInfinite()) {
scorer.setMinCompetitiveScore(minCompetitiveScore);
}
}
@Override

View File

@ -5,24 +5,30 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.ScoreMode;
import org.jetbrains.annotations.Nullable;
public class LuceneParallelStreamCollectorManager implements
CollectorManager<LuceneParallelStreamCollector, LuceneParallelStreamCollectorResult> {
private final ScoreMode scoreMode;
@Nullable
private final Float minCompetitiveScore;
private final LuceneParallelStreamConsumer streamConsumer;
private final AtomicBoolean stopped;
private final AtomicLong totalHitsCounter;
public static LuceneParallelStreamCollectorManager fromConsumer(
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
LuceneParallelStreamConsumer streamConsumer) {
return new LuceneParallelStreamCollectorManager(scoreMode, streamConsumer);
return new LuceneParallelStreamCollectorManager(scoreMode, minCompetitiveScore, streamConsumer);
}
public LuceneParallelStreamCollectorManager(ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
LuceneParallelStreamConsumer streamConsumer) {
this.scoreMode = scoreMode;
this.minCompetitiveScore = minCompetitiveScore;
this.streamConsumer = streamConsumer;
this.stopped = new AtomicBoolean();
this.totalHitsCounter = new AtomicLong();
@ -30,7 +36,13 @@ public class LuceneParallelStreamCollectorManager implements
@Override
public LuceneParallelStreamCollector newCollector() {
return new LuceneParallelStreamCollector(0, scoreMode, streamConsumer, stopped, totalHitsCounter);
return new LuceneParallelStreamCollector(0,
scoreMode,
minCompetitiveScore,
streamConsumer,
stopped,
totalHitsCounter
);
}
@Override

View File

@ -1,12 +1,19 @@
package it.cavallium.dbengine.lucene;
import it.cavallium.dbengine.client.MultiSort;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.database.LLScoreMode;
import it.cavallium.dbengine.database.LLSort;
import it.cavallium.dbengine.database.LLSortType;
import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity;
import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer;
import it.cavallium.dbengine.lucene.similarity.NGramSimilarity;
import java.io.IOException;
import java.util.Set;
import java.util.function.Consumer;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.LowerCaseFilter;
import org.apache.lucene.analysis.TokenStream;
@ -14,7 +21,10 @@ import org.apache.lucene.analysis.en.EnglishPossessiveFilter;
import org.apache.lucene.analysis.en.KStemFilter;
import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.misc.SweetSpotSimilarity;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.similarities.BooleanSimilarity;
import org.apache.lucene.search.similarities.ClassicSimilarity;
import org.apache.lucene.search.similarities.Similarity;
@ -24,6 +34,7 @@ import org.novasearch.lucene.search.similarities.BM25Similarity.BM25Model;
import org.novasearch.lucene.search.similarities.LdpSimilarity;
import org.novasearch.lucene.search.similarities.LtcSimilarity;
import org.novasearch.lucene.search.similarities.RobertsonSimilarity;
import org.slf4j.Logger;
import reactor.core.publisher.Flux;
public class LuceneUtils {
@ -179,4 +190,39 @@ public class LuceneUtils {
}
});
}
public static void checkScoringArgumentsValidity(LLSort sort, LLScoreMode scoreMode) {
if ((sort == null || sort.getType() != LLSortType.SCORE) && scoreMode != LLScoreMode.COMPLETE_NO_SCORES) {
throw new IllegalArgumentException("You must sort by score if the scores are enabled");
}
}
public static void collectTopDoc(Logger logger,
int docId,
float score,
Float minCompetitiveScore,
IndexSearcher indexSearcher,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer) throws IOException {
if (minCompetitiveScore == null || score >= minCompetitiveScore) {
Document d = indexSearcher.doc(docId, Set.of(keyFieldName));
if (d.getFields().isEmpty()) {
logger.error("The document docId: {}, score: {} is empty.", docId, score);
var realFields = indexSearcher.doc(docId).getFields();
if (!realFields.isEmpty()) {
logger.error("Present fields:");
for (IndexableField field : realFields) {
logger.error(" - {}", field.name());
}
}
} else {
var field = d.getField(keyFieldName);
if (field == null) {
logger.error("Can't get key of document docId: {}, score: {}", docId, score);
} else {
resultsConsumer.accept(new LLKeyScore(field.stringValue(), score));
}
}
}
}
}

View File

@ -34,18 +34,46 @@ public class AdaptiveStreamSearcher implements LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> consumer,
LongConsumer totalHitsConsumer) throws IOException {
if (limit == 0) {
totalHitsConsumer.accept(countStreamSearcher.count(indexSearcher, query));
} else if (luceneSort == null && ENABLE_PARALLEL_COLLECTOR) {
parallelCollectorStreamSearcher.search(indexSearcher, query, limit, null, scoreMode, keyFieldName, consumer, totalHitsConsumer);
parallelCollectorStreamSearcher.search(indexSearcher,
query,
limit,
null,
scoreMode,
minCompetitiveScore,
keyFieldName,
consumer,
totalHitsConsumer
);
} else {
if (luceneSort != null && limit > PagedStreamSearcher.MAX_ITEMS_PER_PAGE) {
pagedStreamSearcher.search(indexSearcher, query, limit, luceneSort, scoreMode, keyFieldName, consumer, totalHitsConsumer);
pagedStreamSearcher.search(indexSearcher,
query,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
consumer,
totalHitsConsumer
);
} else {
simpleStreamSearcher.search(indexSearcher, query, limit, luceneSort, scoreMode, keyFieldName, consumer, totalHitsConsumer);
simpleStreamSearcher.search(indexSearcher,
query,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
consumer,
totalHitsConsumer
);
}
}
}

View File

@ -3,7 +3,6 @@ package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.CompletionException;
import java.util.function.Consumer;
import java.util.function.LongConsumer;
import org.apache.lucene.index.LeafReaderContext;
@ -24,7 +23,7 @@ public class AllowOnlyQueryParsingCollectorStreamSearcher implements LuceneStrea
public void search(IndexSearcher indexSearcher,
Query query) throws IOException {
search(indexSearcher, query, 0, null, null, null, null, null);
search(indexSearcher, query, 0, null, null, null, null, null, null);
}
@Override
@ -33,6 +32,7 @@ public class AllowOnlyQueryParsingCollectorStreamSearcher implements LuceneStrea
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException {
@ -45,6 +45,9 @@ public class AllowOnlyQueryParsingCollectorStreamSearcher implements LuceneStrea
if (scoreMode != null) {
throw new IllegalArgumentException("Score mode not allowed");
}
if (minCompetitiveScore != null) {
throw new IllegalArgumentException("Minimum competitive score not allowed");
}
if (keyFieldName != null) {
throw new IllegalArgumentException("Key field name not allowed");
}
@ -62,12 +65,8 @@ public class AllowOnlyQueryParsingCollectorStreamSearcher implements LuceneStrea
public LeafCollector getLeafCollector(LeafReaderContext context) {
return new LeafCollector() {
@Override
public void setScorer(Scorable scorer) {
try {
scorer.setMinCompetitiveScore(Float.MAX_VALUE);
} catch (IOException e) {
throw new CompletionException(e);
}
public void setScorer(Scorable scorer) throws IOException {
scorer.setMinCompetitiveScore(Float.MAX_VALUE);
}
@Override

View File

@ -21,6 +21,7 @@ public class CountStreamSearcher implements LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException {

View File

@ -23,6 +23,7 @@ public interface LuceneStreamSearcher {
* @param limit the maximum number of results
* @param luceneSort the sorting method used for the search
* @param scoreMode score mode
* @param minCompetitiveScore minimum score accepted
* @param keyFieldName the name of the key field
* @param resultsConsumer the consumer of results
* @param totalHitsConsumer the consumer of total count of results
@ -33,6 +34,7 @@ public interface LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException;

View File

@ -1,12 +1,10 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.io.IOException;
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.LongConsumer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -34,12 +32,22 @@ public class PagedStreamSearcher implements LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException {
if (limit < MAX_ITEMS_PER_PAGE) {
// Use a normal search method because the limit is low
baseStreamSearcher.search(indexSearcher, query, limit, luceneSort, scoreMode, keyFieldName, resultsConsumer, totalHitsConsumer);
baseStreamSearcher.search(indexSearcher,
query,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
resultsConsumer,
totalHitsConsumer
);
return;
}
IntWrapper currentAllowedResults = new IntWrapper(limit);
@ -49,7 +57,7 @@ public class PagedStreamSearcher implements LuceneStreamSearcher {
totalHitsConsumer.accept(lastTopDocs.totalHits.value);
if (lastTopDocs.scoreDocs.length > 0) {
ScoreDoc lastScoreDoc = getLastItem(lastTopDocs.scoreDocs);
consumeHits(currentAllowedResults, lastTopDocs.scoreDocs, indexSearcher, keyFieldName, resultsConsumer);
consumeHits(currentAllowedResults, lastTopDocs.scoreDocs, indexSearcher, minCompetitiveScore, keyFieldName, resultsConsumer);
// Run the searches for each page until the end
boolean finished = currentAllowedResults.var <= 0;
@ -57,7 +65,7 @@ public class PagedStreamSearcher implements LuceneStreamSearcher {
lastTopDocs = indexSearcher.searchAfter(lastScoreDoc, query, MAX_ITEMS_PER_PAGE, luceneSort, scoreMode != ScoreMode.COMPLETE_NO_SCORES);
if (lastTopDocs.scoreDocs.length > 0) {
lastScoreDoc = getLastItem(lastTopDocs.scoreDocs);
consumeHits(currentAllowedResults, lastTopDocs.scoreDocs, indexSearcher, keyFieldName, resultsConsumer);
consumeHits(currentAllowedResults, lastTopDocs.scoreDocs, indexSearcher, minCompetitiveScore, keyFieldName, resultsConsumer);
}
if (lastTopDocs.scoreDocs.length < MAX_ITEMS_PER_PAGE || currentAllowedResults.var <= 0) {
finished = true;
@ -69,6 +77,7 @@ public class PagedStreamSearcher implements LuceneStreamSearcher {
private void consumeHits(IntWrapper currentAllowedResults,
ScoreDoc[] hits,
IndexSearcher indexSearcher,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer) throws IOException {
for (ScoreDoc hit : hits) {
@ -76,24 +85,14 @@ public class PagedStreamSearcher implements LuceneStreamSearcher {
float score = hit.score;
if (currentAllowedResults.var-- > 0) {
Document d = indexSearcher.doc(docId, Set.of(keyFieldName));
if (d.getFields().isEmpty()) {
logger.error("The document docId: {}, score: {} is empty.", docId, score);
var realFields = indexSearcher.doc(docId).getFields();
if (!realFields.isEmpty()) {
logger.error("Present fields:");
for (IndexableField field : realFields) {
logger.error(" - {}", field.name());
}
}
} else {
var field = d.getField(keyFieldName);
if (field == null) {
logger.error("Can't get key of document docId: {}, score: {}", docId, score);
} else {
resultsConsumer.accept(new LLKeyScore(field.stringValue(), score));
}
}
LuceneUtils.collectTopDoc(logger,
docId,
score,
minCompetitiveScore,
indexSearcher,
keyFieldName,
resultsConsumer
);
} else {
break;
}

View File

@ -27,6 +27,7 @@ public class ParallelCollectorStreamSearcher implements LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException {
@ -36,7 +37,7 @@ public class ParallelCollectorStreamSearcher implements LuceneStreamSearcher {
AtomicInteger currentCount = new AtomicInteger();
LuceneParallelStreamCollectorResult result = indexSearcher.search(query, LuceneParallelStreamCollectorManager.fromConsumer(scoreMode, (docId, score) -> {
LuceneParallelStreamCollectorResult result = indexSearcher.search(query, LuceneParallelStreamCollectorManager.fromConsumer(scoreMode, minCompetitiveScore, (docId, score) -> {
if (currentCount.getAndIncrement() >= limit) {
return false;
} else {

View File

@ -1,13 +1,11 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
import java.io.IOException;
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.LongConsumer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -27,6 +25,7 @@ public class SimpleStreamSearcher implements LuceneStreamSearcher {
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Consumer<LLKeyScore> resultsConsumer,
LongConsumer totalHitsConsumer) throws IOException {
@ -41,24 +40,14 @@ public class SimpleStreamSearcher implements LuceneStreamSearcher {
for (ScoreDoc hit : hits) {
int docId = hit.doc;
float score = hit.score;
Document d = indexSearcher.doc(docId, Set.of(keyFieldName));
if (d.getFields().isEmpty()) {
logger.error("The document docId: {}, score: {} is empty.", docId, score);
var realFields = indexSearcher.doc(docId).getFields();
if (!realFields.isEmpty()) {
logger.error("Present fields:");
for (IndexableField field : realFields) {
logger.error(" - {}", field.name());
}
}
} else {
var field = d.getField(keyFieldName);
if (field == null) {
logger.error("Can't get key of document docId: {}, score: {}", docId, score);
} else {
resultsConsumer.accept(new LLKeyScore(field.stringValue(), score));
}
}
LuceneUtils.collectTopDoc(logger,
docId,
score,
minCompetitiveScore,
indexSearcher,
keyFieldName,
resultsConsumer
);
}
}
}