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

574 lines
19 KiB
Java
Raw Normal View History

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.QueryParser;
import it.cavallium.dbengine.client.query.current.data.QueryParams;
2021-02-28 14:52:11 +01:00
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.LLSearchResult;
import it.cavallium.dbengine.database.LLSearchResultShard;
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;
import it.cavallium.dbengine.lucene.LuceneUtils;
2021-02-03 13:48:30 +01:00
import it.cavallium.dbengine.lucene.ScheduledTaskLifecycle;
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;
2020-12-07 22:15:18 +01:00
import java.io.IOException;
import java.nio.file.Path;
import java.time.Duration;
import java.util.HashMap;
2020-12-07 22:15:18 +01:00
import java.util.Map;
import java.util.Map.Entry;
2020-12-07 22:15:18 +01:00
import java.util.Objects;
import java.util.Optional;
2020-12-07 22:15:18 +01:00
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;
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.IndexCommit;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
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;
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;
2020-12-07 22:15:18 +01:00
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
2020-12-07 22:15:18 +01:00
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.search.Sort;
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;
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;
import reactor.core.scheduler.Scheduler;
2020-12-07 22:15:18 +01:00
import reactor.core.scheduler.Schedulers;
import reactor.util.function.Tuple2;
2020-12-07 22:15:18 +01:00
import reactor.util.function.Tuples;
public class LLLocalLuceneIndex implements LLLuceneIndex {
2021-02-25 00:00:16 +01:00
protected static final Logger logger = LoggerFactory.getLogger(LLLocalLuceneIndex.class);
private static final LuceneLocalSearcher localSearcher = new AdaptiveLuceneLocalSearcher();
/**
* 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.
*/
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-04 01:34:17 +02:00
false
2021-02-03 13:48:30 +01:00
);
2021-04-15 00:00:42 +02:00
// Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks
2021-07-04 01:34:17 +02:00
private final Scheduler luceneSearcherScheduler = Schedulers.newBoundedElastic(
4,
Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
"lucene-searcher",
60,
false
);
2020-12-07 22:15:18 +01:00
private final String luceneIndexName;
private final SnapshotDeletionPolicy snapshotter;
private final IndexWriter indexWriter;
private final SearcherManager searcherManager;
private final Directory directory;
/**
* Last snapshot sequence number. 0 is not used
*/
private final AtomicLong lastSnapshotSeqNo = new AtomicLong(0);
/**
2021-03-02 01:53:36 +01:00
* LLSnapshot seq no to index commit point
2020-12-07 22:15:18 +01:00
*/
private final ConcurrentHashMap<Long, LuceneIndexSnapshot> snapshots = new ConcurrentHashMap<>();
private final boolean lowMemory;
2021-05-28 16:04:59 +02:00
private final Similarity similarity;
2020-12-07 22:15:18 +01:00
private final ScheduledTaskLifecycle scheduledTasksLifecycle;
2020-12-07 22:15:18 +01:00
public LLLocalLuceneIndex(Path luceneBasePath,
String name,
2021-05-28 16:04:59 +02:00
IndicizerAnalyzers indicizerAnalyzers,
IndicizerSimilarities indicizerSimilarities,
LuceneOptions luceneOptions) throws IOException {
2020-12-07 22:15:18 +01:00
if (name.length() == 0) {
throw new IOException("Empty lucene database name");
}
Path directoryPath = luceneBasePath.resolve(name + ".lucene.db");
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;
this.snapshotter = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
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-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);
if (lowMemory) {
2021-06-25 20:06:58 +02:00
indexWriterConfig.setRAMBufferSizeMB(8);
2020-12-07 22:15:18 +01:00
indexWriterConfig.setRAMPerThreadHardLimitMB(32);
2021-06-25 20:06:58 +02:00
var mergeScheduler = new SerialMergeScheduler();
indexWriterConfig.setMergeScheduler(mergeScheduler);
2020-12-07 22:15:18 +01:00
} else {
2021-06-25 20:06:58 +02:00
indexWriterConfig.setRAMBufferSizeMB(16);
2021-03-20 12:41:11 +01:00
//indexWriterConfig.setRAMPerThreadHardLimitMB(512);
2021-06-25 20:06:58 +02:00
var mergeScheduler = new ConcurrentMergeScheduler();
mergeScheduler.enableAutoIOThrottle();
indexWriterConfig.setMergeScheduler(mergeScheduler);
2020-12-07 22:15:18 +01:00
}
indexWriterConfig.setSimilarity(getSimilarity());
2020-12-07 22:15:18 +01:00
this.indexWriter = new IndexWriter(directory, indexWriterConfig);
2021-06-06 02:23:51 +02:00
this.searcherManager = new SearcherManager(indexWriter, false, false, null);
// Create scheduled tasks lifecycle manager
this.scheduledTasksLifecycle = new ScheduledTaskLifecycle();
// Start scheduled tasks
2021-07-01 21:19:52 +02:00
registerScheduledFixedTask(this::scheduledCommit, luceneOptions.commitDebounceTime());
registerScheduledFixedTask(this::scheduledQueryRefresh, luceneOptions.queryRefreshDebounceTime());
}
private Similarity getSimilarity() {
2021-05-28 16:04:59 +02:00
return similarity;
}
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() {
2020-12-31 12:05:04 +01:00
scheduledTasksLifecycle.startScheduledTask();
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
public Mono<LLSnapshot> takeSnapshot() {
2021-03-03 10:57:45 +01:00
return takeLuceneSnapshot()
.flatMap(snapshot -> Mono
.fromCallable(() -> {
2021-03-03 10:57:45 +01:00
var snapshotSeqNo = lastSnapshotSeqNo.incrementAndGet();
this.snapshots.put(snapshotSeqNo, new LuceneIndexSnapshot(snapshot));
return new LLSnapshot(snapshotSeqNo);
})
2021-04-15 00:00:42 +02:00
.subscribeOn(Schedulers.boundedElastic())
2021-03-03 10:57:45 +01:00
);
2020-12-07 22:15:18 +01:00
}
/**
* Use internally. This method commits before taking the snapshot if there are no commits in a new database,
* avoiding the exception.
*/
private Mono<IndexCommit> takeLuceneSnapshot() {
2021-02-03 14:37:02 +01:00
return Mono
2021-03-03 10:57:45 +01:00
.fromCallable(snapshotter::snapshot)
2021-04-15 00:00:42 +02:00
.subscribeOn(Schedulers.boundedElastic())
2021-03-03 10:57:45 +01:00
.onErrorResume(ex -> Mono
.defer(() -> {
if (ex instanceof IllegalStateException && "No index commit to snapshot".equals(ex.getMessage())) {
return Mono.fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
//noinspection BlockingMethodInNonBlockingContext
indexWriter.commit();
//noinspection BlockingMethodInNonBlockingContext
return snapshotter.snapshot();
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
2021-03-03 10:57:45 +01:00
}).subscribeOn(luceneHeavyTasksScheduler);
} else {
return Mono.error(ex);
}
})
);
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> releaseSnapshot(LLSnapshot snapshot) {
return Mono.<Void>fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
var indexSnapshot = this.snapshots.remove(snapshot.getSequenceNumber());
if (indexSnapshot == null) {
throw new IOException("LLSnapshot " + snapshot.getSequenceNumber() + " not found!");
}
2020-12-07 22:15:18 +01:00
2021-07-01 21:19:52 +02:00
indexSnapshot.close();
2020-12-07 22:15:18 +01:00
2021-07-01 21:19:52 +02:00
var luceneIndexSnapshot = indexSnapshot.getSnapshot();
snapshotter.release(luceneIndexSnapshot);
// Delete unused files after releasing the snapshot
indexWriter.deleteUnusedFiles();
return null;
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> addDocument(LLTerm key, LLDocument doc) {
return Mono.<Void>fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
indexWriter.addDocument(LLUtils.toDocument(doc));
return null;
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
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
.collectList()
.flatMap(documentsList -> Mono
.<Void>fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
indexWriter.addDocuments(LLUtils.toDocumentsFromEntries(documentsList));
return null;
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
})
.subscribeOn(Schedulers.boundedElastic())
);
2020-12-07 22:15:18 +01:00
}
2020-12-07 22:15:18 +01:00
@Override
public Mono<Void> deleteDocument(LLTerm id) {
return Mono.<Void>fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
indexWriter.deleteDocuments(LLUtils.toTerm(id));
return null;
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> updateDocument(LLTerm id, LLDocument document) {
return Mono.<Void>fromCallable(() -> {
2021-07-01 21:19:52 +02:00
scheduledTasksLifecycle.startScheduledTask();
try {
indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document));
} finally {
scheduledTasksLifecycle.endScheduledTask();
}
return null;
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> updateDocuments(Mono<Map<LLTerm, LLDocument>> documents) {
return documents.flatMap(this::updateDocuments).then();
}
private Mono<Void> updateDocuments(Map<LLTerm, LLDocument> documentsMap) {
return Mono
.<Void>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();
}
})
.subscribeOn(Schedulers.boundedElastic());
2020-12-07 22:15:18 +01:00
}
@Override
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
}
private Mono<IndexSearcher> acquireSearcherWrapper(LLSnapshot snapshot) {
return Mono.fromCallable(() -> {
IndexSearcher indexSearcher;
if (snapshot == null) {
indexSearcher = searcherManager.acquire();
indexSearcher.setSimilarity(getSimilarity());
} else {
indexSearcher = resolveSnapshot(snapshot).getIndexSearcher();
}
return indexSearcher;
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
}
2020-12-07 22:15:18 +01:00
private Mono<Void> releaseSearcherWrapper(LLSnapshot snapshot, IndexSearcher indexSearcher) {
return Mono.<Void>fromRunnable(() -> {
if (snapshot == null) {
try {
searcherManager.release(indexSearcher);
} catch (IOException e) {
e.printStackTrace();
}
}
2021-04-15 00:00:42 +02:00
}).subscribeOn(Schedulers.boundedElastic());
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>>> mltDocumentFieldsFlux) {
throw new NotImplementedException();
}
2020-12-07 22:15:18 +01:00
@Override
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<Void> distributedSearch(@Nullable LLSnapshot snapshot,
2021-04-03 02:20:37 +02:00
QueryParams queryParams,
LuceneShardSearcher shardSearcher) {
return Mono
.usingWhen(
this.acquireSearcherWrapper(snapshot),
indexSearcher -> shardSearcher.searchOn(indexSearcher, queryParams, luceneSearcherScheduler),
indexSearcher -> releaseSearcherWrapper(snapshot, indexSearcher)
);
2020-12-07 22:15:18 +01:00
}
@Override
public Mono<Void> close() {
return Mono
.<Void>fromCallable(() -> {
2021-03-19 20:55:38 +01:00
logger.debug("Closing IndexWriter...");
scheduledTasksLifecycle.cancelAndWait();
2021-02-03 14:37:02 +01:00
//noinspection BlockingMethodInNonBlockingContext
indexWriter.close();
2021-02-03 14:37:02 +01:00
//noinspection BlockingMethodInNonBlockingContext
directory.close();
2021-03-19 20:55:38 +01:00
logger.debug("IndexWriter closed");
return null;
})
2021-03-03 10:57:45 +01: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
public Mono<Void> refresh() {
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-05-25 01:12:24 +02:00
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();
}
}
@SuppressWarnings("unused")
2020-12-07 22:15:18 +01:00
private void scheduledQueryRefresh() {
try {
boolean refreshStarted = searcherManager.maybeRefresh();
// if refreshStarted == false, another thread is currently already refreshing
2020-12-07 22:15:18 +01:00
} catch (IOException ex) {
ex.printStackTrace();
}
}
private LuceneIndexSnapshot resolveSnapshot(@Nullable LLSnapshot snapshot) {
if (snapshot == null) {
return null;
}
return Objects.requireNonNull(snapshots.get(snapshot.getSequenceNumber()),
() -> "Can't resolve snapshot " + snapshot.getSequenceNumber()
);
}
@Override
public boolean isLowMemoryMode() {
return lowMemory;
}
}