2020-12-07 22:15:18 +01:00
|
|
|
package it.cavallium.dbengine.database.disk;
|
|
|
|
|
2021-07-01 21:19:52 +02:00
|
|
|
import it.cavallium.dbengine.client.DirectIOOptions;
|
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;
|
|
|
|
import it.cavallium.dbengine.client.NRTCachingOptions;
|
2021-03-02 01:53:36 +01:00
|
|
|
import it.cavallium.dbengine.client.query.current.data.QueryParams;
|
2021-02-28 14:52:11 +01:00
|
|
|
import it.cavallium.dbengine.database.EnglishItalianStopFilter;
|
2020-12-31 12:04:53 +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;
|
2020-12-31 12:04:53 +01:00
|
|
|
import it.cavallium.dbengine.database.LLSnapshot;
|
|
|
|
import it.cavallium.dbengine.database.LLTerm;
|
|
|
|
import it.cavallium.dbengine.database.LLUtils;
|
2021-07-01 21:19:52 +02:00
|
|
|
import it.cavallium.dbengine.lucene.AlwaysDirectIOFSDirectory;
|
2021-02-04 22:42:57 +01:00
|
|
|
import it.cavallium.dbengine.lucene.LuceneUtils;
|
2021-02-03 13:48:30 +01:00
|
|
|
import it.cavallium.dbengine.lucene.ScheduledTaskLifecycle;
|
2021-07-06 00:30:14 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneLocalSearcher;
|
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.LuceneLocalSearcher;
|
|
|
|
import it.cavallium.dbengine.lucene.searcher.LuceneShardSearcher;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.io.IOException;
|
|
|
|
import java.nio.file.Path;
|
|
|
|
import java.time.Duration;
|
2021-01-30 19:57:50 +01:00
|
|
|
import java.util.HashMap;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Map;
|
2021-05-11 21:59:05 +02:00
|
|
|
import java.util.Map.Entry;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Set;
|
|
|
|
import java.util.concurrent.TimeUnit;
|
2021-06-25 20:06:58 +02:00
|
|
|
import org.apache.lucene.index.ConcurrentMergeScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.index.IndexWriter;
|
|
|
|
import org.apache.lucene.index.IndexWriterConfig;
|
|
|
|
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
|
2021-07-06 14:33:47 +02:00
|
|
|
import org.apache.lucene.index.MergeScheduler;
|
2021-06-25 20:06:58 +02:00
|
|
|
import org.apache.lucene.index.SerialMergeScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.index.SnapshotDeletionPolicy;
|
2021-05-25 01:12:24 +02:00
|
|
|
import org.apache.lucene.misc.store.DirectIODirectory;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.queries.mlt.MoreLikeThis;
|
2021-02-27 17:32:57 +01:00
|
|
|
import org.apache.lucene.search.BooleanClause.Occur;
|
|
|
|
import org.apache.lucene.search.BooleanQuery;
|
2021-02-27 19:05:13 +01:00
|
|
|
import org.apache.lucene.search.ConstantScoreQuery;
|
2021-07-06 01:30:37 +02:00
|
|
|
import org.apache.lucene.search.MatchAllDocsQuery;
|
|
|
|
import org.apache.lucene.search.MatchNoDocsQuery;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.search.Query;
|
2021-02-04 22:42:57 +01:00
|
|
|
import org.apache.lucene.search.similarities.Similarity;
|
2021-02-28 14:52:11 +01:00
|
|
|
import org.apache.lucene.search.similarities.TFIDFSimilarity;
|
2021-05-25 01:12:24 +02:00
|
|
|
import org.apache.lucene.store.ByteBuffersDirectory;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.store.Directory;
|
2020-12-31 20:10:47 +01:00
|
|
|
import org.apache.lucene.store.FSDirectory;
|
2021-04-30 19:15:04 +02:00
|
|
|
import org.apache.lucene.store.MMapDirectory;
|
2021-05-25 01:12:24 +02:00
|
|
|
import org.apache.lucene.store.NIOFSDirectory;
|
|
|
|
import org.apache.lucene.store.NRTCachingDirectory;
|
|
|
|
import org.apache.lucene.util.Constants;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.jetbrains.annotations.Nullable;
|
2021-02-25 00:00:16 +01:00
|
|
|
import org.warp.commonutils.log.Logger;
|
|
|
|
import org.warp.commonutils.log.LoggerFactory;
|
2020-12-07 22:15:18 +01:00
|
|
|
import reactor.core.publisher.Flux;
|
|
|
|
import reactor.core.publisher.Mono;
|
2021-02-01 02:21:53 +01:00
|
|
|
import reactor.core.scheduler.Scheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
import reactor.core.scheduler.Schedulers;
|
2021-01-30 19:57:50 +01:00
|
|
|
import reactor.util.function.Tuple2;
|
2020-12-07 22:15:18 +01:00
|
|
|
|
|
|
|
public class LLLocalLuceneIndex implements LLLuceneIndex {
|
|
|
|
|
2021-02-25 00:00:16 +01:00
|
|
|
protected static final Logger logger = LoggerFactory.getLogger(LLLocalLuceneIndex.class);
|
2021-07-06 00:30:14 +02:00
|
|
|
private static final LuceneLocalSearcher localSearcher = new AdaptiveLuceneLocalSearcher();
|
2020-12-12 23:41:09 +01:00
|
|
|
/**
|
|
|
|
* Global lucene index scheduler.
|
|
|
|
* There is only a single thread globally to not overwhelm the disk with
|
2021-02-03 14:37:02 +01:00
|
|
|
* concurrent commits or concurrent refreshes.
|
2020-12-12 23:41:09 +01:00
|
|
|
*/
|
2021-03-03 10:57:45 +01:00
|
|
|
private static final Scheduler luceneHeavyTasksScheduler = Schedulers.newBoundedElastic(1,
|
2021-02-03 13:48:30 +01:00
|
|
|
Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
|
2021-02-28 10:57:16 +01:00
|
|
|
"lucene",
|
2021-03-03 10:57:45 +01:00
|
|
|
Integer.MAX_VALUE,
|
2021-07-23 15:20:33 +02:00
|
|
|
true
|
2021-02-03 13:48:30 +01:00
|
|
|
);
|
2020-12-12 23:41:09 +01:00
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
private final String luceneIndexName;
|
|
|
|
private final IndexWriter indexWriter;
|
2021-09-06 15:06:51 +02:00
|
|
|
private final SnapshotsManager snapshotsManager;
|
|
|
|
private final PooledIndexSearcherManager searcherManager;
|
|
|
|
private final Similarity similarity;
|
2020-12-07 22:15:18 +01:00
|
|
|
private final Directory directory;
|
|
|
|
private final boolean lowMemory;
|
|
|
|
|
2020-12-12 23:41:09 +01:00
|
|
|
private final ScheduledTaskLifecycle scheduledTasksLifecycle;
|
|
|
|
|
2021-07-10 20:52:01 +02:00
|
|
|
public LLLocalLuceneIndex(@Nullable Path luceneBasePath,
|
2020-12-07 22:15:18 +01:00
|
|
|
String name,
|
2021-05-28 16:04:59 +02:00
|
|
|
IndicizerAnalyzers indicizerAnalyzers,
|
|
|
|
IndicizerSimilarities indicizerSimilarities,
|
2021-07-06 00:30:14 +02:00
|
|
|
LuceneOptions luceneOptions) throws IOException {
|
2021-07-10 20:52:01 +02:00
|
|
|
Path directoryPath;
|
|
|
|
if (luceneOptions.inMemory() != (luceneBasePath == null)) {
|
|
|
|
throw new IllegalArgumentException();
|
|
|
|
} else if (luceneBasePath != null) {
|
|
|
|
directoryPath = luceneBasePath.resolve(name + ".lucene.db");
|
|
|
|
} else {
|
|
|
|
directoryPath = null;
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
if (name.length() == 0) {
|
|
|
|
throw new IOException("Empty lucene database name");
|
|
|
|
}
|
2021-05-25 01:12:24 +02:00
|
|
|
if (!MMapDirectory.UNMAP_SUPPORTED) {
|
|
|
|
logger.error("Unmap is unsupported, lucene will run slower: {}", MMapDirectory.UNMAP_NOT_SUPPORTED_REASON);
|
|
|
|
} else {
|
|
|
|
logger.debug("Lucene MMap is supported");
|
|
|
|
}
|
2021-07-01 21:19:52 +02:00
|
|
|
boolean lowMemory = luceneOptions.lowMemory();
|
|
|
|
if (luceneOptions.inMemory()) {
|
2021-05-25 01:12:24 +02:00
|
|
|
this.directory = new ByteBuffersDirectory();
|
|
|
|
} else {
|
|
|
|
Directory directory;
|
|
|
|
{
|
2021-07-01 21:19:52 +02:00
|
|
|
Directory forcedDirectFsDirectory = null;
|
|
|
|
if (luceneOptions.directIOOptions().isPresent()) {
|
|
|
|
DirectIOOptions directIOOptions = luceneOptions.directIOOptions().get();
|
|
|
|
if (directIOOptions.alwaysForceDirectIO()) {
|
|
|
|
try {
|
|
|
|
forcedDirectFsDirectory = new AlwaysDirectIOFSDirectory(directoryPath);
|
|
|
|
} catch (UnsupportedOperationException ex) {
|
|
|
|
logger.warn("Failed to open FSDirectory with DIRECT flag", ex);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (forcedDirectFsDirectory != null) {
|
|
|
|
directory = forcedDirectFsDirectory;
|
|
|
|
} else {
|
|
|
|
FSDirectory fsDirectory;
|
|
|
|
if (luceneOptions.allowMemoryMapping()) {
|
|
|
|
fsDirectory = FSDirectory.open(directoryPath);
|
2021-05-25 01:12:24 +02:00
|
|
|
} else {
|
2021-07-01 21:19:52 +02:00
|
|
|
fsDirectory = new NIOFSDirectory(directoryPath);
|
|
|
|
}
|
|
|
|
if (Constants.LINUX || Constants.MAC_OS_X) {
|
|
|
|
try {
|
|
|
|
int mergeBufferSize;
|
|
|
|
long minBytesDirect;
|
|
|
|
if (luceneOptions.directIOOptions().isPresent()) {
|
|
|
|
var directIOOptions = luceneOptions.directIOOptions().get();
|
|
|
|
mergeBufferSize = directIOOptions.mergeBufferSize();
|
|
|
|
minBytesDirect = directIOOptions.minBytesDirect();
|
|
|
|
} else {
|
|
|
|
mergeBufferSize = DirectIODirectory.DEFAULT_MERGE_BUFFER_SIZE;
|
|
|
|
minBytesDirect = DirectIODirectory.DEFAULT_MIN_BYTES_DIRECT;
|
|
|
|
}
|
|
|
|
directory = new DirectIODirectory(fsDirectory, mergeBufferSize, minBytesDirect);
|
|
|
|
} catch (UnsupportedOperationException ex) {
|
|
|
|
logger.warn("Failed to open FSDirectory with DIRECT flag", ex);
|
|
|
|
directory = fsDirectory;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
directory = fsDirectory;
|
2021-05-25 01:12:24 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-01 21:19:52 +02:00
|
|
|
if (luceneOptions.nrtCachingOptions().isPresent()) {
|
|
|
|
NRTCachingOptions nrtCachingOptions = luceneOptions.nrtCachingOptions().get();
|
|
|
|
directory = new NRTCachingDirectory(directory, nrtCachingOptions.maxMergeSizeMB(), nrtCachingOptions.maxCachedMB());
|
2021-05-25 01:12:24 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
this.directory = directory;
|
|
|
|
}
|
2021-06-06 02:23:51 +02:00
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
this.luceneIndexName = name;
|
2021-09-06 15:06:51 +02:00
|
|
|
var snapshotter = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
|
2020-12-07 22:15:18 +01:00
|
|
|
this.lowMemory = lowMemory;
|
2021-05-28 16:04:59 +02:00
|
|
|
this.similarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities);
|
2021-06-06 02:23:51 +02:00
|
|
|
|
2021-09-06 15:06:51 +02:00
|
|
|
// Create scheduled tasks lifecycle manager
|
|
|
|
this.scheduledTasksLifecycle = new ScheduledTaskLifecycle();
|
|
|
|
|
2021-05-28 16:04:59 +02:00
|
|
|
IndexWriterConfig indexWriterConfig = new IndexWriterConfig(LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers));
|
2020-12-07 22:15:18 +01:00
|
|
|
indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
|
|
|
|
indexWriterConfig.setIndexDeletionPolicy(snapshotter);
|
|
|
|
indexWriterConfig.setCommitOnClose(true);
|
2021-07-06 14:33:47 +02:00
|
|
|
MergeScheduler mergeScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
if (lowMemory) {
|
2021-07-06 14:33:47 +02:00
|
|
|
mergeScheduler = new SerialMergeScheduler();
|
2020-12-07 22:15:18 +01:00
|
|
|
} else {
|
2021-07-06 14:33:47 +02:00
|
|
|
var concurrentMergeScheduler = new ConcurrentMergeScheduler();
|
2021-07-23 15:20:33 +02:00
|
|
|
concurrentMergeScheduler.setDefaultMaxMergesAndThreads(false);
|
|
|
|
if (luceneOptions.inMemory()) {
|
|
|
|
concurrentMergeScheduler.disableAutoIOThrottle();
|
|
|
|
} else {
|
|
|
|
concurrentMergeScheduler.enableAutoIOThrottle();
|
|
|
|
}
|
2021-07-06 14:33:47 +02:00
|
|
|
mergeScheduler = concurrentMergeScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
2021-07-06 14:33:47 +02:00
|
|
|
indexWriterConfig.setMergeScheduler(mergeScheduler);
|
|
|
|
indexWriterConfig.setRAMBufferSizeMB(luceneOptions.indexWriterBufferSize() / 1024D / 1024D);
|
|
|
|
indexWriterConfig.setReaderPooling(false);
|
2021-02-04 22:42:57 +01:00
|
|
|
indexWriterConfig.setSimilarity(getSimilarity());
|
2020-12-07 22:15:18 +01:00
|
|
|
this.indexWriter = new IndexWriter(directory, indexWriterConfig);
|
2021-09-06 15:06:51 +02:00
|
|
|
this.snapshotsManager = new SnapshotsManager(indexWriter, snapshotter, scheduledTasksLifecycle);
|
|
|
|
this.searcherManager = new PooledIndexSearcherManager(indexWriter,
|
|
|
|
snapshotsManager,
|
|
|
|
getSimilarity(),
|
2021-07-06 14:33:47 +02:00
|
|
|
luceneOptions.applyAllDeletes(),
|
|
|
|
luceneOptions.writeAllDeletes(),
|
2021-09-06 15:06:51 +02:00
|
|
|
luceneOptions.queryRefreshDebounceTime()
|
2021-07-06 14:33:47 +02:00
|
|
|
);
|
2020-12-12 23:41:09 +01:00
|
|
|
|
|
|
|
// Start scheduled tasks
|
2021-07-01 21:19:52 +02:00
|
|
|
registerScheduledFixedTask(this::scheduledCommit, luceneOptions.commitDebounceTime());
|
2020-12-12 23:41:09 +01:00
|
|
|
}
|
|
|
|
|
2021-02-04 22:42:57 +01:00
|
|
|
private Similarity getSimilarity() {
|
2021-05-28 16:04:59 +02:00
|
|
|
return similarity;
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
2020-12-12 23:41:09 +01:00
|
|
|
private void registerScheduledFixedTask(Runnable task, Duration duration) {
|
2021-07-01 21:19:52 +02:00
|
|
|
new PeriodicTask(task, duration).start();
|
|
|
|
}
|
|
|
|
|
|
|
|
private class PeriodicTask implements Runnable {
|
|
|
|
|
|
|
|
private final Runnable task;
|
|
|
|
private final Duration duration;
|
|
|
|
private volatile boolean cancelled = false;
|
|
|
|
|
|
|
|
public PeriodicTask(Runnable task, Duration duration) {
|
|
|
|
this.task = task;
|
|
|
|
this.duration = duration;
|
|
|
|
}
|
|
|
|
|
|
|
|
public void start() {
|
|
|
|
luceneHeavyTasksScheduler.schedule(this,
|
|
|
|
duration.toMillis(),
|
|
|
|
TimeUnit.MILLISECONDS
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public void run() {
|
2021-08-04 00:44:06 +02:00
|
|
|
if (!scheduledTasksLifecycle.tryStartScheduledTask()) {
|
|
|
|
return;
|
|
|
|
}
|
2020-12-31 12:05:04 +01:00
|
|
|
try {
|
2021-07-01 21:19:52 +02:00
|
|
|
if (scheduledTasksLifecycle.isCancelled() || cancelled) return;
|
2020-12-31 12:05:04 +01:00
|
|
|
task.run();
|
2021-07-01 21:19:52 +02:00
|
|
|
if (scheduledTasksLifecycle.isCancelled() || cancelled) return;
|
|
|
|
luceneHeavyTasksScheduler.schedule(this, duration.toMillis(), TimeUnit.MILLISECONDS);
|
2020-12-31 12:05:04 +01:00
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-07-01 21:19:52 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
public void cancel() {
|
|
|
|
cancelled = true;
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public String getLuceneIndexName() {
|
|
|
|
return luceneIndexName;
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<LLSnapshot> takeSnapshot() {
|
2021-09-06 15:06:51 +02:00
|
|
|
return snapshotsManager.takeSnapshot().subscribeOn(luceneHeavyTasksScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> releaseSnapshot(LLSnapshot snapshot) {
|
2021-09-06 15:06:51 +02:00
|
|
|
return snapshotsManager.releaseSnapshot(snapshot);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> addDocument(LLTerm key, LLDocument doc) {
|
2021-09-04 16:42:47 +02:00
|
|
|
return Mono.fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
indexWriter.addDocument(LLUtils.toDocument(doc));
|
|
|
|
return null;
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-09-04 16:42:47 +02:00
|
|
|
});
|
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-01-30 19:57:50 +01:00
|
|
|
return documents
|
2021-05-28 16:04:59 +02:00
|
|
|
.collectList()
|
|
|
|
.flatMap(documentsList -> Mono
|
2021-07-31 12:28:53 +02:00
|
|
|
.fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
indexWriter.addDocuments(LLUtils.toDocumentsFromEntries(documentsList));
|
|
|
|
return null;
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-05-11 21:59:05 +02:00
|
|
|
})
|
|
|
|
);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 19:57:50 +01:00
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> deleteDocument(LLTerm id) {
|
2021-09-04 16:42:47 +02:00
|
|
|
return Mono.fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
indexWriter.deleteDocuments(LLUtils.toTerm(id));
|
|
|
|
return null;
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-09-04 16:42:47 +02:00
|
|
|
});
|
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) {
|
2021-09-04 16:42:47 +02:00
|
|
|
return Mono.fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document));
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-01-30 01:41:04 +01:00
|
|
|
return null;
|
2021-09-04 16:42:47 +02:00
|
|
|
});
|
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) {
|
2021-01-30 19:57:50 +01:00
|
|
|
return documents.flatMap(this::updateDocuments).then();
|
|
|
|
}
|
2021-01-30 01:41:04 +01:00
|
|
|
|
2021-05-11 21:59:05 +02:00
|
|
|
private Mono<Void> updateDocuments(Map<LLTerm, LLDocument> documentsMap) {
|
|
|
|
return Mono
|
2021-09-04 16:42:47 +02:00
|
|
|
.fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
for (Entry<LLTerm, LLDocument> entry : documentsMap.entrySet()) {
|
|
|
|
LLTerm key = entry.getKey();
|
|
|
|
LLDocument value = entry.getValue();
|
|
|
|
indexWriter.updateDocument(LLUtils.toTerm(key), LLUtils.toDocument(value));
|
|
|
|
}
|
|
|
|
return null;
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
2021-05-11 21:59:05 +02:00
|
|
|
}
|
2021-09-04 16:42:47 +02:00
|
|
|
});
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> deleteAll() {
|
|
|
|
return Mono.<Void>fromCallable(() -> {
|
2021-07-01 21:19:52 +02:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
indexWriter.deleteAll();
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
indexWriter.forceMergeDeletes(true);
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
indexWriter.commit();
|
|
|
|
return null;
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
2021-03-03 10:57:45 +01:00
|
|
|
}).subscribeOn(luceneHeavyTasksScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-07-06 00:30:14 +02:00
|
|
|
public Mono<LLSearchResultShard> moreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux) {
|
2021-07-06 01:30:37 +02:00
|
|
|
return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux)
|
2021-09-06 15:06:51 +02:00
|
|
|
.flatMap(modifiedLocalQuery -> searcherManager.captureIndexSearcher(snapshot)
|
2021-07-10 20:52:01 +02:00
|
|
|
.flatMap(indexSearcher -> {
|
2021-09-06 18:23:47 +02:00
|
|
|
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
|
2021-07-10 20:52:01 +02:00
|
|
|
return localSearcher
|
2021-09-06 17:35:02 +02:00
|
|
|
.collect(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, keyFieldName)
|
2021-07-10 20:52:01 +02:00
|
|
|
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()))
|
|
|
|
.onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
|
|
|
|
})
|
2021-07-06 01:30:37 +02:00
|
|
|
);
|
|
|
|
}
|
|
|
|
|
|
|
|
public Mono<Void> distributedMoreLikeThis(@Nullable LLSnapshot snapshot,
|
|
|
|
QueryParams queryParams,
|
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
|
|
|
LuceneShardSearcher shardSearcher) {
|
|
|
|
return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux)
|
2021-09-06 15:06:51 +02:00
|
|
|
.flatMap(modifiedLocalQuery -> searcherManager.captureIndexSearcher(snapshot)
|
2021-07-10 20:52:01 +02:00
|
|
|
.flatMap(indexSearcher -> {
|
2021-09-06 18:23:47 +02:00
|
|
|
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
|
2021-07-10 20:52:01 +02:00
|
|
|
return shardSearcher
|
2021-09-06 17:35:02 +02:00
|
|
|
.searchOn(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery)
|
2021-07-10 20:52:01 +02:00
|
|
|
.onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
|
|
|
|
})
|
2021-07-06 01:30:37 +02:00
|
|
|
);
|
|
|
|
}
|
|
|
|
|
|
|
|
public Mono<LocalQueryParams> getMoreLikeThisQuery(@Nullable LLSnapshot snapshot,
|
|
|
|
LocalQueryParams localQueryParams,
|
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux) {
|
|
|
|
Query luceneAdditionalQuery;
|
|
|
|
try {
|
|
|
|
luceneAdditionalQuery = localQueryParams.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 LocalQueryParams(new MatchNoDocsQuery(),
|
|
|
|
localQueryParams.offset(),
|
|
|
|
localQueryParams.limit(),
|
|
|
|
localQueryParams.minCompetitiveScore(),
|
|
|
|
localQueryParams.sort(),
|
|
|
|
localQueryParams.scoreMode()
|
|
|
|
));
|
|
|
|
}
|
2021-09-06 15:06:51 +02:00
|
|
|
return this.searcherManager.search(snapshot, 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(localQueryParams.scoreMode().needsScores());
|
|
|
|
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
|
|
|
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
|
|
|
var mltQuery = mlt.like((Map) mltDocumentFields);
|
|
|
|
Query luceneQuery;
|
|
|
|
if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) {
|
|
|
|
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 -> new LocalQueryParams(luceneQuery,
|
|
|
|
localQueryParams.offset(),
|
|
|
|
localQueryParams.limit(),
|
|
|
|
localQueryParams.minCompetitiveScore(),
|
|
|
|
localQueryParams.sort(),
|
|
|
|
localQueryParams.scoreMode()
|
|
|
|
)));
|
2021-07-06 01:30:37 +02:00
|
|
|
});
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-07-06 00:30:14 +02:00
|
|
|
public Mono<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) {
|
2021-07-06 01:30:37 +02:00
|
|
|
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams);
|
2021-09-06 15:06:51 +02:00
|
|
|
return searcherManager.captureIndexSearcher(snapshot).flatMap(indexSearcher -> {
|
2021-09-06 18:23:47 +02:00
|
|
|
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
|
2021-09-06 15:06:51 +02:00
|
|
|
return localSearcher
|
2021-09-06 17:35:02 +02:00
|
|
|
.collect(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, keyFieldName)
|
2021-09-06 15:06:51 +02:00
|
|
|
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()))
|
|
|
|
.onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
|
|
|
|
});
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
2021-07-06 00:30:14 +02:00
|
|
|
public Mono<Void> distributedSearch(@Nullable LLSnapshot snapshot,
|
2021-04-03 02:20:37 +02:00
|
|
|
QueryParams queryParams,
|
2021-07-06 00:30:14 +02:00
|
|
|
LuceneShardSearcher shardSearcher) {
|
2021-07-06 01:30:37 +02:00
|
|
|
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams);
|
2021-09-06 15:06:51 +02:00
|
|
|
return searcherManager.captureIndexSearcher(snapshot)
|
2021-07-10 20:52:01 +02:00
|
|
|
.flatMap(indexSearcher -> {
|
2021-09-06 18:23:47 +02:00
|
|
|
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
|
2021-09-06 17:35:02 +02:00
|
|
|
return shardSearcher.searchOn(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams)
|
2021-07-10 20:52:01 +02:00
|
|
|
.onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
|
|
|
|
});
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 01:41:04 +01:00
|
|
|
public Mono<Void> close() {
|
|
|
|
return Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
2021-03-19 20:55:38 +01:00
|
|
|
logger.debug("Closing IndexWriter...");
|
2021-01-30 01:41:04 +01:00
|
|
|
scheduledTasksLifecycle.cancelAndWait();
|
2021-09-06 15:06:51 +02:00
|
|
|
return null;
|
|
|
|
})
|
|
|
|
.subscribeOn(luceneHeavyTasksScheduler)
|
|
|
|
.then(searcherManager.close())
|
|
|
|
.then(Mono.<Void>fromCallable(() -> {
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-01-30 01:41:04 +01:00
|
|
|
indexWriter.close();
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-01-30 01:41:04 +01:00
|
|
|
directory.close();
|
2021-03-19 20:55:38 +01:00
|
|
|
logger.debug("IndexWriter closed");
|
2021-01-30 01:41:04 +01:00
|
|
|
return null;
|
2021-09-06 15:06:51 +02:00
|
|
|
}).subscribeOn(luceneHeavyTasksScheduler));
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-03 13:48:30 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> flush() {
|
|
|
|
return Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
2021-07-01 21:19:52 +02:00
|
|
|
if (scheduledTasksLifecycle.isCancelled()) return null;
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-02-03 13:48:30 +01:00
|
|
|
indexWriter.commit();
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
|
|
|
return null;
|
|
|
|
})
|
2021-03-03 10:57:45 +01:00
|
|
|
.subscribeOn(luceneHeavyTasksScheduler);
|
2021-02-03 13:48:30 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-07-18 19:37:24 +02:00
|
|
|
public Mono<Void> refresh(boolean force) {
|
2021-02-03 13:48:30 +01:00
|
|
|
return Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
2021-07-01 21:19:52 +02:00
|
|
|
if (scheduledTasksLifecycle.isCancelled()) return null;
|
2021-07-18 19:37:24 +02:00
|
|
|
if (force) {
|
|
|
|
if (scheduledTasksLifecycle.isCancelled()) return null;
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
searcherManager.maybeRefreshBlocking();
|
|
|
|
} else {
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
searcherManager.maybeRefresh();
|
|
|
|
}
|
2021-02-03 13:48:30 +01:00
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
|
|
|
return null;
|
|
|
|
})
|
2021-03-03 10:57:45 +01:00
|
|
|
.subscribeOn(luceneHeavyTasksScheduler);
|
2021-02-03 13:48:30 +01:00
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
private void scheduledCommit() {
|
|
|
|
try {
|
|
|
|
if (indexWriter.hasUncommittedChanges()) {
|
|
|
|
indexWriter.commit();
|
|
|
|
}
|
|
|
|
} catch (IOException ex) {
|
|
|
|
ex.printStackTrace();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public boolean isLowMemoryMode() {
|
|
|
|
return lowMemory;
|
|
|
|
}
|
|
|
|
}
|