Implement adaptive multi searcher for shard indexes

This commit is contained in:
Andrea Cavalli 2021-07-06 00:30:14 +02:00
parent 545c7111de
commit b5ccf315be
45 changed files with 837 additions and 1865 deletions

View File

@ -43,16 +43,16 @@ public interface LuceneIndex<T, U> extends LLSnapshottable {
Mono<Void> deleteAll();
<V> Mono<SearchResultKeys<T>> moreLikeThis(ClientQueryParams<SearchResultKey<T>, V> queryParams, T key, U mltDocumentValue);
<V> Mono<SearchResultKeys<T>> moreLikeThis(ClientQueryParams<SearchResultKey<T>> queryParams, T key, U mltDocumentValue);
<V> Mono<SearchResult<T, U>> moreLikeThisWithValues(ClientQueryParams<SearchResultItem<T, U>, V> queryParams,
<V> Mono<SearchResult<T, U>> moreLikeThisWithValues(ClientQueryParams<SearchResultItem<T, U>> queryParams,
T key,
U mltDocumentValue,
ValueGetter<T, U> valueGetter);
<V> Mono<SearchResultKeys<T>> search(ClientQueryParams<SearchResultKey<T>, V> queryParams);
<V> Mono<SearchResultKeys<T>> search(ClientQueryParams<SearchResultKey<T>> queryParams);
<V> Mono<SearchResult<T, U>> searchWithValues(ClientQueryParams<SearchResultItem<T, U>, V> queryParams,
<V> Mono<SearchResult<T, U>> searchWithValues(ClientQueryParams<SearchResultItem<T, U>> queryParams,
ValueGetter<T, U> valueGetter);
Mono<Long> count(@Nullable CompositeSnapshot snapshot, Query query);

View File

@ -6,6 +6,7 @@ import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLScoreMode;
import it.cavallium.dbengine.database.LLSearchResult;
import it.cavallium.dbengine.database.LLSearchResultShard;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLTerm;
import it.cavallium.dbengine.database.collections.Joiner.ValueGetter;
@ -83,64 +84,19 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
return luceneIndex.deleteAll();
}
private static QueryParams fixOffset(LLLuceneIndex luceneIndex, QueryParams queryParams) {
if (luceneIndex.supportsOffset()) {
return queryParams;
} else {
return queryParams.setOffset(0).setLimit(queryParams.limit() + queryParams.offset());
}
private Mono<SearchResultKeys<T>> transformLuceneResult(LLSearchResultShard llSearchResult) {
return Mono.just(new SearchResultKeys<>(llSearchResult.results()
.map(signal -> new SearchResultKey<>(signal.key().map(indicizer::getKey), signal.score())),
llSearchResult.totalHitsCount()
));
}
private static long fixTransformOffset(LLLuceneIndex luceneIndex, long offset) {
if (luceneIndex.supportsOffset()) {
return 0;
} else {
return offset;
}
}
private Mono<SearchResultKeys<T>> transformLuceneResult(LLSearchResult llSearchResult,
@Nullable MultiSort<SearchResultKey<T>, ?> sort,
LLScoreMode scoreMode,
long offset,
@Nullable Long limit) {
Flux<SearchResultKeys<T>> mappedKeys = llSearchResult
.results()
.map(flux -> new SearchResultKeys<>(flux
.results()
.map(signal -> new SearchResultKey<>(signal.key().map(indicizer::getKey), signal.score())),
flux.totalHitsCount()
));
MultiSort<SearchResultKey<T>, ?> finalSort;
if (scoreMode != LLScoreMode.COMPLETE_NO_SCORES && sort == null) {
finalSort = MultiSort.topScore();
} else {
finalSort = sort;
}
return LuceneUtils.mergeSignalStreamKeys(mappedKeys, finalSort, offset, limit);
}
private <V> Mono<SearchResult<T, U>> transformLuceneResultWithValues(LLSearchResult llSearchResult,
@Nullable MultiSort<SearchResultItem<T, U>, V> sort,
LLScoreMode scoreMode,
long offset,
@Nullable Long limit,
private <V> Mono<SearchResult<T, U>> transformLuceneResultWithValues(LLSearchResultShard llSearchResult,
ValueGetter<T, U> valueGetter) {
Flux<SearchResult<T, U>> mappedKeys = llSearchResult
.results()
.map(flux -> new SearchResult<>(flux
.results()
.map(signal -> {
var key = signal.key().map(indicizer::getKey);
return new SearchResultItem<>(key, key.flatMap(valueGetter::get), signal.score());
}), flux.totalHitsCount()));
MultiSort<SearchResultItem<T, U>, ?> finalSort;
if (scoreMode != LLScoreMode.COMPLETE_NO_SCORES && sort == null) {
finalSort = MultiSort.topScoreWithValues();
} else {
finalSort = sort;
}
return LuceneUtils.mergeSignalStreamItems(mappedKeys, finalSort, offset, limit);
return Mono.just(new SearchResult<>(llSearchResult.results().map(signal -> {
var key = signal.key().map(indicizer::getKey);
return new SearchResultItem<>(key, key.flatMap(valueGetter::get), signal.score());
}), llSearchResult.totalHitsCount()));
}
/**
@ -151,19 +107,14 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
* @return the collection has one or more flux
*/
@Override
public <V> Mono<SearchResultKeys<T>> moreLikeThis(ClientQueryParams<SearchResultKey<T>, V> queryParams,
public <V> Mono<SearchResultKeys<T>> moreLikeThis(ClientQueryParams<SearchResultKey<T>> queryParams,
T key,
U mltDocumentValue) {
Flux<Tuple2<String, Set<String>>> mltDocumentFields
= indicizer.getMoreLikeThisDocumentFields(key, mltDocumentValue);
return luceneIndex
.moreLikeThis(resolveSnapshot(queryParams.snapshot()), fixOffset(luceneIndex, queryParams.toQueryParams()), indicizer.getKeyFieldName(), mltDocumentFields)
.flatMap(llSearchResult -> this.transformLuceneResult(llSearchResult,
queryParams.sort(),
queryParams.scoreMode(),
fixTransformOffset(luceneIndex, queryParams.offset()),
queryParams.limit()
));
.moreLikeThis(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName(), mltDocumentFields)
.flatMap(this::transformLuceneResult);
}
@ -176,7 +127,7 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
* @return the collection has one or more flux
*/
@Override
public <V> Mono<SearchResult<T, U>> moreLikeThisWithValues(ClientQueryParams<SearchResultItem<T, U>, V> queryParams,
public <V> Mono<SearchResult<T, U>> moreLikeThisWithValues(ClientQueryParams<SearchResultItem<T, U>> queryParams,
T key,
U mltDocumentValue,
ValueGetter<T, U> valueGetter) {
@ -184,15 +135,11 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
= indicizer.getMoreLikeThisDocumentFields(key, mltDocumentValue);
return luceneIndex
.moreLikeThis(resolveSnapshot(queryParams.snapshot()),
fixOffset(luceneIndex, queryParams.toQueryParams()),
queryParams.toQueryParams(),
indicizer.getKeyFieldName(),
mltDocumentFields
)
.flatMap(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult,
queryParams.sort(),
queryParams.scoreMode(),
fixTransformOffset(luceneIndex, queryParams.offset()),
queryParams.limit(),
valueGetter
));
}
@ -205,18 +152,13 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
* @return the collection has one or more flux
*/
@Override
public <V> Mono<SearchResultKeys<T>> search(ClientQueryParams<SearchResultKey<T>, V> queryParams) {
public <V> Mono<SearchResultKeys<T>> search(ClientQueryParams<SearchResultKey<T>> queryParams) {
return luceneIndex
.search(resolveSnapshot(queryParams.snapshot()),
fixOffset(luceneIndex, queryParams.toQueryParams()),
queryParams.toQueryParams(),
indicizer.getKeyFieldName()
)
.flatMap(llSearchResult -> this.transformLuceneResult(llSearchResult,
queryParams.sort(),
queryParams.scoreMode(),
fixTransformOffset(luceneIndex, queryParams.offset()),
queryParams.limit()
));
.flatMap(this::transformLuceneResult);
}
/**
@ -227,22 +169,16 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
* @return the collection has one or more flux
*/
@Override
public <V> Mono<SearchResult<T, U>> searchWithValues(ClientQueryParams<SearchResultItem<T, U>, V> queryParams,
public <V> Mono<SearchResult<T, U>> searchWithValues(ClientQueryParams<SearchResultItem<T, U>> queryParams,
ValueGetter<T, U> valueGetter) {
return luceneIndex
.search(resolveSnapshot(queryParams.snapshot()), fixOffset(luceneIndex, queryParams.toQueryParams()), indicizer.getKeyFieldName())
.flatMap(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult,
queryParams.sort(),
queryParams.scoreMode(),
fixTransformOffset(luceneIndex, queryParams.offset()),
queryParams.limit(),
valueGetter
));
.search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName())
.flatMap(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, valueGetter));
}
@Override
public Mono<Long> count(@Nullable CompositeSnapshot snapshot, Query query) {
return this.search(ClientQueryParams.<SearchResultKey<T>, Object>builder().snapshot(snapshot).query(query).limit(0).build())
return this.search(ClientQueryParams.<SearchResultKey<T>>builder().snapshot(snapshot).query(query).limit(0).build())
.map(SearchResultKeys::totalHitsCount);
}

View File

@ -12,98 +12,61 @@ import java.util.function.ToLongFunction;
import org.jetbrains.annotations.NotNull;
import reactor.core.publisher.Mono;
public class MultiSort<T, U> {
public class MultiSort<T> {
private final Sort querySort;
@NotNull
private final Function<T, Mono<U>> transformer;
private final Comparator<U> resultSort;
public MultiSort(Sort querySort, Function<T, Mono<U>> transformer, Comparator<U> resultSort) {
public MultiSort(Sort querySort) {
this.querySort = querySort;
this.transformer = transformer;
this.resultSort = resultSort;
}
/**
* Sort a lucene field and the results by a numeric sort field and an int value
* @param fieldName Lucene SortedNumericSortField field name
* @param transformer Transform a value to a comparable value asynchronously
* @param toIntFunction function to retrieve the integer value of each result
* @param reverse descending sort
* @param <T> result type
* @return MultiSort object
*/
public static <T, U> MultiSort<T, U> sortedNumericInt(String fieldName,
Function<T, Mono<U>> transformer,
ToIntFunction<U> toIntFunction,
boolean reverse) {
public static <T> MultiSort<T> sortedNumericInt(String fieldName, boolean reverse) {
// Create lucene sort
Sort querySort = NumericSort.of(fieldName, reverse);
// Create result sort
Comparator<U> resultSort = Comparator.comparingInt(toIntFunction);
if (reverse) {
resultSort = resultSort.reversed();
}
// Return the multi sort
return new MultiSort<>(querySort, transformer, resultSort);
return new MultiSort<>(querySort);
}
/**
* Sort a lucene field and the results by a numeric sort field and an long value
* @param fieldName Lucene SortedNumericSortField field name
* @param transformer Transform a value to a comparable value asynchronously
* @param toLongFunction function to retrieve the long value of each result
* @param reverse descending sort
* @param <T> result type
* @return MultiSort object
*/
public static <T, U> MultiSort<T, U> sortedNumericLong(String fieldName,
Function<T, Mono<U>> transformer,
ToLongFunction<U> toLongFunction,
boolean reverse) {
public static <T> MultiSort<T> sortedNumericLong(String fieldName, boolean reverse) {
// Create lucene sort
Sort querySort = NumericSort.of(fieldName, reverse);
// Create result sort
Comparator<U> resultSort = Comparator.comparingLong(toLongFunction);
if (!reverse) {
resultSort = resultSort.reversed();
}
// Return the multi sort
return new MultiSort<>(querySort, transformer, resultSort);
return new MultiSort<>(querySort);
}
public static <T> MultiSort<T, T> randomSortField() {
return new MultiSort<>(RandomSort.of(), Mono::just, (a, b) -> 0);
public static <T> MultiSort<T> randomSortField() {
return new MultiSort<>(RandomSort.of());
}
public static MultiSort<LLKeyScore, LLKeyScore> topScoreRaw() {
Comparator<LLKeyScore> comp = Comparator.comparingDouble(LLKeyScore::score).reversed();
return new MultiSort<>(ScoreSort.of(), Mono::just, comp);
public static MultiSort<LLKeyScore> topScoreRaw() {
return new MultiSort<>(ScoreSort.of());
}
public static <T> MultiSort<SearchResultKey<T>, SearchResultKey<T>> topScore() {
return new MultiSort<>(ScoreSort.of(), Mono::just, Comparator.<SearchResultKey<T>>comparingDouble(SearchResultKey::score).reversed());
public static <T> MultiSort<SearchResultKey<T>> topScore() {
return new MultiSort<>(ScoreSort.of());
}
public static <T, U> MultiSort<SearchResultItem<T, U>, SearchResultItem<T, U>> topScoreWithValues() {
return new MultiSort<>(ScoreSort.of(), Mono::just, Comparator.<SearchResultItem<T, U>>comparingDouble(SearchResultItem::score).reversed());
public static <T, U> MultiSort<SearchResultItem<T, U>> topScoreWithValues() {
return new MultiSort<>(ScoreSort.of());
}
public Sort getQuerySort() {
return querySort;
}
@NotNull
public Function<T, Mono<U>> getTransformer() {
return transformer;
}
public Comparator<U> getResultSort() {
return resultSort;
}
}

View File

@ -14,17 +14,17 @@ import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@RecordBuilder
public final record ClientQueryParams<T, U>(@Nullable CompositeSnapshot snapshot,
public final record ClientQueryParams<T>(@Nullable CompositeSnapshot snapshot,
@NotNull Query query,
long offset,
long limit,
@Nullable Float minCompetitiveScore,
@Nullable MultiSort<T, U> sort,
@Nullable MultiSort<T> sort,
@NotNull LLScoreMode scoreMode) {
public static <T, U> ClientQueryParamsBuilder<T, U> builder() {
public static <T> ClientQueryParamsBuilder<T> builder() {
return ClientQueryParamsBuilder
.<T, U>builder()
.<T>builder()
.snapshot(null)
.offset(0)
.limit(Long.MAX_VALUE)

View File

@ -39,7 +39,7 @@ public interface LLLuceneIndex extends LLSnapshottable {
* The additional query will be used with the moreLikeThis query: "mltQuery AND additionalQuery"
* @return the collection has one or more flux
*/
Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
Mono<LLSearchResultShard> moreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFields);
@ -49,20 +49,17 @@ public interface LLLuceneIndex extends LLSnapshottable {
* returned can be at most <code>limit * 15</code>
* @return the collection has one or more flux
*/
Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName);
Mono<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName);
default Mono<Long> count(@Nullable LLSnapshot snapshot, Query query) {
QueryParams params = QueryParams.of(query, 0, 0, Nullablefloat.empty(), NoSort.of(), ScoreMode.of(false, false));
return Mono.from(this.search(snapshot, params, null)
.flatMap(results -> LuceneUtils.mergeSignalStreamRaw(results.results(), null, 0, null))
.map(LLSearchResultShard::totalHitsCount)
.defaultIfEmpty(0L));
}
boolean isLowMemoryMode();
boolean supportsOffset();
Mono<Void> close();
/**

View File

@ -1,13 +0,0 @@
package it.cavallium.dbengine.database;
import java.io.IOException;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.IndexSearcher;
public interface LLSearchCollectionStatisticsGetter {
CollectionStatistics collectionStatistics(IndexSearcher indexSearcher,
String field,
boolean distributedPre,
long actionId) throws IOException;
}

View File

@ -1,33 +0,0 @@
package it.cavallium.dbengine.database.disk;
import it.cavallium.dbengine.database.LLSearchCollectionStatisticsGetter;
import java.io.IOException;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.IndexSearcher;
public class LLIndexSearcherWithCustomCollectionStatistics extends IndexSearcher {
private final IndexSearcher indexSearcher;
private final LLSearchCollectionStatisticsGetter customCollectionStatisticsGetter;
private final boolean distributedPre;
private final long actionId;
public LLIndexSearcherWithCustomCollectionStatistics(IndexSearcher indexSearcher,
LLSearchCollectionStatisticsGetter customCollectionStatisticsGetter,
boolean distributedPre,
long actionId) {
super(indexSearcher.getIndexReader());
this.indexSearcher = indexSearcher;
this.setSimilarity(indexSearcher.getSimilarity());
this.setQueryCache(indexSearcher.getQueryCache());
this.setQueryCachingPolicy(indexSearcher.getQueryCachingPolicy());
this.customCollectionStatisticsGetter = customCollectionStatisticsGetter;
this.distributedPre = distributedPre;
this.actionId = actionId;
}
@Override
public CollectionStatistics collectionStatistics(String field) throws IOException {
return customCollectionStatisticsGetter.collectionStatistics(indexSearcher, field, distributedPre, actionId);
}
}

View File

@ -85,8 +85,7 @@ public class LLLocalDatabaseConnection implements LLDatabaseConnection {
name,
indicizerAnalyzers,
indicizerSimilarities,
luceneOptions,
null
luceneOptions
);
}
})

View File

@ -11,7 +11,6 @@ import it.cavallium.dbengine.database.EnglishItalianStopFilter;
import it.cavallium.dbengine.database.LLDocument;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLSearchCollectionStatisticsGetter;
import it.cavallium.dbengine.database.LLSearchResult;
import it.cavallium.dbengine.database.LLSearchResultShard;
import it.cavallium.dbengine.database.LLSnapshot;
@ -20,11 +19,11 @@ import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.lucene.AlwaysDirectIOFSDirectory;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.lucene.ScheduledTaskLifecycle;
import it.cavallium.dbengine.lucene.searcher.AdaptiveReactiveSearcher;
import it.cavallium.dbengine.lucene.searcher.AdaptiveStreamSearcher;
import it.cavallium.dbengine.lucene.searcher.AllowOnlyQueryParsingCollectorStreamSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneReactiveSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher;
import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneLocalSearcher;
import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneLocalSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneShardSearcher;
import java.io.IOException;
import java.nio.file.Path;
import java.time.Duration;
@ -37,6 +36,7 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexWriter;
@ -76,10 +76,7 @@ import reactor.util.function.Tuples;
public class LLLocalLuceneIndex implements LLLuceneIndex {
protected static final Logger logger = LoggerFactory.getLogger(LLLocalLuceneIndex.class);
private static final LuceneStreamSearcher streamSearcher = new AdaptiveStreamSearcher();
private static final LuceneReactiveSearcher reactiveSearcher = new AdaptiveReactiveSearcher();
private static final AllowOnlyQueryParsingCollectorStreamSearcher allowOnlyQueryParsingCollectorStreamSearcher
= new AllowOnlyQueryParsingCollectorStreamSearcher();
private static final LuceneLocalSearcher localSearcher = new AdaptiveLuceneLocalSearcher();
/**
* Global lucene index scheduler.
* There is only a single thread globally to not overwhelm the disk with
@ -117,14 +114,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
private final Similarity similarity;
private final ScheduledTaskLifecycle scheduledTasksLifecycle;
private final @Nullable LLSearchCollectionStatisticsGetter distributedCollectionStatisticsGetter;
public LLLocalLuceneIndex(Path luceneBasePath,
String name,
IndicizerAnalyzers indicizerAnalyzers,
IndicizerSimilarities indicizerSimilarities,
LuceneOptions luceneOptions,
@Nullable LLSearchCollectionStatisticsGetter distributedCollectionStatisticsGetter) throws IOException {
LuceneOptions luceneOptions) throws IOException {
if (name.length() == 0) {
throw new IOException("Empty lucene database name");
}
@ -195,7 +190,6 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
this.snapshotter = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
this.lowMemory = lowMemory;
this.similarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities);
this.distributedCollectionStatisticsGetter = distributedCollectionStatisticsGetter;
IndexWriterConfig indexWriterConfig = new IndexWriterConfig(LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers));
indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
@ -438,7 +432,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
}).subscribeOn(luceneHeavyTasksScheduler);
}
private Mono<IndexSearcher> acquireSearcherWrapper(LLSnapshot snapshot, boolean distributedPre, long actionId) {
private Mono<IndexSearcher> acquireSearcherWrapper(LLSnapshot snapshot) {
return Mono.fromCallable(() -> {
IndexSearcher indexSearcher;
if (snapshot == null) {
@ -447,15 +441,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
} else {
indexSearcher = resolveSnapshot(snapshot).getIndexSearcher();
}
if (distributedCollectionStatisticsGetter != null && actionId != -1) {
return new LLIndexSearcherWithCustomCollectionStatistics(indexSearcher,
distributedCollectionStatisticsGetter,
distributedPre,
actionId
);
} else {
return indexSearcher;
}
return indexSearcher;
}).subscribeOn(Schedulers.boundedElastic());
}
@ -472,240 +458,35 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
}
@Override
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
public Mono<LLSearchResultShard> moreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux) {
return moreLikeThis(snapshot,
queryParams,
keyFieldName,
mltDocumentFieldsFlux,
false,
0,
1
);
}
public Mono<LLSearchResult> distributedMoreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
long actionId,
int scoreDivisor) {
return moreLikeThis(snapshot,
queryParams,
keyFieldName,
mltDocumentFieldsFlux,
false,
actionId,
scoreDivisor
);
}
public Mono<Void> distributedPreMoreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
long actionId) {
return moreLikeThis(snapshot, queryParams, keyFieldName, mltDocumentFieldsFlux, true, actionId, 1).then();
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
boolean doDistributedPre,
long actionId,
int scoreDivisor) {
Query luceneAdditionalQuery;
try {
luceneAdditionalQuery = QueryParser.toQuery(queryParams.query());
} catch (Exception e) {
return Mono.error(e);
}
return mltDocumentFieldsFlux
.collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new)
.flatMap(mltDocumentFields -> {
mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty());
if (mltDocumentFields.isEmpty()) {
return Mono.just(new LLSearchResult(Flux.empty()));
}
return acquireSearcherWrapper(snapshot, doDistributedPre, actionId)
.flatMap(indexSearcher -> Mono
.fromCallable(() -> {
var mlt = new MoreLikeThis(indexSearcher.getIndexReader());
mlt.setAnalyzer(indexWriter.getAnalyzer());
mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new));
mlt.setMinTermFreq(1);
mlt.setMinDocFreq(3);
mlt.setMaxDocFreqPct(20);
mlt.setBoost(QueryParser.isScoringEnabled(queryParams));
mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString());
var similarity = getSimilarity();
if (similarity instanceof TFIDFSimilarity) {
mlt.setSimilarity((TFIDFSimilarity) similarity);
} else {
logger.trace("Using an unsupported similarity algorithm for MoreLikeThis:"
+ " {}. You must use a similarity instance based on TFIDFSimilarity!", similarity);
}
// Get the reference docId and apply it to MoreLikeThis, to generate the query
var mltQuery = mlt.like((Map) mltDocumentFields);
Query luceneQuery;
if (luceneAdditionalQuery != null) {
luceneQuery = new BooleanQuery.Builder()
.add(mltQuery, Occur.MUST)
.add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST)
.build();
} else {
luceneQuery = mltQuery;
}
return luceneQuery;
})
.subscribeOn(Schedulers.boundedElastic())
.map(luceneQuery -> luceneSearch(doDistributedPre,
indexSearcher,
queryParams.offset(),
queryParams.limit(),
queryParams.minCompetitiveScore().getNullable(),
keyFieldName,
scoreDivisor,
luceneQuery,
QueryParser.toSort(queryParams.sort()),
QueryParser.toScoreMode(queryParams.scoreMode()),
releaseSearcherWrapper(snapshot, indexSearcher)
))
.onErrorResume(ex -> releaseSearcherWrapper(snapshot, indexSearcher).then(Mono.error(ex)))
);
});
}
private LLKeyScore fixKeyScore(LLKeyScore keyScore, int scoreDivisor) {
return scoreDivisor == 1 ? keyScore
: new LLKeyScore(keyScore.docId(), keyScore.score() / (float) scoreDivisor, keyScore.key());
throw new NotImplementedException();
}
@Override
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) {
return search(snapshot, queryParams, keyFieldName, false, 0, 1);
public Mono<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) {
return Mono
.usingWhen(
this.acquireSearcherWrapper(snapshot),
indexSearcher -> localSearcher.collect(indexSearcher, queryParams, keyFieldName, luceneSearcherScheduler),
indexSearcher -> releaseSearcherWrapper(snapshot, indexSearcher)
)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount()));
}
public Mono<LLSearchResult> distributedSearch(@Nullable LLSnapshot snapshot,
public Mono<Void> distributedSearch(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
long actionId,
int scoreDivisor) {
return search(snapshot, queryParams, keyFieldName, false, actionId, scoreDivisor);
}
public Mono<Void> distributedPreSearch(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
long actionId) {
return this
.search(snapshot, queryParams, keyFieldName, true, actionId, 1)
.then();
}
@SuppressWarnings("RedundantTypeArguments")
private Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
QueryParams queryParams, String keyFieldName,
boolean doDistributedPre, long actionId, int scoreDivisor) {
return this
.acquireSearcherWrapper(snapshot, doDistributedPre, actionId)
.flatMap(indexSearcher -> Mono
.fromCallable(() -> {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
Sort luceneSort = QueryParser.toSort(queryParams.sort());
org.apache.lucene.search.ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.scoreMode());
return Tuples.of(luceneQuery, Optional.ofNullable(luceneSort), luceneScoreMode);
})
.subscribeOn(Schedulers.boundedElastic())
.<LLSearchResult>flatMap(tuple -> Mono
.fromSupplier(() -> {
Query luceneQuery = tuple.getT1();
Sort luceneSort = tuple.getT2().orElse(null);
ScoreMode luceneScoreMode = tuple.getT3();
return luceneSearch(doDistributedPre,
indexSearcher,
queryParams.offset(),
queryParams.limit(),
queryParams.minCompetitiveScore().getNullable(),
keyFieldName,
scoreDivisor,
luceneQuery,
luceneSort,
luceneScoreMode,
releaseSearcherWrapper(snapshot, indexSearcher)
);
})
.onErrorResume(ex -> releaseSearcherWrapper(snapshot, indexSearcher).then(Mono.error(ex)))
)
LuceneShardSearcher shardSearcher) {
return Mono
.usingWhen(
this.acquireSearcherWrapper(snapshot),
indexSearcher -> shardSearcher.searchOn(indexSearcher, queryParams, luceneSearcherScheduler),
indexSearcher -> releaseSearcherWrapper(snapshot, indexSearcher)
);
}
/**
* This method always returns 1 shard! Not zero, not more than one.
*/
private LLSearchResult luceneSearch(boolean doDistributedPre,
IndexSearcher indexSearcher,
long offset,
long limit,
@Nullable Float minCompetitiveScore,
String keyFieldName,
int scoreDivisor,
Query luceneQuery,
Sort luceneSort,
ScoreMode luceneScoreMode,
Mono<Void> successCleanup) {
Flux<LLSearchResultShard> results = Mono
.defer(() -> {
if (doDistributedPre) {
return Mono.<LLSearchResultShard>create(monoSink -> {
try {
//noinspection BlockingMethodInNonBlockingContext
allowOnlyQueryParsingCollectorStreamSearcher.search(indexSearcher, luceneQuery);
monoSink.success(new LLSearchResultShard(successCleanup.thenMany(Flux.empty()), 0));
} catch (Exception ex) {
monoSink.error(ex);
}
}).subscribeOn(luceneSearcherScheduler);
} else {
int boundedOffset = Math.max(0, offset > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) offset);
int boundedLimit = Math.max(0, limit > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limit);
return reactiveSearcher
.search(indexSearcher,
luceneQuery,
boundedOffset,
boundedLimit,
luceneSort,
luceneScoreMode,
minCompetitiveScore,
keyFieldName,
luceneSearcherScheduler
)
.map(searchInstance -> new LLSearchResultShard(
Flux
.usingWhen(
Mono.just(true),
_unused -> searchInstance
.results()
.map(keyScore -> fixKeyScore(keyScore, scoreDivisor)),
_unused -> successCleanup
),
searchInstance.totalHitsCount()
));
}
})
.flux();
return new LLSearchResult(results);
}
@Override
public Mono<Void> close() {
return Mono
@ -789,9 +570,4 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
public boolean isLowMemoryMode() {
return lowMemory;
}
@Override
public boolean supportsOffset() {
return true;
}
}

View File

@ -5,6 +5,7 @@ import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader.InvalidCacheLoadException;
import it.cavallium.dbengine.client.IndicizerAnalyzers;
import it.cavallium.dbengine.client.IndicizerSimilarities;
import it.cavallium.dbengine.client.LuceneIndex;
import it.cavallium.dbengine.client.LuceneOptions;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLDocument;
@ -15,6 +16,9 @@ import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLTerm;
import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity;
import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneShardSearcher;
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap;
@ -37,6 +41,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.IndexSearcher;
import org.jetbrains.annotations.Nullable;
@ -56,10 +61,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
private final LLLocalLuceneIndex[] luceneIndices;
private final AtomicLong nextActionId = new AtomicLong(0);
private final ConcurrentHashMap<Long, Cache<String, Optional<CollectionStatistics>>[]> statistics = new ConcurrentHashMap<>();
private final ConcurrentHashMap<Long, AtomicInteger> completedStreams = new ConcurrentHashMap<>();
private final LuceneMultiSearcher multiSearcher = new AdaptiveLuceneMultiSearcher();
public LLLocalMultiLuceneIndex(Path lucene,
String name,
@ -74,7 +76,6 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
LLLocalLuceneIndex[] luceneIndices = new LLLocalLuceneIndex[instancesCount];
for (int i = 0; i < instancesCount; i++) {
int finalI = i;
String instanceName;
if (i == 0) {
instanceName = name;
@ -85,76 +86,12 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
instanceName,
indicizerAnalyzers,
indicizerSimilarities,
luceneOptions, (indexSearcher, field, distributedPre, actionId) -> distributedCustomCollectionStatistics(finalI,
indexSearcher,
field,
distributedPre,
actionId
)
luceneOptions
);
}
this.luceneIndices = luceneIndices;
}
private long newAction() {
var statistics = new Cache[luceneIndices.length];
for (int i = 0; i < luceneIndices.length; i++) {
statistics[i] = CacheBuilder.newBuilder().build();
}
long actionId = nextActionId.getAndIncrement();
//noinspection unchecked
this.statistics.put(actionId, statistics);
this.completedStreams.put(actionId, new AtomicInteger(0));
return actionId;
}
private void completedAction(long actionId) {
var completedStreamsCount = completedStreams.get(actionId);
if (completedStreamsCount != null) {
if (completedStreamsCount.incrementAndGet() >= luceneIndices.length) {
this.statistics.remove(actionId);
this.completedStreams.remove(actionId);
}
}
}
private CollectionStatistics distributedCustomCollectionStatistics(int luceneIndex,
IndexSearcher indexSearcher, String field, boolean distributedPre, long actionId) throws IOException {
if (distributedPre) {
try {
var optional = statistics.get(actionId)[luceneIndex].get(field,
() -> Optional.ofNullable(indexSearcher.collectionStatistics(field))
);
return optional.orElse(null);
} catch ( InvalidCacheLoadException | ExecutionException e) {
throw new IOException(e);
}
} else {
long maxDoc = 0;
long docCount = 0;
long sumTotalTermFreq = 0;
long sumDocFreq = 0;
for (int i = 0; i < luceneIndices.length; i++) {
CollectionStatistics iCollStats = Objects
.requireNonNull(statistics.get(actionId)[i].getIfPresent(field))
.orElse(null);
if (iCollStats != null) {
maxDoc += iCollStats.maxDoc();
docCount += iCollStats.docCount();
sumTotalTermFreq += iCollStats.sumTotalTermFreq();
sumDocFreq += iCollStats.sumDocFreq();
}
}
return new CollectionStatistics(field,
(int) Math.max(1, Math.min(maxDoc, Integer.MAX_VALUE)),
Math.max(1, docCount),
Math.max(1, sumTotalTermFreq),
Math.max(1, sumDocFreq)
);
}
}
private LLLocalLuceneIndex getLuceneIndex(LLTerm id) {
return luceneIndices[getLuceneIndexId(id)];
}
@ -258,155 +195,39 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
}
@Override
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
public Mono<LLSearchResultShard> moreLikeThis(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFields) {
if (queryParams.offset() != 0) {
return Mono.error(new IllegalArgumentException("MultiLuceneIndex requires an offset equal to 0"));
}
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsShared;
Mono<DistributedSearch> distributedPre;
if (luceneIndices.length > 1) {
long actionId = newAction();
mltDocumentFieldsShared = mltDocumentFields.publish().refCount();
distributedPre = Flux
.fromArray(luceneIndices)
.index()
.flatMap(tuple -> Mono
.fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1()))
.map(luceneSnapshot -> Tuples.of(tuple.getT2(), luceneSnapshot))
)
.flatMap(tuple -> tuple
.getT1()
.distributedPreMoreLikeThis(tuple.getT2().orElse(null),
queryParams,
keyFieldName,
mltDocumentFieldsShared,
actionId
)
)
.then(Mono.just(new DistributedSearch(actionId, 20)));
} else {
mltDocumentFieldsShared = mltDocumentFields;
distributedPre = Mono.just(new DistributedSearch(-1, 1));
}
//noinspection DuplicatedCode
return distributedPre
.flatMap(distributedSearch -> Flux
.fromArray(luceneIndices)
.index()
.flatMap(tuple -> Mono
.fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1()))
.map(luceneSnapshot -> Tuples.of(tuple.getT2(), luceneSnapshot)))
.flatMap(tuple -> tuple
.getT1()
.distributedMoreLikeThis(tuple.getT2().orElse(null),
queryParams,
keyFieldName,
mltDocumentFieldsShared,
distributedSearch.actionId(),
distributedSearch.scoreDivisor()
)
)
.reduce(LLSearchResult.accumulator())
.map(result -> {
if (distributedSearch.actionId() != -1) {
Flux<LLSearchResultShard> resultsWithTermination = result
.results()
.map(flux -> new LLSearchResultShard(Flux
.using(
distributedSearch::actionId,
actionId -> flux.results(),
this::completedAction
), flux.totalHitsCount())
);
return new LLSearchResult(resultsWithTermination);
} else {
return result;
}
})
.doOnError(ex -> {
if (distributedSearch.actionId() != -1) {
completedAction(distributedSearch.actionId());
}
})
);
throw new NotImplementedException();
}
private static record DistributedSearch(long actionId, int scoreDivisor) {}
@Override
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
public Mono<LLSearchResultShard> search(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
String keyFieldName) {
if (queryParams.offset() != 0) {
return Mono.error(new IllegalArgumentException("MultiLuceneIndex requires an offset equal to 0"));
}
Mono<DistributedSearch> distributedSearchMono;
if (luceneIndices.length <= 1 || !queryParams.scoreMode().computeScores()) {
distributedSearchMono = Mono.just(new DistributedSearch(-1, 1));
} else {
var actionId = newAction();
distributedSearchMono = Flux
.fromArray(luceneIndices)
.index()
.flatMap(tuple -> Mono
.fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1()))
.map(luceneSnapshot -> Tuples.of(tuple.getT2(), luceneSnapshot))
)
.flatMap(tuple -> tuple
.getT1()
.distributedPreSearch(tuple.getT2().orElse(null),
queryParams,
keyFieldName,
actionId
)
)
.then(Mono.just(new DistributedSearch(actionId, 20)));
}
record LuceneIndexWithSnapshot(LLLocalLuceneIndex luceneIndex, Optional<LLSnapshot> snapshot) {}
return distributedSearchMono
.flatMap(distributedSearch -> Flux
.fromArray(luceneIndices)
.index()
return multiSearcher
// Create shard searcher
.createShardSearcher(queryParams)
.flatMap(shardSearcher -> Flux
// Iterate the indexed shards
.fromArray(luceneIndices).index()
// Resolve the snapshot of each shard
.flatMap(tuple -> Mono
.fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1()))
.map(luceneSnapshot -> Tuples.of(tuple.getT2(), luceneSnapshot))
.map(luceneSnapshot -> new LuceneIndexWithSnapshot(tuple.getT2(), luceneSnapshot))
)
.flatMap(tuple -> tuple
.getT1()
.distributedSearch(tuple.getT2().orElse(null),
queryParams,
keyFieldName,
distributedSearch.actionId(),
distributedSearch.scoreDivisor()
))
.reduce(LLSearchResult.accumulator())
.map(result -> {
if (distributedSearch.actionId() != -1) {
Flux<LLSearchResultShard> resultsWithTermination = result
.results()
.map(flux -> new LLSearchResultShard(Flux
.using(
distributedSearch::actionId,
actionId -> flux.results(),
this::completedAction
), flux.totalHitsCount())
);
return new LLSearchResult(resultsWithTermination);
} else {
return result;
}
})
.doOnError(ex -> {
if (distributedSearch.actionId() != -1) {
completedAction(distributedSearch.actionId());
}
})
);
// Execute the query and collect it using the shard searcher
.flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex()
.distributedSearch(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, shardSearcher))
// Collect all the shards results into a single global result
.then(shardSearcher.collect(queryParams, keyFieldName, Schedulers.boundedElastic()))
)
// Fix the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount()));
}
@Override
@ -466,9 +287,4 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
public boolean isLowMemoryMode() {
return luceneIndices[0].isLowMemoryMode();
}
@Override
public boolean supportsOffset() {
return false;
}
}

View File

@ -1,76 +0,0 @@
package it.cavallium.dbengine.lucene;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.HandleResult;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.LeafReaderContext;
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;
private Scorable scorer;
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;
}
@Override
public final LeafCollector getLeafCollector(LeafReaderContext context) {
return new LuceneParallelStreamCollector(context.docBase,
scoreMode,
minCompetitiveScore,
streamConsumer,
stopped,
totalHitsCounter
);
}
@Override
public void setScorer(Scorable scorer) throws IOException {
this.scorer = scorer;
if (minCompetitiveScore != null && !minCompetitiveScore.isNaN() && !minCompetitiveScore.isInfinite()) {
scorer.setMinCompetitiveScore(minCompetitiveScore);
}
}
@Override
public void collect(int doc) throws IOException {
doc += base;
totalHitsCounter.incrementAndGet();
if (!stopped.get()) {
var score = scorer == null ? 0 : scorer.score();
if (minCompetitiveScore == null || score >= minCompetitiveScore) {
if (streamConsumer.consume(doc, score) == HandleResult.HALT) {
stopped.set(true);
}
}
}
}
@Override
public ScoreMode scoreMode() {
return scoreMode;
}
}

View File

@ -1,55 +0,0 @@
package it.cavallium.dbengine.lucene;
import java.util.Collection;
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, 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();
}
@Override
public LuceneParallelStreamCollector newCollector() {
return new LuceneParallelStreamCollector(0,
scoreMode,
minCompetitiveScore,
streamConsumer,
stopped,
totalHitsCounter
);
}
@Override
public LuceneParallelStreamCollectorResult reduce(
Collection<LuceneParallelStreamCollector> collectors) {
return new LuceneParallelStreamCollectorResult(totalHitsCounter.get());
}
}

View File

@ -1,15 +0,0 @@
package it.cavallium.dbengine.lucene;
public class LuceneParallelStreamCollectorResult {
private final long totalHitsCount;
public LuceneParallelStreamCollectorResult(long totalHitsCount) {
this.totalHitsCount = totalHitsCount;
}
public long getTotalHitsCount() {
return totalHitsCount;
}
}

View File

@ -1,13 +0,0 @@
package it.cavallium.dbengine.lucene;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.HandleResult;
import java.io.IOException;
public interface LuceneParallelStreamConsumer {
/**
* @param docId document id
* @param score score of document
*/
HandleResult consume(int docId, float score) throws IOException;
}

View File

@ -21,13 +21,12 @@ 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.searcher.LuceneStreamSearcher.HandleResult;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.ResultItemConsumer;
import it.cavallium.dbengine.lucene.similarity.NGramSimilarity;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
@ -42,8 +41,11 @@ import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.similarities.BooleanSimilarity;
import org.apache.lucene.search.similarities.ClassicSimilarity;
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
@ -160,80 +162,6 @@ public class LuceneUtils {
return tokenStream;
}
/**
* Merge streams together maintaining absolute order
*/
@SuppressWarnings({"unchecked"})
public static <T, U> Flux<T> mergeStream(Flux<Flux<T>> mappedMultiResults,
@Nullable MultiSort<T, U> sort,
long offset,
@Nullable Long limit) {
if (limit != null && limit == 0) {
return mappedMultiResults.flatMap(f -> f).ignoreElements().flux();
} else {
return mappedMultiResults.collectList().flatMapMany(mappedMultiResultsList -> {
Flux<T> mergedFlux;
if (sort == null) {
mergedFlux = Flux.merge(mappedMultiResultsList);
} else {
mergedFlux = Flux
.mergeOrdered(32,
(a, b) -> sort.getResultSort().compare(a.getT2(), b.getT2()),
(Flux<Tuple2<T, U>>[]) mappedMultiResultsList.stream()
.map(flux -> flux.flatMapSequential(entry -> sort.getTransformer().apply(entry)
.map(transformed -> Tuples.of(entry, transformed))))
.toArray(Flux[]::new)
)
.map(Tuple2::getT1);
}
Flux<T> offsetedFlux;
if (offset > 0) {
offsetedFlux = mergedFlux.skip(offset);
} else {
offsetedFlux = mergedFlux;
}
if (limit == null || limit == Long.MAX_VALUE) {
return offsetedFlux;
} else {
return offsetedFlux.take(limit, true);
}
});
}
}
public static HandleResult collectTopDoc(Logger logger,
int docId,
float score,
Float minCompetitiveScore,
IndexSearcher indexSearcher,
String keyFieldName,
ResultItemConsumer 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 {
if (resultsConsumer.accept(new LLKeyScore(docId, score, Mono.just(field.stringValue())))
== HandleResult.HALT) {
return HandleResult.HALT;
}
}
}
}
return HandleResult.CONTINUE;
}
/**
*
* @return false if the result is not relevant
@ -244,13 +172,17 @@ public class LuceneUtils {
}
@Nullable
public static String keyOfTopDoc(Logger logger, int docId, IndexSearcher indexSearcher,
public static String keyOfTopDoc(Logger logger, int docId, IndexReader indexReader,
String keyFieldName) throws IOException {
Document d = indexSearcher.doc(docId, Set.of(keyFieldName));
if (docId > indexReader.maxDoc()) {
logger.warn("Document " + docId + " > maxDoc (" +indexReader.maxDoc() + ")");
return null;
}
Document d = indexReader.document(docId, Set.of(keyFieldName));
if (d.getFields().isEmpty()) {
StringBuilder sb = new StringBuilder();
sb.append("The document docId: ").append(docId).append(" is empty.");
var realFields = indexSearcher.doc(docId).getFields();
var realFields = indexReader.document(docId).getFields();
if (!realFields.isEmpty()) {
sb.append("\n");
logger.error("Present fields:\n");
@ -275,43 +207,6 @@ public class LuceneUtils {
}
}
public static <T, V> Mono<SearchResultKeys<T>> mergeSignalStreamKeys(Flux<SearchResultKeys<T>> mappedKeys,
MultiSort<SearchResultKey<T>, V> sort,
long offset,
Long limit) {
return mappedKeys.reduce(
new SearchResultKeys<>(Flux.empty(), 0L),
(a, b) -> new SearchResultKeys<>(LuceneUtils.mergeStream(Flux.just(a.results(), b.results()),
sort, offset, limit
), a.totalHitsCount() + b.totalHitsCount())
);
}
public static <T, U, V> Mono<SearchResult<T, U>> mergeSignalStreamItems(Flux<SearchResult<T, U>> mappedKeys,
MultiSort<SearchResultItem<T, U>, V> sort,
long offset,
Long limit) {
return mappedKeys.reduce(
new SearchResult<>(Flux.empty(), 0L),
(a, b) -> new SearchResult<>(LuceneUtils.mergeStream(Flux.just(a.results(), b.results()),
sort, offset, limit
), a.totalHitsCount() + b.totalHitsCount())
);
}
public static Mono<LLSearchResultShard> mergeSignalStreamRaw(Flux<LLSearchResultShard> mappedKeys,
MultiSort<LLKeyScore, LLKeyScore> mappedSort,
long offset,
Long limit) {
return mappedKeys.reduce(
new LLSearchResultShard(Flux.empty(), 0),
(s1, s2) -> new LLSearchResultShard(
LuceneUtils.mergeStream(Flux.just(s1.results(), s2.results()), mappedSort, offset, limit),
s1.totalHitsCount() + s2.totalHitsCount()
)
);
}
public static <T, U, V> ValueGetter<Entry<T, U>, V> getAsyncDbValueGetterDeep(
CompositeSnapshot snapshot,
DatabaseMapDictionaryDeep<T, Map<U, V>, DatabaseMapDictionary<U, V>> dictionaryDeep) {
@ -406,4 +301,36 @@ public class LuceneUtils {
assert readLength == 0;
}
public static int safeLongToInt(long l) {
if (l > 2147483630) {
return 2147483630;
} else if (l < -2147483630) {
return -2147483630;
} else {
return (int) l;
}
}
@Nullable
public static FieldDoc getLastFieldDoc(ScoreDoc[] scoreDocs) {
if (scoreDocs == null) {
return null;
}
if (scoreDocs.length == 0) {
return null;
}
return (FieldDoc) scoreDocs[scoreDocs.length - 1];
}
@Nullable
public static ScoreDoc getLastScoreDoc(ScoreDoc[] scoreDocs) {
if (scoreDocs == null) {
return null;
}
if (scoreDocs.length == 0) {
return null;
}
return scoreDocs[scoreDocs.length - 1];
}
}

View File

@ -0,0 +1,19 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import org.apache.lucene.search.IndexSearcher;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher {
private static final LuceneLocalSearcher localSearcher = new SimpleLuceneLocalSearcher();
@Override
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
QueryParams queryParams,
String keyFieldName,
Scheduler scheduler) {
return localSearcher.collect(indexSearcher, queryParams, keyFieldName, scheduler);
}
}

View File

@ -0,0 +1,27 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import org.apache.lucene.search.Sort;
import reactor.core.publisher.Mono;
public class AdaptiveLuceneMultiSearcher implements LuceneMultiSearcher {
private static final LuceneMultiSearcher sharedSortedLuceneMultiSearcher = new SharedSortedLuceneMultiSearcher();
private static final LuceneMultiSearcher unscoredLuceneMultiSearcher = new UnscoredLuceneMultiSearcher();
private static final LuceneMultiSearcher countLuceneMultiSearcher = new CountLuceneMultiSearcher();
@Override
public Mono<LuceneShardSearcher> createShardSearcher(QueryParams queryParams) {
Sort luceneSort = QueryParser.toSort(queryParams.sort());
if (queryParams.limit() <= 0) {
return countLuceneMultiSearcher.createShardSearcher(queryParams);
} else if ((luceneSort != null && luceneSort != Sort.RELEVANCE) || queryParams.scoreMode().computeScores()) {
return sharedSortedLuceneMultiSearcher.createShardSearcher(queryParams);
} else {
return unscoredLuceneMultiSearcher.createShardSearcher(queryParams);
}
}
}

View File

@ -1,65 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class AdaptiveReactiveSearcher implements LuceneReactiveSearcher {
public static final int PAGED_THRESHOLD = 1000;
private static final LuceneReactiveSearcher count = new CountLuceneReactiveSearcher();
private static final LuceneReactiveSearcher paged = new PagedLuceneReactiveSearcher();
private static final LuceneReactiveSearcher simple = new SimpleLuceneReactiveSearcher();
@Override
public Mono<LuceneReactiveSearchInstance> search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Scheduler scheduler) {
if (limit == 0) {
return count.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
scheduler
);
}
if (offset + limit > PAGED_THRESHOLD) {
return paged.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
scheduler
);
} else {
return simple.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName,
scheduler
);
}
}
}

View File

@ -1,73 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import java.io.IOException;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
/**
* Use a different searcher based on the situation
*/
public class AdaptiveStreamSearcher implements LuceneStreamSearcher {
private static final boolean ENABLE_PARALLEL_COLLECTOR = true;
private final SimpleStreamSearcher simpleStreamSearcher;
private final ParallelCollectorStreamSearcher parallelCollectorStreamSearcher;
private final PagedStreamSearcher pagedStreamSearcher;
private final CountStreamSearcher countStreamSearcher;
public AdaptiveStreamSearcher() {
this.simpleStreamSearcher = new SimpleStreamSearcher();
this.countStreamSearcher = new CountStreamSearcher();
this.parallelCollectorStreamSearcher = new ParallelCollectorStreamSearcher(countStreamSearcher);
this.pagedStreamSearcher = new PagedStreamSearcher(simpleStreamSearcher);
}
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
if (limit == 0) {
return countStreamSearcher.count(indexSearcher, query);
} else if (offset == 0 && luceneSort == null && ENABLE_PARALLEL_COLLECTOR) {
return parallelCollectorStreamSearcher.search(indexSearcher,
query,
offset,
limit,
null,
scoreMode,
minCompetitiveScore,
keyFieldName
);
} else {
if (offset > 0 || limit > PagedStreamSearcher.MAX_ITEMS_PER_PAGE) {
return pagedStreamSearcher.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName
);
} else {
return simpleStreamSearcher.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName
);
}
}
}
}

View File

@ -1,93 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
/**
* Search that only parse query without doing any effective search
*/
public class AllowOnlyQueryParsingCollectorStreamSearcher implements LuceneStreamSearcher {
public void search(IndexSearcher indexSearcher,
Query query) throws IOException {
search(indexSearcher, query, 0, 0, null, null, null, null);
}
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
if (limit > 0) {
throw new IllegalArgumentException("Limit > 0 not allowed");
}
if (luceneSort != null) {
throw new IllegalArgumentException("Lucene sort not allowed");
}
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");
}
indexSearcher.search(query, new CollectorManager<>() {
@Override
public Collector newCollector() {
return new Collector() {
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) {
return new LeafCollector() {
@Override
public void setScorer(Scorable scorer) throws IOException {
scorer.setMinCompetitiveScore(Float.MAX_VALUE);
}
@Override
public void collect(int doc) {
}
};
}
@Override
public ScoreMode scoreMode() {
return ScoreMode.TOP_SCORES;
}
};
}
@Override
public Object reduce(Collection<Collector> collectors) {
return null;
}
});
return new LuceneSearchInstance() {
@Override
public long getTotalHitsCount() {
throw new IllegalArgumentException("Total hits consumer not allowed");
}
@Override
public void getResults(ResultItemConsumer consumer) {
throw new IllegalArgumentException("Results consumer not allowed");
}
};
}
}

View File

@ -0,0 +1,39 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class CountLuceneMultiSearcher implements LuceneMultiSearcher {
@Override
public Mono<LuceneShardSearcher> createShardSearcher(QueryParams queryParams) {
return Mono
.fromCallable(() -> {
AtomicLong totalHits = new AtomicLong(0);
return new LuceneShardSearcher() {
@Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, QueryParams queryParams, Scheduler scheduler) {
return Mono
.<Void>fromCallable(() -> {
Query luceneQuery = QueryParser.toQuery(queryParams.query());
//noinspection BlockingMethodInNonBlockingContext
totalHits.addAndGet(indexSearcher.count(luceneQuery));
return null;
})
.subscribeOn(scheduler);
}
@Override
public Mono<LuceneSearchResult> collect(QueryParams queryParams, String keyFieldName, Scheduler scheduler) {
return Mono.fromCallable(() -> new LuceneSearchResult(totalHits.get(), Flux.empty()));
}
};
});
}
}

View File

@ -1,37 +0,0 @@
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.ArrayList;
import java.util.List;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class CountLuceneReactiveSearcher implements LuceneReactiveSearcher {
@SuppressWarnings("BlockingMethodInNonBlockingContext")
@Override
public Mono<LuceneReactiveSearchInstance> search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Scheduler scheduler) {
return Mono
.fromCallable(() -> new LuceneReactiveSearchInstance(indexSearcher.count(query), Flux.empty()))
.subscribeOn(scheduler);
}
}

View File

@ -1,54 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import java.io.IOException;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
/**
* Search that only count approximate results without returning any result
*/
public class CountStreamSearcher implements LuceneStreamSearcher {
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
if (limit != 0) {
throw new IllegalArgumentException("CountStream doesn't support a limit different than 0");
}
if (luceneSort != null) {
throw new IllegalArgumentException("CountStream doesn't support sorting");
}
if (keyFieldName != null) {
throw new IllegalArgumentException("CountStream doesn't support a key field");
}
return count(indexSearcher, query);
}
public long countLong(IndexSearcher indexSearcher, Query query) throws IOException {
return indexSearcher.count(query);
}
public LuceneSearchInstance count(IndexSearcher indexSearcher, Query query) throws IOException {
long totalHitsCount = countLong(indexSearcher, query);
return new LuceneSearchInstance() {
@Override
public long getTotalHitsCount() {
return totalHitsCount;
}
@Override
public void getResults(ResultItemConsumer consumer) {
}
};
}
}

View File

@ -0,0 +1,24 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.util.Comparator;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.ScoreDoc;
import org.jetbrains.annotations.Nullable;
record CurrentPageInfo(@Nullable ScoreDoc last, long remainingLimit, int pageIndex) {
private static final int MAX_ITEMS_PER_PAGE = 100;
public static final Comparator<ScoreDoc> TIE_BREAKER = Comparator.comparingInt((d) -> d.shardIndex);
public static final CurrentPageInfo EMPTY_STATUS = new CurrentPageInfo(null, 0, 0);
int currentPageLimit() {
if (pageIndex >= 10) { // safety
return MAX_ITEMS_PER_PAGE;
}
var min = Math.min(MAX_ITEMS_PER_PAGE, LuceneUtils.safeLongToInt(pageIndex * (0b1L << pageIndex)));
assert min > 0;
return min;
}
}

View File

@ -0,0 +1,152 @@
package it.cavallium.dbengine.lucene.searcher;
import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS;
import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.TIE_BREAKER;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopFieldDocs;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
class FieldSimpleLuceneShardSearcher implements LuceneShardSearcher {
private final Object lock = new Object();
private final List<IndexSearcher> indexSearchersArray = new ArrayList<>();
private final List<TopFieldCollector> collectors = new ArrayList<>();
private final CollectorManager<TopFieldCollector, TopFieldDocs> sharedManager;
private final Query luceneQuery;
private final PaginationInfo paginationInfo;
public FieldSimpleLuceneShardSearcher(CollectorManager<TopFieldCollector, TopFieldDocs> sharedManager,
Query luceneQuery, PaginationInfo paginationInfo) {
this.sharedManager = sharedManager;
this.luceneQuery = luceneQuery;
this.paginationInfo = paginationInfo;
}
@Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, QueryParams queryParams, Scheduler scheduler) {
return Mono.<Void>fromCallable(() -> {
TopFieldCollector collector;
synchronized (lock) {
//noinspection BlockingMethodInNonBlockingContext
collector = sharedManager.newCollector();
indexSearchersArray.add(indexSearcher);
collectors.add(collector);
}
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(luceneQuery, collector);
return null;
}).subscribeOn(scheduler);
}
@Override
public Mono<LuceneSearchResult> collect(QueryParams queryParams, String keyFieldName, Scheduler scheduler) {
return Mono
.fromCallable(() -> {
TopDocs[] topDocs;
synchronized (lock) {
topDocs = new TopDocs[collectors.size()];
var i = 0;
for (TopFieldCollector collector : collectors) {
topDocs[i] = collector.topDocs();
for (ScoreDoc scoreDoc : topDocs[i].scoreDocs) {
scoreDoc.shardIndex = i;
}
i++;
}
}
var result = TopDocs.merge(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()),
LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()),
topDocs,
TIE_BREAKER
);
IndexSearchers indexSearchers;
synchronized (lock) {
indexSearchers = IndexSearchers.of(indexSearchersArray);
}
Flux<LLKeyScore> firstPageHits = LuceneMultiSearcher
.convertHits(result.scoreDocs, indexSearchers, keyFieldName, scheduler);
Flux<LLKeyScore> nextHits = Flux.defer(() -> {
if (paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) {
return Flux.empty();
}
return Flux
.<TopDocs, CurrentPageInfo>generate(
() -> new CurrentPageInfo(LuceneUtils.getLastFieldDoc(result.scoreDocs), paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1),
(s, sink) -> {
if (s.last() != null && s.remainingLimit() > 0) {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
Sort luceneSort = QueryParser.toSort(queryParams.sort());
ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.scoreMode());
if (luceneSort == null && luceneScoreMode.needsScores()) {
luceneSort = Sort.RELEVANCE;
}
CollectorManager<TopFieldCollector, TopFieldDocs> sharedManager = TopFieldCollector
.createSharedManager(luceneSort, s.currentPageLimit(), (FieldDoc) s.last(), 1000);
//noinspection BlockingMethodInNonBlockingContext
TopDocs pageTopDocs = Flux
.fromIterable(indexSearchersArray)
.index()
.flatMapSequential(tuple -> Mono
.fromCallable(() -> {
long shardIndex = tuple.getT1();
IndexSearcher indexSearcher = tuple.getT2();
//noinspection BlockingMethodInNonBlockingContext
var results = indexSearcher.search(luceneQuery, sharedManager);
for (ScoreDoc scoreDoc : results.scoreDocs) {
scoreDoc.shardIndex = LuceneUtils.safeLongToInt(shardIndex);
}
return results;
})
.subscribeOn(scheduler)
)
.collect(Collectors.toCollection(ObjectArrayList::new))
.map(topFieldDocs -> topFieldDocs.toArray(TopFieldDocs[]::new))
.map(topFieldDocs -> TopDocs.merge(0, s.currentPageLimit(),
topFieldDocs,
TIE_BREAKER
))
.blockOptional().orElseThrow();
var pageLastDoc = LuceneUtils.getLastFieldDoc(pageTopDocs.scoreDocs);
sink.next(pageTopDocs);
return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), s.pageIndex() + 1);
} else {
sink.complete();
return EMPTY_STATUS;
}
},
s -> {}
)
.subscribeOn(scheduler)
.concatMap(topFieldDoc -> LuceneMultiSearcher
.convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, scheduler)
);
});
return new LuceneSearchResult(result.totalHits.value, firstPageHits.concatWith(nextHits));
})
.subscribeOn(scheduler);
}
}

View File

@ -0,0 +1,27 @@
package it.cavallium.dbengine.lucene.searcher;
import java.util.List;
import org.apache.lucene.search.IndexSearcher;
public interface IndexSearchers {
static IndexSearchers of(List<IndexSearcher> indexSearchers) {
return shardIndex -> {
if (shardIndex < 0) {
throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid");
}
return indexSearchers.get(shardIndex);
};
}
static IndexSearchers unsharded(IndexSearcher indexSearcher) {
return shardIndex -> {
if (shardIndex != -1) {
throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid, this is a unsharded index");
}
return indexSearcher;
};
}
IndexSearcher shard(int shardIndex);
}

View File

@ -0,0 +1,20 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import org.apache.lucene.search.IndexSearcher;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public interface LuceneLocalSearcher {
/**
* @param indexSearcher Lucene index searcher
* @param queryParams the query parameters
* @param keyFieldName the name of the key field
* @param scheduler a blocking scheduler
*/
Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
QueryParams queryParams,
String keyFieldName,
Scheduler scheduler);
}

View File

@ -1,11 +1,10 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -18,49 +17,31 @@ import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public interface LuceneReactiveSearcher {
public interface LuceneMultiSearcher {
Logger logger = LoggerFactory.getLogger(LuceneReactiveSearcher.class);
Logger logger = LoggerFactory.getLogger(LuceneMultiSearcher.class);
/**
* Do a lucene query, receiving the single results using a consumer
* @param indexSearcher the index searcher, which contains all the lucene data
* @param query the query
* @param offset the offset of the first result (use 0 to disable offset)
* @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 scheduler a blocking scheduler
* @param queryParams the query parameters
*/
Mono<LuceneReactiveSearchInstance> search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Scheduler scheduler);
Mono<LuceneShardSearcher> createShardSearcher(QueryParams queryParams);
static Flux<LLKeyScore> convertHits(
ScoreDoc[] hits,
IndexSearcher indexSearcher,
@Nullable Float minCompetitiveScore,
IndexSearchers indexSearchers,
String keyFieldName,
Scheduler scheduler) {
return Flux
.fromArray(hits)
.map(hit -> {
int shardDocId = hit.doc;
int shardIndex = hit.shardIndex;
float score = hit.score;
var indexSearcher = indexSearchers.shard(shardIndex);
var keyMono = Mono.fromCallable(() -> {
if (!LuceneUtils.filterTopDoc(score, minCompetitiveScore)) {
return null;
}
//noinspection BlockingMethodInNonBlockingContext
@Nullable String collectedDoc = LuceneUtils.keyOfTopDoc(logger, shardDocId, indexSearcher, keyFieldName);
@Nullable String collectedDoc = LuceneUtils.keyOfTopDoc(logger, shardDocId, indexSearcher.getIndexReader(), keyFieldName);
return collectedDoc;
}).subscribeOn(scheduler);
return new LLKeyScore(shardDocId, score, keyMono);

View File

@ -1,9 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.ResultItemConsumer;
import java.io.IOException;
import reactor.core.publisher.Flux;
public record LuceneReactiveSearchInstance(long totalHitsCount, Flux<LLKeyScore> results) {
}

View File

@ -1,11 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.ResultItemConsumer;
import java.io.IOException;
public interface LuceneSearchInstance {
long getTotalHitsCount() throws IOException;
void getResults(ResultItemConsumer consumer) throws IOException;
}

View File

@ -0,0 +1,8 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import java.io.IOException;
import reactor.core.publisher.Flux;
public record LuceneSearchResult(long totalHitsCount, Flux<LLKeyScore> results) {
}

View File

@ -0,0 +1,25 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import org.apache.lucene.search.IndexSearcher;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public interface LuceneShardSearcher {
/**
* @param indexSearcher the index searcher, which contains all the lucene data
* @param queryParams the query parameters
* @param scheduler a blocking scheduler
*/
Mono<Void> searchOn(IndexSearcher indexSearcher,
QueryParams queryParams,
Scheduler scheduler);
/**
* @param queryParams the query parameters
* @param keyFieldName the name of the key field
* @param scheduler a blocking scheduler
*/
Mono<LuceneSearchResult> collect(QueryParams queryParams, String keyFieldName, Scheduler scheduler);
}

View File

@ -1,47 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import java.io.IOException;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
import org.warp.commonutils.log.Logger;
import org.warp.commonutils.log.LoggerFactory;
public interface LuceneStreamSearcher {
Logger logger = LoggerFactory.getLogger(LuceneStreamSearcher.class);
/**
* Do a lucene query, receiving the single results using a consumer
* @param indexSearcher the index searcher, which contains all the lucene data
* @param query the query
* @param offset the offset of the first result (use 0 to disable offset)
* @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
* @throws IOException thrown if there is an error
*/
LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException;
@FunctionalInterface
interface ResultItemConsumer {
HandleResult accept(LLKeyScore item);
}
enum HandleResult {
CONTINUE, HALT
}
}

View File

@ -1,140 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.HandleResult;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldDocs;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class PagedLuceneReactiveSearcher implements LuceneReactiveSearcher {
private static final int FIRST_PAGE_HITS_MAX_COUNT = 1;
private static final long MIN_HITS_PER_PAGE = 5;
private static final long MAX_HITS_PER_PAGE = 100;
@SuppressWarnings("BlockingMethodInNonBlockingContext")
@Override
public Mono<LuceneReactiveSearchInstance> search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Scheduler scheduler) {
return Mono
.fromCallable(() -> {
// Run the first page search
TopDocs firstTopDocsVal;
if (offset == 0) {
firstTopDocsVal = TopDocsSearcher.getTopDocs(indexSearcher,
query,
luceneSort,
FIRST_PAGE_HITS_MAX_COUNT,
null,
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000
);
} else {
firstTopDocsVal = TopDocsSearcher.getTopDocs(indexSearcher,
query,
luceneSort,
offset + FIRST_PAGE_HITS_MAX_COUNT,
null,
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000,
offset, FIRST_PAGE_HITS_MAX_COUNT);
}
long totalHitsCount = firstTopDocsVal.totalHits.value;
Flux<LLKeyScore> firstPageHitsFlux = LuceneReactiveSearcher.convertHits(
firstTopDocsVal.scoreDocs,
indexSearcher,
minCompetitiveScore,
keyFieldName,
scheduler
);
Flux<LLKeyScore> nextPagesFlux = Flux
.<Flux<LLKeyScore>, PageState>generate(
() -> new PageState(getLastItem(firstTopDocsVal.scoreDocs), 0),
(s, sink) -> {
if (s.lastItem() == null) {
sink.complete();
return new PageState(null, 0);
}
try {
var lastTopDocs = TopDocsSearcher.getTopDocs(indexSearcher,
query,
luceneSort,
s.hitsPerPage(),
s.lastItem(),
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000
);
if (lastTopDocs.scoreDocs.length > 0) {
ScoreDoc lastItem = getLastItem(lastTopDocs.scoreDocs);
var hitsList = LuceneReactiveSearcher.convertHits(
lastTopDocs.scoreDocs,
indexSearcher,
minCompetitiveScore,
keyFieldName,
scheduler
);
sink.next(hitsList);
return new PageState(lastItem, s.currentPageIndex() + 1);
} else {
sink.complete();
return new PageState(null, 0);
}
} catch (IOException e) {
sink.error(e);
return new PageState(null, 0);
}
}
)
.subscribeOn(scheduler)
.concatMap(Flux::hide);
Flux<LLKeyScore> resultsFlux = firstPageHitsFlux
.concatWith(nextPagesFlux)
.take(limit, true);
if (limit == 0) {
return new LuceneReactiveSearchInstance(totalHitsCount, Flux.empty());
} else {
return new LuceneReactiveSearchInstance(totalHitsCount, resultsFlux);
}
})
.subscribeOn(scheduler);
}
private static ScoreDoc getLastItem(ScoreDoc[] scoreDocs) {
if (scoreDocs.length == 0) {
return null;
}
return scoreDocs[scoreDocs.length - 1];
}
private record PageState(ScoreDoc lastItem, int currentPageIndex) {
public int hitsPerPage() {
return (int) Math.min(MAX_HITS_PER_PAGE, MIN_HITS_PER_PAGE * (1L << currentPageIndex));
}
}
}

View File

@ -1,154 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.jetbrains.annotations.Nullable;
import org.warp.commonutils.type.IntWrapper;
/**
* Sorted paged search (the most memory-efficient stream searcher for big queries)
*/
public class PagedStreamSearcher implements LuceneStreamSearcher {
public static final int MAX_ITEMS_PER_PAGE = 1000;
private final SimpleStreamSearcher simpleStreamSearcher;
public PagedStreamSearcher(SimpleStreamSearcher simpleStreamSearcher) {
this.simpleStreamSearcher = simpleStreamSearcher;
}
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
if (limit < MAX_ITEMS_PER_PAGE) {
// Use a normal search method because the limit is low
simpleStreamSearcher.search(indexSearcher,
query,
offset,
limit,
luceneSort,
scoreMode,
minCompetitiveScore,
keyFieldName
);
}
IntWrapper currentAllowedResults = new IntWrapper(limit);
// Run the first page search
TopDocs firstTopDocsVal;
if (offset == 0) {
firstTopDocsVal = indexSearcher.search(query,
MAX_ITEMS_PER_PAGE,
luceneSort,
scoreMode != ScoreMode.COMPLETE_NO_SCORES
);
} else {
firstTopDocsVal = new TopDocsSearcher(indexSearcher,
query,
luceneSort,
MAX_ITEMS_PER_PAGE,
null,
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000
).getTopDocs(offset, MAX_ITEMS_PER_PAGE);
}
AtomicReference<TopDocs> firstTopDocs = new AtomicReference<>(firstTopDocsVal);
long totalHitsCount = firstTopDocs.getPlain().totalHits.value;
return new LuceneSearchInstance() {
@Override
public long getTotalHitsCount() {
return totalHitsCount;
}
@Override
public void getResults(ResultItemConsumer resultsConsumer) throws IOException {
TopDocs lastTopDocs = firstTopDocs.getAndSet(null);
if (lastTopDocs.scoreDocs.length > 0) {
ScoreDoc lastScoreDoc = getLastItem(lastTopDocs.scoreDocs);
consumeHits(currentAllowedResults,
lastTopDocs.scoreDocs,
indexSearcher,
minCompetitiveScore,
keyFieldName,
resultsConsumer
);
// Run the searches for each page until the end
boolean finished = currentAllowedResults.var <= 0;
while (!finished) {
boolean halted;
lastTopDocs = indexSearcher.searchAfter(lastScoreDoc,
query,
MAX_ITEMS_PER_PAGE,
luceneSort,
scoreMode != ScoreMode.COMPLETE_NO_SCORES
);
if (lastTopDocs.scoreDocs.length > 0) {
lastScoreDoc = getLastItem(lastTopDocs.scoreDocs);
halted = consumeHits(currentAllowedResults,
lastTopDocs.scoreDocs,
indexSearcher,
minCompetitiveScore,
keyFieldName,
resultsConsumer
) == HandleResult.HALT;
} else {
halted = false;
}
if (lastTopDocs.scoreDocs.length < MAX_ITEMS_PER_PAGE || currentAllowedResults.var <= 0 || halted) {
finished = true;
}
}
}
}
};
}
private HandleResult consumeHits(IntWrapper currentAllowedResults,
ScoreDoc[] hits,
IndexSearcher indexSearcher,
@Nullable Float minCompetitiveScore,
String keyFieldName,
ResultItemConsumer resultsConsumer) throws IOException {
for (ScoreDoc hit : hits) {
int docId = hit.doc;
float score = hit.score;
if (currentAllowedResults.var-- > 0) {
if (LuceneUtils.collectTopDoc(logger,
docId,
score,
minCompetitiveScore,
indexSearcher,
keyFieldName,
resultsConsumer
) == HandleResult.HALT) {
return HandleResult.HALT;
}
} else {
break;
}
}
return HandleResult.CONTINUE;
}
private static ScoreDoc getLastItem(ScoreDoc[] scoreDocs) {
return scoreDocs[scoreDocs.length - 1];
}
}

View File

@ -0,0 +1,10 @@
package it.cavallium.dbengine.lucene.searcher;
import java.util.Comparator;
import org.apache.lucene.search.ScoreDoc;
public record PaginationInfo(long totalLimit, long firstPageOffset, long firstPageLimit, boolean forceSinglePage) {
public static final int MAX_SINGLE_SEARCH_LIMIT = 1000;
public static final int FIRST_PAGE_LIMIT = 10;
}

View File

@ -1,91 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneParallelStreamCollectorManager;
import it.cavallium.dbengine.lucene.LuceneParallelStreamCollectorResult;
import java.io.IOException;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
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.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Mono;
/**
* Unsorted search (low latency and constant memory usage)
*/
public class ParallelCollectorStreamSearcher implements LuceneStreamSearcher {
private final CountStreamSearcher countStreamSearcher;
public ParallelCollectorStreamSearcher(CountStreamSearcher countStreamSearcher) {
this.countStreamSearcher = countStreamSearcher;
}
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
if (offset != 0) {
throw new IllegalArgumentException("ParallelCollectorStreamSearcher doesn't support a offset different than 0");
}
if (luceneSort != null) {
throw new IllegalArgumentException("ParallelCollectorStreamSearcher doesn't support sorted searches");
}
return new LuceneSearchInstance() {
long totalHitsCount = countStreamSearcher.countLong(indexSearcher, query);
@Override
public long getTotalHitsCount() {
return totalHitsCount;
}
@Override
public void getResults(ResultItemConsumer resultsConsumer) throws IOException {
AtomicInteger currentCount = new AtomicInteger();
LuceneParallelStreamCollectorResult result = indexSearcher.search(query,
LuceneParallelStreamCollectorManager.fromConsumer(scoreMode, minCompetitiveScore, (docId, score) -> {
if (currentCount.getAndIncrement() >= limit) {
return HandleResult.HALT;
} else {
Document d = indexSearcher.doc(docId, Set.of(keyFieldName));
if (d.getFields().isEmpty()) {
logger.error("The document docId: {} is empty.", docId);
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: {}", docId);
} else {
if (resultsConsumer.accept(new LLKeyScore(docId, score, Mono.just(field.stringValue())))
== HandleResult.HALT) {
return HandleResult.HALT;
}
}
}
return HandleResult.CONTINUE;
}
}));
this.totalHitsCount = result.getTotalHitsCount();
}
};
}
}

View File

@ -0,0 +1,43 @@
package it.cavallium.dbengine.lucene.searcher;
import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT;
import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.util.Objects;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopFieldDocs;
import reactor.core.publisher.Mono;
public class SharedSortedLuceneMultiSearcher implements LuceneMultiSearcher {
@Override
public Mono<LuceneShardSearcher> createShardSearcher(QueryParams queryParams) {
return Mono
.fromCallable(() -> {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
Sort luceneSort = QueryParser.toSort(queryParams.sort());
ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.scoreMode());
if (luceneSort == null && luceneScoreMode.needsScores()) {
luceneSort = Sort.RELEVANCE;
}
PaginationInfo paginationInfo;
if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) {
paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true);
} else {
paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, true);
}
CollectorManager<TopFieldCollector, TopFieldDocs> sharedManager = TopFieldCollector
.createSharedManager(luceneSort, LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), null, 1000);
return new FieldSimpleLuceneShardSearcher(sharedManager, luceneQuery, paginationInfo);
});
}
}

View File

@ -0,0 +1,51 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.util.List;
import java.util.Objects;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
@Override
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
QueryParams queryParams,
String keyFieldName,
Scheduler scheduler) {
return Mono
.fromCallable(() -> {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
Sort luceneSort = QueryParser.toSort(queryParams.sort());
ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.scoreMode());
TopDocs topDocs = TopDocsSearcher.getTopDocs(indexSearcher,
luceneQuery,
luceneSort,
LuceneUtils.safeLongToInt(queryParams.offset() + queryParams.limit()),
null,
luceneScoreMode.needsScores(),
1000,
LuceneUtils.safeLongToInt(queryParams.offset()), LuceneUtils.safeLongToInt(queryParams.limit()));
Flux<LLKeyScore> hitsMono = LuceneMultiSearcher
.convertHits(
topDocs.scoreDocs,
IndexSearchers.unsharded(indexSearcher),
keyFieldName,
scheduler
)
.take(queryParams.limit(), true);
return new LuceneSearchResult(topDocs.totalHits.value, hitsMono);
})
.subscribeOn(scheduler);
}
}

View File

@ -1,56 +0,0 @@
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.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldDocs;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
public class SimpleLuceneReactiveSearcher implements LuceneReactiveSearcher {
@Override
public Mono<LuceneReactiveSearchInstance> search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName,
Scheduler scheduler) {
return Mono
.fromCallable(() -> {
TopDocs topDocs = TopDocsSearcher.getTopDocs(indexSearcher,
query,
luceneSort,
offset + limit,
null,
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000,
offset, limit);
Flux<LLKeyScore> hitsMono = LuceneReactiveSearcher
.convertHits(
topDocs.scoreDocs,
indexSearcher,
minCompetitiveScore,
keyFieldName,
scheduler
)
.take(limit, true);
return new LuceneReactiveSearchInstance(topDocs.totalHits.value, hitsMono);
})
.subscribeOn(scheduler);
}
}

View File

@ -1,69 +0,0 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
import java.io.IOException;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.jetbrains.annotations.Nullable;
/**
* Sorted search (slower and more memory-intensive)
*/
public class SimpleStreamSearcher implements LuceneStreamSearcher {
@Override
public LuceneSearchInstance search(IndexSearcher indexSearcher,
Query query,
int offset,
int limit,
@Nullable Sort luceneSort,
ScoreMode scoreMode,
@Nullable Float minCompetitiveScore,
String keyFieldName) throws IOException {
var searcher = new TopDocsSearcher(indexSearcher,
query,
luceneSort,
offset + limit,
null,
scoreMode != ScoreMode.COMPLETE_NO_SCORES,
1000
);
var topDocs = searcher.getTopDocs();
long totalHitsCount = topDocs.totalHits.value;
return new LuceneSearchInstance() {
@Override
public long getTotalHitsCount() {
return totalHitsCount;
}
@Override
public void getResults(ResultItemConsumer resultsConsumer) throws IOException {
if (topDocs.scoreDocs.length > 0) {
ObjectArrayList<ScoreDoc> hits = ObjectArrayList.wrap(topDocs.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
);
}
};
}
}

View File

@ -91,11 +91,11 @@ class TopDocsSearcher {
return topDocs;
}
public static TopDocsCollector<?> getTopDocsCollector(Sort luceneSort,
public static TopDocsCollector<? extends ScoreDoc> getTopDocsCollector(Sort luceneSort,
int limit,
ScoreDoc after,
int totalHitsThreshold) {
TopDocsCollector<?> collector;
TopDocsCollector<? extends ScoreDoc> collector;
if (luceneSort == null) {
if (after == null) {
collector = TopScoreDocCollector.create(limit, totalHitsThreshold);

View File

@ -0,0 +1,45 @@
package it.cavallium.dbengine.lucene.searcher;
import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT;
import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.util.Objects;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import reactor.core.publisher.Mono;
public class UnscoredLuceneMultiSearcher implements LuceneMultiSearcher {
@Override
public Mono<LuceneShardSearcher> createShardSearcher(QueryParams queryParams) {
return Mono
.fromCallable(() -> {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
Sort luceneSort = QueryParser.toSort(queryParams.sort());
ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.scoreMode());
if (luceneScoreMode.needsScores()) {
throw new UnsupportedOperationException("Can't use the unscored searcher to do a scored query");
}
if (luceneSort != null && luceneSort != Sort.RELEVANCE) {
throw new UnsupportedOperationException("Can't use the unscored searcher to do a sorted query");
}
PaginationInfo paginationInfo;
if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) {
paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true);
} else {
paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, true);
}
UnsortedCollectorManager unsortedCollectorManager = new UnsortedCollectorManager(() -> TopDocsSearcher.getTopDocsCollector(null,
LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()),
null,
1000
), queryParams.offset(), queryParams.limit());
return new UnscoredLuceneShardSearcher(unsortedCollectorManager, luceneQuery, paginationInfo);
});
}
}

View File

@ -0,0 +1,146 @@
package it.cavallium.dbengine.lucene.searcher;
import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS;
import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.TIE_BREAKER;
import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopDocsCollector;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
class UnscoredLuceneShardSearcher implements LuceneShardSearcher {
private final Object lock = new Object();
private final List<IndexSearcher> indexSearchersArray = new ArrayList<>();
private final List<TopDocsCollector<? extends ScoreDoc>> collectors = new ArrayList<>();
private final CollectorManager<TopDocsCollector<? extends ScoreDoc>, ? extends TopDocs> firstPageUnsortedCollectorManager;
private final Query luceneQuery;
private final PaginationInfo paginationInfo;
public UnscoredLuceneShardSearcher(CollectorManager<TopDocsCollector<? extends ScoreDoc>, ? extends TopDocs> unsortedCollectorManager,
Query luceneQuery,
PaginationInfo paginationInfo) {
this.firstPageUnsortedCollectorManager = unsortedCollectorManager;
this.luceneQuery = luceneQuery;
this.paginationInfo = paginationInfo;
}
@Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, QueryParams queryParams, Scheduler scheduler) {
return Mono.<Void>fromCallable(() -> {
TopDocsCollector<? extends ScoreDoc> collector;
synchronized (lock) {
//noinspection BlockingMethodInNonBlockingContext
collector = firstPageUnsortedCollectorManager.newCollector();
indexSearchersArray.add(indexSearcher);
collectors.add(collector);
}
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(luceneQuery, collector);
return null;
}).subscribeOn(scheduler);
}
@Override
public Mono<LuceneSearchResult> collect(QueryParams queryParams, String keyFieldName, Scheduler scheduler) {
return Mono
.fromCallable(() -> {
TopDocs[] topDocs;
synchronized (lock) {
topDocs = new TopDocs[collectors.size()];
var i = 0;
for (TopDocsCollector<?> collector : collectors) {
topDocs[i] = collector.topDocs();
for (ScoreDoc scoreDoc : topDocs[i].scoreDocs) {
scoreDoc.shardIndex = i;
}
i++;
}
}
var result = TopDocs.merge(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()),
LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()),
topDocs,
TIE_BREAKER
);
IndexSearchers indexSearchers;
synchronized (lock) {
indexSearchers = IndexSearchers.of(indexSearchersArray);
}
Flux<LLKeyScore> firstPageHits = LuceneMultiSearcher
.convertHits(result.scoreDocs, indexSearchers, keyFieldName, scheduler);
Flux<LLKeyScore> nextHits = Flux.defer(() -> {
if (paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) {
return Flux.empty();
}
return Flux
.<TopDocs, CurrentPageInfo>generate(
() -> new CurrentPageInfo(LuceneUtils.getLastScoreDoc(result.scoreDocs), paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1),
(s, sink) -> {
if (s.last() != null && s.remainingLimit() > 0 && s.currentPageLimit() > 0) {
Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null");
Query luceneQuery = QueryParser.toQuery(queryParams.query());
UnsortedCollectorManager currentPageUnsortedCollectorManager = new UnsortedCollectorManager(() -> TopDocsSearcher.getTopDocsCollector(null,
s.currentPageLimit(),
s.last(),
1000
), 0, s.currentPageLimit());
//noinspection BlockingMethodInNonBlockingContext
TopDocs pageTopDocs = Flux
.fromIterable(indexSearchersArray)
.index()
.flatMapSequential(tuple -> Mono
.fromCallable(() -> {
long shardIndex = tuple.getT1();
IndexSearcher indexSearcher = tuple.getT2();
//noinspection BlockingMethodInNonBlockingContext
var results = indexSearcher.search(luceneQuery, currentPageUnsortedCollectorManager);
for (ScoreDoc scoreDoc : results.scoreDocs) {
scoreDoc.shardIndex = LuceneUtils.safeLongToInt(shardIndex);
}
return results;
})
.subscribeOn(scheduler)
)
.collect(Collectors.toCollection(ObjectArrayList::new))
.map(topFieldDocs -> topFieldDocs.toArray(TopDocs[]::new))
.map(topFieldDocs -> TopDocs.merge(0, s.currentPageLimit(),
topFieldDocs,
TIE_BREAKER
))
.blockOptional().orElseThrow();
var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs);
sink.next(pageTopDocs);
return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), s.pageIndex() + 1);
} else {
sink.complete();
return EMPTY_STATUS;
}
},
s -> {}
)
.subscribeOn(scheduler)
.concatMap(topFieldDoc -> LuceneMultiSearcher
.convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, scheduler)
);
});
return new LuceneSearchResult(result.totalHits.value, firstPageHits.concatWith(nextHits));
})
.subscribeOn(scheduler);
}
}

View File

@ -0,0 +1,45 @@
package it.cavallium.dbengine.lucene.searcher;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.io.IOException;
import java.util.Collection;
import java.util.function.Supplier;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopDocsCollector;
public class UnsortedCollectorManager implements
CollectorManager<TopDocsCollector<? extends ScoreDoc>, TopDocs> {
private final Supplier<TopDocsCollector<? extends ScoreDoc>> collectorSupplier;
private final long offset;
private final long limit;
public UnsortedCollectorManager(Supplier<TopDocsCollector<? extends ScoreDoc>> collectorSupplier, long offset, long limit) {
this.collectorSupplier = collectorSupplier;
this.offset = offset;
this.limit = limit;
}
@Override
public TopDocsCollector<? extends ScoreDoc> newCollector() throws IOException {
return collectorSupplier.get();
}
@Override
public TopDocs reduce(Collection<TopDocsCollector<? extends ScoreDoc>> collection) throws IOException {
int i = 0;
TopDocs[] topDocsArray = new TopDocs[collection.size()];
for (TopDocsCollector<? extends ScoreDoc> topDocsCollector : collection) {
var topDocs = topDocsCollector.topDocs();
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
scoreDoc.shardIndex = i;
}
topDocsArray[i] = topDocs;
i++;
}
return TopDocs.merge(LuceneUtils.safeLongToInt(offset), LuceneUtils.safeLongToInt(limit), topDocsArray);
}
}