2020-12-07 22:15:18 +01:00
|
|
|
package it.cavallium.dbengine.database.disk;
|
|
|
|
|
2021-09-19 19:59:37 +02:00
|
|
|
import io.net5.buffer.api.Send;
|
2021-05-28 16:04:59 +02:00
|
|
|
import it.cavallium.dbengine.client.IndicizerAnalyzers;
|
|
|
|
import it.cavallium.dbengine.client.IndicizerSimilarities;
|
2021-07-01 21:19:52 +02:00
|
|
|
import it.cavallium.dbengine.client.LuceneOptions;
|
2021-03-02 01:53:36 +01:00
|
|
|
import it.cavallium.dbengine.client.query.current.data.QueryParams;
|
2021-01-24 03:15:05 +01:00
|
|
|
import it.cavallium.dbengine.database.LLDocument;
|
|
|
|
import it.cavallium.dbengine.database.LLLuceneIndex;
|
2021-03-27 03:35:27 +01:00
|
|
|
import it.cavallium.dbengine.database.LLSearchResultShard;
|
2021-01-24 03:15:05 +01:00
|
|
|
import it.cavallium.dbengine.database.LLSnapshot;
|
|
|
|
import it.cavallium.dbengine.database.LLTerm;
|
2021-07-06 01:30:37 +02:00
|
|
|
import it.cavallium.dbengine.lucene.LuceneUtils;
|
2021-07-06 00:30:14 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneMultiSearcher;
|
2021-09-19 19:59:37 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LLSearchTransformer;
|
2021-07-06 01:30:37 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LocalQueryParams;
|
2021-07-06 00:30:14 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.io.IOException;
|
|
|
|
import java.nio.file.Path;
|
|
|
|
import java.time.Duration;
|
2021-05-11 21:59:05 +02:00
|
|
|
import java.util.ArrayList;
|
2021-06-06 02:23:51 +02:00
|
|
|
import java.util.Collections;
|
2021-05-11 21:59:05 +02:00
|
|
|
import java.util.HashMap;
|
2021-05-28 16:04:59 +02:00
|
|
|
import java.util.List;
|
2021-05-11 21:59:05 +02:00
|
|
|
import java.util.Map;
|
2021-05-28 16:04:59 +02:00
|
|
|
import java.util.Map.Entry;
|
2021-02-03 13:48:30 +01:00
|
|
|
import java.util.Optional;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Set;
|
2021-02-04 22:42:57 +01:00
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
2021-09-20 11:52:21 +02:00
|
|
|
import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
|
|
|
|
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.jetbrains.annotations.Nullable;
|
|
|
|
import reactor.core.publisher.Flux;
|
|
|
|
import reactor.core.publisher.Mono;
|
2021-01-30 20:01:22 +01:00
|
|
|
import reactor.util.function.Tuple2;
|
2020-12-07 22:15:18 +01:00
|
|
|
|
|
|
|
public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
|
|
|
|
|
2021-06-25 20:07:19 +02:00
|
|
|
private final ConcurrentHashMap<Long, LLSnapshot[]> registeredSnapshots = new ConcurrentHashMap<>();
|
2020-12-07 22:15:18 +01:00
|
|
|
private final AtomicLong nextSnapshotNumber = new AtomicLong(1);
|
|
|
|
private final LLLocalLuceneIndex[] luceneIndices;
|
2021-09-20 11:52:21 +02:00
|
|
|
private final PerFieldAnalyzerWrapper luceneAnalyzer;
|
|
|
|
private final PerFieldSimilarityWrapper luceneSimilarity;
|
2021-06-25 20:07:19 +02:00
|
|
|
|
2021-07-06 00:30:14 +02:00
|
|
|
private final LuceneMultiSearcher multiSearcher = new AdaptiveLuceneMultiSearcher();
|
2021-06-25 20:07:19 +02:00
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
public LLLocalMultiLuceneIndex(Path lucene,
|
|
|
|
String name,
|
|
|
|
int instancesCount,
|
2021-05-28 16:04:59 +02:00
|
|
|
IndicizerAnalyzers indicizerAnalyzers,
|
|
|
|
IndicizerSimilarities indicizerSimilarities,
|
2021-07-01 21:19:52 +02:00
|
|
|
LuceneOptions luceneOptions) throws IOException {
|
2020-12-07 22:15:18 +01:00
|
|
|
|
|
|
|
if (instancesCount <= 1 || instancesCount > 100) {
|
|
|
|
throw new IOException("Unsupported instances count: " + instancesCount);
|
|
|
|
}
|
|
|
|
|
|
|
|
LLLocalLuceneIndex[] luceneIndices = new LLLocalLuceneIndex[instancesCount];
|
|
|
|
for (int i = 0; i < instancesCount; i++) {
|
|
|
|
String instanceName;
|
|
|
|
if (i == 0) {
|
|
|
|
instanceName = name;
|
|
|
|
} else {
|
|
|
|
instanceName = name + "_" + String.format("%03d", i);
|
|
|
|
}
|
|
|
|
luceneIndices[i] = new LLLocalLuceneIndex(lucene,
|
|
|
|
instanceName,
|
2021-05-28 16:04:59 +02:00
|
|
|
indicizerAnalyzers,
|
|
|
|
indicizerSimilarities,
|
2021-07-06 00:30:14 +02:00
|
|
|
luceneOptions
|
2020-12-07 22:15:18 +01:00
|
|
|
);
|
|
|
|
}
|
|
|
|
this.luceneIndices = luceneIndices;
|
2021-09-20 11:52:21 +02:00
|
|
|
this.luceneAnalyzer = LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers);
|
|
|
|
this.luceneSimilarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private LLLocalLuceneIndex getLuceneIndex(LLTerm id) {
|
|
|
|
return luceneIndices[getLuceneIndexId(id)];
|
|
|
|
}
|
|
|
|
|
|
|
|
private int getLuceneIndexId(LLTerm id) {
|
|
|
|
return Math.abs(id.getValue().hashCode()) % luceneIndices.length;
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public String getLuceneIndexName() {
|
2021-01-30 01:41:04 +01:00
|
|
|
return luceneIndices[0].getLuceneIndexName();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-09-20 11:35:01 +02:00
|
|
|
private Mono<Send<LLIndexSearchers>> getIndexSearchers(LLSnapshot snapshot) {
|
2021-09-19 19:59:37 +02:00
|
|
|
return Flux
|
|
|
|
.fromArray(luceneIndices)
|
|
|
|
.index()
|
|
|
|
// Resolve the snapshot of each shard
|
|
|
|
.flatMap(tuple -> Mono
|
|
|
|
.fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1()))
|
2021-09-20 11:35:01 +02:00
|
|
|
.flatMap(luceneSnapshot -> tuple.getT2().retrieveSearcher(
|
|
|
|
luceneSnapshot.orElse(null))
|
2021-09-19 19:59:37 +02:00
|
|
|
)
|
2021-09-20 11:35:01 +02:00
|
|
|
)
|
|
|
|
.collectList()
|
|
|
|
.map(searchers -> LLIndexSearchers.of(searchers).send());
|
2021-09-19 19:59:37 +02:00
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> addDocument(LLTerm id, LLDocument doc) {
|
|
|
|
return getLuceneIndex(id).addDocument(id, doc);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-05-28 16:04:59 +02:00
|
|
|
@SuppressWarnings({"unchecked"})
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-05-28 16:04:59 +02:00
|
|
|
public Mono<Void> addDocuments(Flux<Entry<LLTerm, LLDocument>> documents) {
|
2021-05-11 21:59:05 +02:00
|
|
|
return documents
|
2021-07-27 01:31:18 +02:00
|
|
|
.transform(normal -> new BufferTimeOutPublisher<>(normal, 512, Duration.ofSeconds(2)))
|
2021-05-28 16:04:59 +02:00
|
|
|
.flatMap(inputEntries -> {
|
|
|
|
List<Entry<LLTerm, LLDocument>>[] sortedEntries = new List[luceneIndices.length];
|
|
|
|
Mono<Void>[] results = new Mono[luceneIndices.length];
|
|
|
|
|
|
|
|
// Sort entries
|
|
|
|
for(var inputEntry : inputEntries) {
|
|
|
|
int luceneIndexId = getLuceneIndexId(inputEntry.getKey());
|
|
|
|
if (sortedEntries[luceneIndexId] == null) {
|
|
|
|
sortedEntries[luceneIndexId] = new ArrayList<>();
|
|
|
|
}
|
|
|
|
sortedEntries[luceneIndexId].add(inputEntry);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Add documents
|
|
|
|
int luceneIndexId = 0;
|
|
|
|
for (List<Entry<LLTerm, LLDocument>> docs : sortedEntries) {
|
|
|
|
if (docs != null && !docs.isEmpty()) {
|
|
|
|
LLLocalLuceneIndex luceneIndex = luceneIndices[luceneIndexId];
|
|
|
|
results[luceneIndexId] = luceneIndex.addDocuments(Flux.fromIterable(docs));
|
|
|
|
} else {
|
|
|
|
results[luceneIndexId] = Mono.empty();
|
|
|
|
}
|
|
|
|
luceneIndexId++;
|
|
|
|
}
|
|
|
|
|
|
|
|
return Mono.when(results);
|
2021-05-11 21:59:05 +02:00
|
|
|
})
|
|
|
|
.then();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> deleteDocument(LLTerm id) {
|
|
|
|
return getLuceneIndex(id).deleteDocument(id);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> updateDocument(LLTerm id, LLDocument document) {
|
|
|
|
return getLuceneIndex(id).updateDocument(id, document);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-05-11 21:59:05 +02:00
|
|
|
public Mono<Void> updateDocuments(Mono<Map<LLTerm, LLDocument>> documents) {
|
|
|
|
return documents
|
|
|
|
.flatMapMany(map -> {
|
|
|
|
var sortedMap = new HashMap<LLLocalLuceneIndex, Map<LLTerm, LLDocument>>();
|
|
|
|
map.forEach((key, value) -> sortedMap
|
|
|
|
.computeIfAbsent(getLuceneIndex(key), _unused -> new HashMap<>())
|
|
|
|
.put(key, value)
|
|
|
|
);
|
2021-06-06 02:23:51 +02:00
|
|
|
return Flux.fromIterable(Collections.unmodifiableMap(sortedMap).entrySet());
|
2021-05-11 21:59:05 +02:00
|
|
|
})
|
|
|
|
.flatMap(luceneIndexWithNewDocuments -> {
|
|
|
|
var luceneIndex = luceneIndexWithNewDocuments.getKey();
|
|
|
|
var docs = luceneIndexWithNewDocuments.getValue();
|
|
|
|
return luceneIndex.updateDocuments(Mono.just(docs));
|
|
|
|
})
|
|
|
|
.then();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> deleteAll() {
|
|
|
|
return Flux
|
|
|
|
.fromArray(luceneIndices)
|
|
|
|
.flatMap(LLLocalLuceneIndex::deleteAll)
|
|
|
|
.then();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private LLSnapshot resolveSnapshot(LLSnapshot multiSnapshot, int instanceId) {
|
|
|
|
if (multiSnapshot != null) {
|
|
|
|
return registeredSnapshots.get(multiSnapshot.getSequenceNumber())[instanceId];
|
|
|
|
} else {
|
|
|
|
return null;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-03 13:48:30 +01:00
|
|
|
private Optional<LLSnapshot> resolveSnapshotOptional(LLSnapshot multiSnapshot, int instanceId) {
|
|
|
|
return Optional.ofNullable(resolveSnapshot(multiSnapshot, instanceId));
|
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-09-19 19:59:37 +02:00
|
|
|
public Mono<Send<LLSearchResultShard>> moreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFields) {
|
2021-07-06 01:30:37 +02:00
|
|
|
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams);
|
2021-09-20 11:35:01 +02:00
|
|
|
var searchers = this.getIndexSearchers(snapshot);
|
2021-09-20 11:52:21 +02:00
|
|
|
var transformer = new MultiMoreLikeThisTransformer(mltDocumentFields);
|
2021-09-19 19:59:37 +02:00
|
|
|
|
|
|
|
// Collect all the shards results into a single global result
|
|
|
|
return multiSearcher
|
2021-09-20 11:35:01 +02:00
|
|
|
.collectMulti(searchers, localQueryParams, keyFieldName, transformer)
|
2021-09-19 19:59:37 +02:00
|
|
|
// Transform the result type
|
|
|
|
.map(resultToReceive -> {
|
|
|
|
var result = resultToReceive.receive();
|
|
|
|
return new LLSearchResultShard(result.results(), result.totalHitsCount(),
|
|
|
|
d -> result.close()).send();
|
2021-09-20 11:52:21 +02:00
|
|
|
})
|
|
|
|
.doOnDiscard(Send.class, Send::close);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-09-19 19:59:37 +02:00
|
|
|
public Mono<Send<LLSearchResultShard>> search(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
2020-12-07 22:15:18 +01:00
|
|
|
String keyFieldName) {
|
2021-07-06 01:30:37 +02:00
|
|
|
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams);
|
2021-09-20 11:35:01 +02:00
|
|
|
var searchers = getIndexSearchers(snapshot);
|
2021-04-14 02:37:03 +02:00
|
|
|
|
2021-09-19 19:59:37 +02:00
|
|
|
// Collect all the shards results into a single global result
|
2021-07-06 00:30:14 +02:00
|
|
|
return multiSearcher
|
2021-09-20 11:35:01 +02:00
|
|
|
.collectMulti(searchers, localQueryParams, keyFieldName, LLSearchTransformer.NO_TRANSFORMATION)
|
2021-09-19 19:59:37 +02:00
|
|
|
// Transform the result type
|
|
|
|
.map(resultToReceive -> {
|
|
|
|
var result = resultToReceive.receive();
|
|
|
|
return new LLSearchResultShard(result.results(), result.totalHitsCount(),
|
|
|
|
d -> result.close()).send();
|
2021-09-20 11:52:21 +02:00
|
|
|
})
|
|
|
|
.doOnDiscard(Send.class, Send::close);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> close() {
|
|
|
|
return Flux
|
|
|
|
.fromArray(luceneIndices)
|
|
|
|
.flatMap(LLLocalLuceneIndex::close)
|
|
|
|
.then();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-03 13:48:30 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> flush() {
|
|
|
|
return Flux
|
|
|
|
.fromArray(luceneIndices)
|
|
|
|
.flatMap(LLLocalLuceneIndex::flush)
|
|
|
|
.then();
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-07-18 19:37:24 +02:00
|
|
|
public Mono<Void> refresh(boolean force) {
|
2021-02-03 13:48:30 +01:00
|
|
|
return Flux
|
|
|
|
.fromArray(luceneIndices)
|
2021-07-18 19:37:24 +02:00
|
|
|
.flatMap(index -> index.refresh(force))
|
2021-02-03 13:48:30 +01:00
|
|
|
.then();
|
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<LLSnapshot> takeSnapshot() {
|
|
|
|
return Mono
|
2021-06-25 20:07:19 +02:00
|
|
|
// Generate next snapshot index
|
|
|
|
.fromCallable(nextSnapshotNumber::getAndIncrement)
|
|
|
|
.flatMap(snapshotIndex -> Flux
|
|
|
|
.fromArray(luceneIndices)
|
|
|
|
.flatMapSequential(LLLocalLuceneIndex::takeSnapshot)
|
|
|
|
.collectList()
|
|
|
|
.map(list -> list.toArray(LLSnapshot[]::new))
|
|
|
|
.doOnNext(instancesSnapshotsArray -> registeredSnapshots.put(snapshotIndex, instancesSnapshotsArray))
|
|
|
|
.thenReturn(new LLSnapshot(snapshotIndex))
|
|
|
|
);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> releaseSnapshot(LLSnapshot snapshot) {
|
|
|
|
return Mono
|
2021-06-25 20:07:19 +02:00
|
|
|
.fromCallable(() -> registeredSnapshots.remove(snapshot.getSequenceNumber()))
|
|
|
|
.flatMapMany(Flux::fromArray)
|
|
|
|
.index()
|
|
|
|
.flatMapSequential(tuple -> {
|
|
|
|
int index = (int) (long) tuple.getT1();
|
|
|
|
LLSnapshot instanceSnapshot = tuple.getT2();
|
|
|
|
return luceneIndices[index].releaseSnapshot(instanceSnapshot);
|
2021-01-30 01:41:04 +01:00
|
|
|
})
|
2021-06-25 20:07:19 +02:00
|
|
|
.then();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public boolean isLowMemoryMode() {
|
|
|
|
return luceneIndices[0].isLowMemoryMode();
|
|
|
|
}
|
2021-09-19 19:59:37 +02:00
|
|
|
|
2021-09-20 11:52:21 +02:00
|
|
|
private class MultiMoreLikeThisTransformer implements LLSearchTransformer {
|
2021-09-19 19:59:37 +02:00
|
|
|
|
|
|
|
private final Flux<Tuple2<String, Set<String>>> mltDocumentFields;
|
|
|
|
|
2021-09-20 11:52:21 +02:00
|
|
|
public MultiMoreLikeThisTransformer(Flux<Tuple2<String, Set<String>>> mltDocumentFields) {
|
2021-09-19 19:59:37 +02:00
|
|
|
this.mltDocumentFields = mltDocumentFields;
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-09-20 11:35:01 +02:00
|
|
|
public Mono<LocalQueryParams> transform(Mono<TransformerInput> inputMono) {
|
2021-09-20 11:52:21 +02:00
|
|
|
return inputMono.flatMap(input -> LuceneUtils.getMoreLikeThisQuery(input.indexSearchers(), input.queryParams(),
|
|
|
|
luceneAnalyzer, luceneSimilarity, mltDocumentFields));
|
2021-09-19 19:59:37 +02:00
|
|
|
}
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|