CavalliumDBEngine/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java

291 lines
9.5 KiB
Java
Raw Normal View History

2020-12-07 22:15:18 +01:00
package it.cavallium.dbengine.database.disk;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
2021-03-18 13:12:40 +01:00
import com.google.common.cache.CacheLoader.InvalidCacheLoadException;
2021-05-28 16:04:59 +02:00
import it.cavallium.dbengine.client.IndicizerAnalyzers;
import it.cavallium.dbengine.client.IndicizerSimilarities;
import it.cavallium.dbengine.client.LuceneIndex;
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;
import it.cavallium.dbengine.database.LLSearchResult;
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;
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;
2020-12-07 22:15:18 +01:00
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
2021-05-28 16:04:59 +02:00
import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap;
2020-12-07 22:15:18 +01:00
import java.io.IOException;
import java.nio.file.Path;
import java.time.Duration;
import java.util.ArrayList;
2021-06-06 02:23:51 +02:00
import java.util.Collections;
import java.util.HashMap;
2021-05-28 16:04:59 +02:00
import java.util.List;
import java.util.Map;
2021-05-28 16:04:59 +02:00
import java.util.Map.Entry;
2021-03-18 13:12:40 +01:00
import java.util.Objects;
2021-02-03 13:48:30 +01:00
import java.util.Optional;
2020-12-07 22:15:18 +01:00
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
2020-12-07 22:15:18 +01:00
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
2020-12-07 22:15:18 +01:00
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.IndexSearcher;
2020-12-07 22:15:18 +01:00
import org.jetbrains.annotations.Nullable;
import org.warp.commonutils.batch.ParallelUtils;
import org.warp.commonutils.functional.IOBiConsumer;
import reactor.core.publisher.Flux;
2021-01-30 20:01:22 +01:00
import reactor.core.publisher.GroupedFlux;
2020-12-07 22:15:18 +01:00
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
2021-01-30 20:01:22 +01:00
import reactor.util.function.Tuple2;
2020-12-07 22:15:18 +01:00
import reactor.util.function.Tuples;
public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
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;
private final LuceneMultiSearcher multiSearcher = new AdaptiveLuceneMultiSearcher();
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,
luceneOptions
2020-12-07 22:15:18 +01:00
);
}
this.luceneIndices = luceneIndices;
}
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() {
return luceneIndices[0].getLuceneIndexName();
2020-12-07 22:15:18 +01:00
}
@Override
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) {
return documents
2021-05-28 16:04:59 +02:00
.bufferTimeout(512, Duration.ofSeconds(2))
.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);
})
.then();
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> deleteDocument(LLTerm id) {
return getLuceneIndex(id).deleteDocument(id);
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> updateDocument(LLTerm id, LLDocument document) {
return getLuceneIndex(id).updateDocument(id, document);
2020-12-07 22:15:18 +01:00
}
@Override
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());
})
.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
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
public Mono<LLSearchResultShard> moreLikeThis(@Nullable LLSnapshot snapshot,
2021-03-02 01:53:36 +01:00
QueryParams queryParams,
String keyFieldName,
Flux<Tuple2<String, Set<String>>> mltDocumentFields) {
throw new NotImplementedException();
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<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-04-14 02:37:03 +02:00
record LuceneIndexWithSnapshot(LLLocalLuceneIndex luceneIndex, Optional<LLSnapshot> snapshot) {}
2021-04-14 02:37:03 +02:00
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 -> new LuceneIndexWithSnapshot(tuple.getT2(), luceneSnapshot))
)
// 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()));
2020-12-07 22:15:18 +01:00
}
@Override
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
public Mono<Void> refresh() {
return Flux
.fromArray(luceneIndices)
.flatMap(LLLocalLuceneIndex::refresh)
.then();
}
2020-12-07 22:15:18 +01:00
@Override
public Mono<LLSnapshot> takeSnapshot() {
return Mono
// 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
public Mono<Void> releaseSnapshot(LLSnapshot snapshot) {
return Mono
.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);
})
.then();
2020-12-07 22:15:18 +01:00
}
@Override
public boolean isLowMemoryMode() {
return luceneIndices[0].isLowMemoryMode();
}
}