2020-12-07 22:15:18 +01:00
|
|
|
package it.cavallium.dbengine.database.disk;
|
|
|
|
|
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;
|
2020-12-31 12:04:53 +01:00
|
|
|
import it.cavallium.dbengine.database.LLDocument;
|
|
|
|
import it.cavallium.dbengine.database.LLKeyScore;
|
|
|
|
import it.cavallium.dbengine.database.LLLuceneIndex;
|
2021-02-04 22:42:57 +01:00
|
|
|
import it.cavallium.dbengine.database.LLSearchCollectionStatisticsGetter;
|
2021-01-29 17:19:01 +01:00
|
|
|
import it.cavallium.dbengine.database.LLSearchResult;
|
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-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-02-04 22:42:57 +01:00
|
|
|
import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer;
|
|
|
|
import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity;
|
2021-01-30 22:14:48 +01:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.AdaptiveStreamSearcher;
|
2021-02-04 22:42:57 +01:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.AllowOnlyQueryParsingCollectorStreamSearcher;
|
2021-04-01 19:48:25 +02:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LuceneSearchInstance;
|
2021-01-30 22:14:48 +01:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher;
|
2021-03-03 00:13:57 +01:00
|
|
|
import it.cavallium.dbengine.lucene.searcher.LuceneStreamSearcher.HandleResult;
|
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;
|
|
|
|
import java.util.Objects;
|
2021-01-29 17:19:01 +01:00
|
|
|
import java.util.Optional;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Set;
|
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
2021-04-15 00:00:42 +02:00
|
|
|
import java.util.concurrent.Executors;
|
2021-03-03 15:03:25 +01:00
|
|
|
import java.util.concurrent.Semaphore;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.concurrent.TimeUnit;
|
2021-03-03 15:03:25 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
import org.apache.lucene.index.IndexCommit;
|
|
|
|
import org.apache.lucene.index.IndexWriter;
|
|
|
|
import org.apache.lucene.index.IndexWriterConfig;
|
|
|
|
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
|
|
|
|
import org.apache.lucene.index.SnapshotDeletionPolicy;
|
|
|
|
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;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.apache.lucene.search.IndexSearcher;
|
|
|
|
import org.apache.lucene.search.Query;
|
2021-01-29 17:19:01 +01:00
|
|
|
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;
|
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;
|
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;
|
|
|
|
import org.apache.lucene.store.RAMDirectory;
|
|
|
|
import org.apache.solr.core.RAMDirectoryFactory;
|
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;
|
2021-04-15 00:00:42 +02:00
|
|
|
import org.warp.commonutils.type.ShortNamedThreadFactory;
|
2020-12-07 22:15:18 +01:00
|
|
|
import reactor.core.publisher.Flux;
|
2021-03-03 21:32:45 +01:00
|
|
|
import reactor.core.publisher.FluxSink.OverflowStrategy;
|
2021-01-30 19:57:50 +01:00
|
|
|
import reactor.core.publisher.GroupedFlux;
|
2020-12-07 22:15:18 +01:00
|
|
|
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
|
|
|
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);
|
2020-12-07 22:15:18 +01:00
|
|
|
private static final LuceneStreamSearcher streamSearcher = new AdaptiveStreamSearcher();
|
2021-04-03 02:20:37 +02:00
|
|
|
private static final AllowOnlyQueryParsingCollectorStreamSearcher allowOnlyQueryParsingCollectorStreamSearcher
|
|
|
|
= new AllowOnlyQueryParsingCollectorStreamSearcher();
|
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-02-03 13:48:30 +01:00
|
|
|
true
|
|
|
|
);
|
2021-04-15 00:00:42 +02:00
|
|
|
// Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks
|
|
|
|
private static final Scheduler luceneSearcherScheduler = Schedulers
|
|
|
|
.fromExecutorService(Executors
|
|
|
|
.newCachedThreadPool(new ShortNamedThreadFactory("lucene-searcher")));
|
2020-12-12 23:41:09 +01:00
|
|
|
|
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-02-04 22:42:57 +01:00
|
|
|
private final TextFieldsSimilarity similarity;
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2020-12-12 23:41:09 +01:00
|
|
|
private final ScheduledTaskLifecycle scheduledTasksLifecycle;
|
2021-02-04 22:42:57 +01:00
|
|
|
private final @Nullable LLSearchCollectionStatisticsGetter distributedCollectionStatisticsGetter;
|
2020-12-12 23:41:09 +01:00
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
public LLLocalLuceneIndex(Path luceneBasePath,
|
|
|
|
String name,
|
|
|
|
TextFieldsAnalyzer analyzer,
|
2021-02-04 22:42:57 +01:00
|
|
|
TextFieldsSimilarity similarity,
|
2020-12-07 22:15:18 +01:00
|
|
|
Duration queryRefreshDebounceTime,
|
|
|
|
Duration commitDebounceTime,
|
2021-04-30 19:15:04 +02:00
|
|
|
boolean lowMemory, boolean inMemory, @Nullable LLSearchCollectionStatisticsGetter distributedCollectionStatisticsGetter) 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-04-30 19:15:04 +02:00
|
|
|
this.directory = inMemory ? new RAMDirectory() : FSDirectory.open(directoryPath);
|
2020-12-07 22:15:18 +01:00
|
|
|
this.luceneIndexName = name;
|
|
|
|
this.snapshotter = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
|
|
|
|
this.lowMemory = lowMemory;
|
2021-02-04 22:42:57 +01:00
|
|
|
this.similarity = similarity;
|
|
|
|
this.distributedCollectionStatisticsGetter = distributedCollectionStatisticsGetter;
|
2020-12-07 22:15:18 +01:00
|
|
|
IndexWriterConfig indexWriterConfig = new IndexWriterConfig(LuceneUtils.getAnalyzer(analyzer));
|
|
|
|
indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
|
|
|
|
indexWriterConfig.setIndexDeletionPolicy(snapshotter);
|
|
|
|
indexWriterConfig.setCommitOnClose(true);
|
|
|
|
if (lowMemory) {
|
|
|
|
indexWriterConfig.setRAMBufferSizeMB(32);
|
|
|
|
indexWriterConfig.setRAMPerThreadHardLimitMB(32);
|
|
|
|
} else {
|
|
|
|
indexWriterConfig.setRAMBufferSizeMB(128);
|
2021-03-20 12:41:11 +01:00
|
|
|
//indexWriterConfig.setRAMPerThreadHardLimitMB(512);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
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-04-03 02:20:37 +02:00
|
|
|
this.searcherManager
|
|
|
|
= new SearcherManager(indexWriter, false, false, null);
|
2020-12-12 23:41:09 +01:00
|
|
|
|
|
|
|
// Create scheduled tasks lifecycle manager
|
|
|
|
this.scheduledTasksLifecycle = new ScheduledTaskLifecycle();
|
|
|
|
|
|
|
|
// Start scheduled tasks
|
|
|
|
registerScheduledFixedTask(this::scheduledCommit, commitDebounceTime);
|
|
|
|
registerScheduledFixedTask(this::scheduledQueryRefresh, queryRefreshDebounceTime);
|
|
|
|
}
|
|
|
|
|
2021-02-04 22:42:57 +01:00
|
|
|
private Similarity getSimilarity() {
|
|
|
|
return LuceneUtils.getSimilarity(similarity);
|
|
|
|
}
|
|
|
|
|
2020-12-12 23:41:09 +01:00
|
|
|
private void registerScheduledFixedTask(Runnable task, Duration duration) {
|
2021-03-03 10:57:45 +01:00
|
|
|
scheduledTasksLifecycle.registerScheduledTask(luceneHeavyTasksScheduler.schedulePeriodically(() -> {
|
2020-12-31 12:05:04 +01:00
|
|
|
scheduledTasksLifecycle.startScheduledTask();
|
|
|
|
try {
|
|
|
|
task.run();
|
|
|
|
} finally {
|
|
|
|
scheduledTasksLifecycle.endScheduledTask();
|
|
|
|
}
|
|
|
|
}, duration.toMillis(), duration.toMillis(), TimeUnit.MILLISECONDS));
|
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-03-03 10:57:45 +01:00
|
|
|
return takeLuceneSnapshot()
|
2021-01-30 01:41:04 +01:00
|
|
|
.flatMap(snapshot -> Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-03 10:57:45 +01:00
|
|
|
var snapshotSeqNo = lastSnapshotSeqNo.incrementAndGet();
|
2021-01-30 01:41:04 +01:00
|
|
|
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.
|
|
|
|
*/
|
2021-01-30 01:41:04 +01:00
|
|
|
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(() -> {
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
indexWriter.commit();
|
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
|
|
|
return snapshotter.snapshot();
|
|
|
|
}).subscribeOn(luceneHeavyTasksScheduler);
|
|
|
|
} else {
|
|
|
|
return Mono.error(ex);
|
|
|
|
}
|
|
|
|
})
|
|
|
|
);
|
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.<Void>fromCallable(() -> {
|
|
|
|
var indexSnapshot = this.snapshots.remove(snapshot.getSequenceNumber());
|
|
|
|
if (indexSnapshot == null) {
|
2021-03-02 01:53:36 +01:00
|
|
|
throw new IOException("LLSnapshot " + snapshot.getSequenceNumber() + " not found!");
|
2021-01-30 01:41:04 +01:00
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-01-30 01:41:04 +01:00
|
|
|
indexSnapshot.close();
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-01-30 01:41:04 +01:00
|
|
|
var luceneIndexSnapshot = indexSnapshot.getSnapshot();
|
|
|
|
snapshotter.release(luceneIndexSnapshot);
|
|
|
|
// Delete unused files after releasing the snapshot
|
|
|
|
indexWriter.deleteUnusedFiles();
|
|
|
|
return null;
|
2021-04-15 00:00:42 +02:00
|
|
|
}).subscribeOn(Schedulers.boundedElastic());
|
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) {
|
|
|
|
return Mono.<Void>fromCallable(() -> {
|
|
|
|
indexWriter.addDocument(LLUtils.toDocument(doc));
|
|
|
|
return null;
|
2021-04-15 00:00:42 +02:00
|
|
|
}).subscribeOn(Schedulers.boundedElastic());
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 19:57:50 +01:00
|
|
|
public Mono<Void> addDocuments(Flux<GroupedFlux<LLTerm, LLDocument>> documents) {
|
|
|
|
return documents
|
|
|
|
.flatMap(group -> group
|
|
|
|
.collectList()
|
|
|
|
.flatMap(docs -> Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
|
|
|
indexWriter.addDocuments(LLUtils.toDocuments(docs));
|
|
|
|
return null;
|
|
|
|
})
|
2021-04-15 00:00:42 +02:00
|
|
|
.subscribeOn(Schedulers.boundedElastic()))
|
2021-01-30 19:57:50 +01:00
|
|
|
)
|
|
|
|
.then();
|
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) {
|
|
|
|
return Mono.<Void>fromCallable(() -> {
|
|
|
|
indexWriter.deleteDocuments(LLUtils.toTerm(id));
|
|
|
|
return null;
|
2021-04-15 00:00:42 +02:00
|
|
|
}).subscribeOn(Schedulers.boundedElastic());
|
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 Mono.<Void>fromCallable(() -> {
|
|
|
|
indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document));
|
|
|
|
return null;
|
2021-04-15 00:00:42 +02:00
|
|
|
}).subscribeOn(Schedulers.boundedElastic());
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 19:57:50 +01:00
|
|
|
public Mono<Void> updateDocuments(Flux<GroupedFlux<LLTerm, LLDocument>> documents) {
|
|
|
|
return documents.flatMap(this::updateDocuments).then();
|
|
|
|
}
|
2021-01-30 01:41:04 +01:00
|
|
|
|
2021-01-30 19:57:50 +01:00
|
|
|
private Mono<Void> updateDocuments(GroupedFlux<LLTerm, LLDocument> documents) {
|
|
|
|
return documents
|
|
|
|
.map(LLUtils::toDocument)
|
|
|
|
.collectList()
|
|
|
|
.flatMap(luceneDocuments -> Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
|
|
|
indexWriter.updateDocuments(LLUtils.toTerm(documents.key()), luceneDocuments);
|
|
|
|
return null;
|
|
|
|
})
|
2021-04-15 00:00:42 +02:00
|
|
|
.subscribeOn(Schedulers.boundedElastic())
|
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> deleteAll() {
|
|
|
|
return Mono.<Void>fromCallable(() -> {
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-01-30 01:41:04 +01:00
|
|
|
indexWriter.deleteAll();
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-01-30 01:41:04 +01:00
|
|
|
indexWriter.forceMergeDeletes(true);
|
2021-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-01-30 01:41:04 +01:00
|
|
|
indexWriter.commit();
|
|
|
|
return null;
|
2021-03-03 10:57:45 +01:00
|
|
|
}).subscribeOn(luceneHeavyTasksScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-04 22:42:57 +01:00
|
|
|
private Mono<IndexSearcher> acquireSearcherWrapper(LLSnapshot snapshot, boolean distributedPre, long actionId) {
|
2021-01-29 17:19:01 +01:00
|
|
|
return Mono.fromCallable(() -> {
|
2021-02-04 22:42:57 +01:00
|
|
|
IndexSearcher indexSearcher;
|
2021-01-29 17:19:01 +01:00
|
|
|
if (snapshot == null) {
|
2021-02-04 22:42:57 +01:00
|
|
|
indexSearcher = searcherManager.acquire();
|
|
|
|
indexSearcher.setSimilarity(getSimilarity());
|
|
|
|
} else {
|
|
|
|
indexSearcher = resolveSnapshot(snapshot).getIndexSearcher();
|
|
|
|
}
|
|
|
|
if (distributedCollectionStatisticsGetter != null && actionId != -1) {
|
|
|
|
return new LLIndexSearcherWithCustomCollectionStatistics(indexSearcher,
|
|
|
|
distributedCollectionStatisticsGetter,
|
|
|
|
distributedPre,
|
|
|
|
actionId
|
|
|
|
);
|
2021-01-29 17:19:01 +01:00
|
|
|
} else {
|
2021-02-04 22:42:57 +01:00
|
|
|
return indexSearcher;
|
2021-01-29 17:19:01 +01:00
|
|
|
}
|
2021-04-15 00:00:42 +02:00
|
|
|
}).subscribeOn(Schedulers.boundedElastic());
|
2021-01-29 17:19:01 +01:00
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-01-29 17:19:01 +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
|
2021-01-29 17:19:01 +01:00
|
|
|
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux) {
|
2021-02-28 16:50:59 +01:00
|
|
|
return moreLikeThis(snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
queryParams,
|
2021-02-28 16:50:59 +01:00
|
|
|
keyFieldName,
|
2021-03-02 01:53:36 +01:00
|
|
|
mltDocumentFieldsFlux,
|
2021-02-28 16:50:59 +01:00
|
|
|
false,
|
|
|
|
0,
|
|
|
|
1
|
|
|
|
);
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
public Mono<LLSearchResult> distributedMoreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
2021-02-04 22:42:57 +01:00
|
|
|
String keyFieldName,
|
2021-03-02 01:53:36 +01:00
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
2021-02-04 22:42:57 +01:00
|
|
|
long actionId,
|
|
|
|
int scoreDivisor) {
|
2021-02-28 16:50:59 +01:00
|
|
|
return moreLikeThis(snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
queryParams,
|
2021-02-28 16:50:59 +01:00
|
|
|
keyFieldName,
|
2021-03-02 01:53:36 +01:00
|
|
|
mltDocumentFieldsFlux,
|
2021-02-28 16:50:59 +01:00
|
|
|
false,
|
|
|
|
actionId,
|
|
|
|
scoreDivisor
|
|
|
|
);
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
public Mono<Void> distributedPreMoreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
2021-02-04 22:42:57 +01:00
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
2021-03-02 01:53:36 +01:00
|
|
|
long actionId) {
|
2021-03-27 03:35:27 +01:00
|
|
|
return moreLikeThis(snapshot, queryParams, keyFieldName, mltDocumentFieldsFlux, true, actionId, 1).then();
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
2021-02-27 17:32:57 +01:00
|
|
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
2021-02-04 22:42:57 +01:00
|
|
|
private Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams,
|
2021-02-04 22:42:57 +01:00
|
|
|
String keyFieldName,
|
2021-03-02 01:53:36 +01:00
|
|
|
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
2021-02-04 22:42:57 +01:00
|
|
|
boolean doDistributedPre,
|
|
|
|
long actionId,
|
|
|
|
int scoreDivisor) {
|
2021-02-27 17:32:57 +01:00
|
|
|
Query luceneAdditionalQuery;
|
|
|
|
try {
|
2021-03-02 01:53:36 +01:00
|
|
|
luceneAdditionalQuery = QueryParser.toQuery(queryParams.getQuery());
|
|
|
|
} catch (Exception e) {
|
2021-02-27 17:32:57 +01:00
|
|
|
return Mono.error(e);
|
|
|
|
}
|
2021-01-30 19:57:50 +01:00
|
|
|
return mltDocumentFieldsFlux
|
|
|
|
.collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new)
|
|
|
|
.flatMap(mltDocumentFields -> {
|
2021-02-27 19:05:13 +01:00
|
|
|
mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty());
|
2021-01-30 19:57:50 +01:00
|
|
|
if (mltDocumentFields.isEmpty()) {
|
2021-03-27 03:35:27 +01:00
|
|
|
return Mono.just(new LLSearchResult(Flux.empty()));
|
2021-01-30 19:57:50 +01:00
|
|
|
}
|
|
|
|
|
2021-04-14 02:37:03 +02:00
|
|
|
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 doc 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;
|
|
|
|
})
|
2021-04-15 00:00:42 +02:00
|
|
|
.subscribeOn(Schedulers.boundedElastic())
|
2021-04-14 02:37:03 +02:00
|
|
|
.map(luceneQuery -> luceneSearch(doDistributedPre,
|
|
|
|
indexSearcher,
|
|
|
|
queryParams.getOffset(),
|
|
|
|
queryParams.getLimit(),
|
|
|
|
queryParams.getMinCompetitiveScore().getNullable(),
|
|
|
|
keyFieldName,
|
|
|
|
scoreDivisor,
|
|
|
|
luceneQuery,
|
|
|
|
QueryParser.toSort(queryParams.getSort()),
|
|
|
|
QueryParser.toScoreMode(queryParams.getScoreMode()),
|
|
|
|
releaseSearcherWrapper(snapshot, indexSearcher)
|
|
|
|
))
|
|
|
|
.onErrorResume(ex -> releaseSearcherWrapper(snapshot, indexSearcher).then(Mono.error(ex)))
|
|
|
|
);
|
2021-01-30 19:57:50 +01:00
|
|
|
});
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-04 22:42:57 +01:00
|
|
|
private LLKeyScore fixKeyScore(LLKeyScore keyScore, int scoreDivisor) {
|
|
|
|
return scoreDivisor == 1 ? keyScore : new LLKeyScore(keyScore.getKey(), keyScore.getScore() / (float) scoreDivisor);
|
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-03-02 01:53:36 +01:00
|
|
|
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) {
|
|
|
|
return search(snapshot, queryParams, keyFieldName, false, 0, 1);
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
2021-01-29 17:19:01 +01:00
|
|
|
|
2021-04-03 02:20:37 +02:00
|
|
|
public Mono<LLSearchResult> distributedSearch(@Nullable LLSnapshot snapshot,
|
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
|
|
|
long actionId,
|
|
|
|
int scoreDivisor) {
|
2021-03-02 01:53:36 +01:00
|
|
|
return search(snapshot, queryParams, keyFieldName, false, actionId, scoreDivisor);
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
2021-04-03 02:20:37 +02:00
|
|
|
public Mono<Void> distributedPreSearch(@Nullable LLSnapshot snapshot,
|
|
|
|
QueryParams queryParams,
|
|
|
|
String keyFieldName,
|
|
|
|
long actionId) {
|
2021-02-14 13:46:11 +01:00
|
|
|
return this
|
2021-03-02 01:53:36 +01:00
|
|
|
.search(snapshot, queryParams, keyFieldName, true, actionId, 1)
|
2021-03-27 03:35:27 +01:00
|
|
|
.then();
|
2021-02-04 22:42:57 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
|
2021-03-02 01:53:36 +01:00
|
|
|
QueryParams queryParams, String keyFieldName,
|
2021-02-04 22:42:57 +01:00
|
|
|
boolean doDistributedPre, long actionId, int scoreDivisor) {
|
2021-04-14 02:37:03 +02:00
|
|
|
return this
|
|
|
|
.acquireSearcherWrapper(snapshot, doDistributedPre, actionId)
|
2021-01-29 17:19:01 +01:00
|
|
|
.flatMap(indexSearcher -> Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-02 01:53:36 +01:00
|
|
|
Objects.requireNonNull(queryParams.getScoreMode(), "ScoreMode must not be null");
|
|
|
|
Query luceneQuery = QueryParser.toQuery(queryParams.getQuery());
|
|
|
|
Sort luceneSort = QueryParser.toSort(queryParams.getSort());
|
|
|
|
org.apache.lucene.search.ScoreMode luceneScoreMode = QueryParser.toScoreMode(queryParams.getScoreMode());
|
2021-02-03 13:48:30 +01:00
|
|
|
return Tuples.of(luceneQuery, Optional.ofNullable(luceneSort), luceneScoreMode);
|
2021-01-29 17:19:01 +01:00
|
|
|
})
|
2021-04-15 00:00:42 +02:00
|
|
|
.subscribeOn(Schedulers.boundedElastic())
|
2021-04-14 02:37:03 +02:00
|
|
|
.<LLSearchResult>flatMap(tuple -> Mono
|
2021-03-03 20:00:58 +01:00
|
|
|
.fromSupplier(() -> {
|
2021-02-03 13:48:30 +01:00
|
|
|
Query luceneQuery = tuple.getT1();
|
2021-01-29 17:19:01 +01:00
|
|
|
Sort luceneSort = tuple.getT2().orElse(null);
|
|
|
|
ScoreMode luceneScoreMode = tuple.getT3();
|
|
|
|
|
2021-02-27 19:05:13 +01:00
|
|
|
return luceneSearch(doDistributedPre,
|
|
|
|
indexSearcher,
|
2021-04-01 19:48:25 +02:00
|
|
|
queryParams.getOffset(),
|
2021-03-02 01:53:36 +01:00
|
|
|
queryParams.getLimit(),
|
|
|
|
queryParams.getMinCompetitiveScore().getNullable(),
|
2021-02-27 19:05:13 +01:00
|
|
|
keyFieldName,
|
|
|
|
scoreDivisor,
|
|
|
|
luceneQuery,
|
|
|
|
luceneSort,
|
2021-04-14 02:37:03 +02:00
|
|
|
luceneScoreMode,
|
|
|
|
releaseSearcherWrapper(snapshot, indexSearcher)
|
2021-02-25 00:00:16 +01:00
|
|
|
);
|
2021-03-03 20:00:58 +01:00
|
|
|
})
|
2021-04-14 02:37:03 +02:00
|
|
|
.onErrorResume(ex -> releaseSearcherWrapper(snapshot, indexSearcher).then(Mono.error(ex)))
|
2021-01-29 17:19:01 +01:00
|
|
|
)
|
|
|
|
);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-04-14 02:37:03 +02:00
|
|
|
/**
|
|
|
|
* This method always returns 1 shard! Not zero, not more than one.
|
|
|
|
*/
|
2021-02-27 19:05:13 +01:00
|
|
|
private LLSearchResult luceneSearch(boolean doDistributedPre,
|
|
|
|
IndexSearcher indexSearcher,
|
2021-04-01 19:48:25 +02:00
|
|
|
long offset,
|
2021-02-27 19:05:13 +01:00
|
|
|
long limit,
|
|
|
|
@Nullable Float minCompetitiveScore,
|
|
|
|
String keyFieldName,
|
|
|
|
int scoreDivisor,
|
|
|
|
Query luceneQuery,
|
|
|
|
Sort luceneSort,
|
2021-04-14 02:37:03 +02:00
|
|
|
ScoreMode luceneScoreMode,
|
|
|
|
Mono<Void> successCleanup) {
|
2021-03-27 03:35:27 +01:00
|
|
|
return new LLSearchResult(Mono.<LLSearchResultShard>create(monoSink -> {
|
2021-04-15 00:00:42 +02:00
|
|
|
LuceneSearchInstance luceneSearchInstance;
|
|
|
|
long totalHitsCount;
|
|
|
|
try {
|
|
|
|
if (doDistributedPre) {
|
|
|
|
allowOnlyQueryParsingCollectorStreamSearcher.search(indexSearcher, luceneQuery);
|
|
|
|
monoSink.success(new LLSearchResultShard(successCleanup.thenMany(Flux.empty()), 0));
|
|
|
|
return;
|
|
|
|
} 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);
|
|
|
|
luceneSearchInstance = streamSearcher.search(indexSearcher,
|
|
|
|
luceneQuery,
|
|
|
|
boundedOffset,
|
|
|
|
boundedLimit,
|
|
|
|
luceneSort,
|
|
|
|
luceneScoreMode,
|
|
|
|
minCompetitiveScore,
|
|
|
|
keyFieldName
|
|
|
|
);
|
|
|
|
totalHitsCount = luceneSearchInstance.getTotalHitsCount();
|
|
|
|
}
|
|
|
|
} catch (Exception ex) {
|
|
|
|
monoSink.error(ex);
|
2021-04-01 19:48:25 +02:00
|
|
|
return;
|
2021-03-27 03:35:27 +01:00
|
|
|
}
|
|
|
|
|
2021-04-15 00:00:42 +02:00
|
|
|
AtomicBoolean alreadySubscribed = new AtomicBoolean(false);
|
|
|
|
var resultsFlux = Flux.<LLKeyScore>create(sink -> {
|
2021-03-27 03:35:27 +01:00
|
|
|
|
2021-04-15 00:00:42 +02:00
|
|
|
if (!alreadySubscribed.compareAndSet(false, true)) {
|
|
|
|
sink.error(new IllegalStateException("Already subscribed to results"));
|
|
|
|
return;
|
|
|
|
}
|
2021-03-27 03:35:27 +01:00
|
|
|
|
2021-04-15 00:00:42 +02:00
|
|
|
AtomicBoolean cancelled = new AtomicBoolean();
|
|
|
|
Semaphore requests = new Semaphore(0);
|
|
|
|
sink.onDispose(() -> cancelled.set(true));
|
|
|
|
sink.onCancel(() -> cancelled.set(true));
|
|
|
|
sink.onRequest(delta -> requests.release((int) Math.min(delta, Integer.MAX_VALUE)));
|
|
|
|
|
|
|
|
luceneSearcherScheduler
|
|
|
|
.schedule(() -> {
|
|
|
|
try {
|
|
|
|
luceneSearchInstance.getResults(keyScore -> {
|
|
|
|
try {
|
|
|
|
if (cancelled.get()) {
|
|
|
|
return HandleResult.HALT;
|
|
|
|
}
|
|
|
|
while (!requests.tryAcquire(500, TimeUnit.MILLISECONDS)) {
|
|
|
|
if (cancelled.get()) {
|
|
|
|
return HandleResult.HALT;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
sink.next(fixKeyScore(keyScore, scoreDivisor));
|
|
|
|
if (cancelled.get()) {
|
|
|
|
return HandleResult.HALT;
|
|
|
|
} else {
|
|
|
|
return HandleResult.CONTINUE;
|
|
|
|
}
|
|
|
|
} catch (Exception ex) {
|
|
|
|
sink.error(ex);
|
|
|
|
cancelled.set(true);
|
|
|
|
requests.release(Integer.MAX_VALUE);
|
|
|
|
return HandleResult.HALT;
|
|
|
|
}
|
|
|
|
});
|
|
|
|
sink.complete();
|
|
|
|
} catch (Exception ex) {
|
|
|
|
sink.error(ex);
|
2021-04-01 19:48:25 +02:00
|
|
|
}
|
2021-04-15 00:00:42 +02:00
|
|
|
});
|
2021-03-27 03:35:27 +01:00
|
|
|
|
2021-04-15 00:00:42 +02:00
|
|
|
}, OverflowStrategy.ERROR).subscribeOn(Schedulers.boundedElastic());
|
2021-03-27 03:35:27 +01:00
|
|
|
|
2021-04-15 00:00:42 +02:00
|
|
|
monoSink.success(new LLSearchResultShard(Flux
|
|
|
|
.usingWhen(
|
|
|
|
Mono.just(true),
|
|
|
|
b -> resultsFlux,
|
|
|
|
b -> successCleanup),
|
|
|
|
totalHitsCount));
|
|
|
|
}).subscribeOn(Schedulers.boundedElastic()).flux());
|
2021-02-27 19:05:13 +01:00
|
|
|
}
|
|
|
|
|
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-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-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-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-02-03 14:37:02 +01:00
|
|
|
//noinspection BlockingMethodInNonBlockingContext
|
2021-02-03 13:48:30 +01:00
|
|
|
searcherManager.maybeRefreshBlocking();
|
|
|
|
} 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();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-29 17:19:01 +01:00
|
|
|
@SuppressWarnings("unused")
|
2020-12-07 22:15:18 +01:00
|
|
|
private void scheduledQueryRefresh() {
|
|
|
|
try {
|
2021-01-29 17:19:01 +01:00
|
|
|
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;
|
|
|
|
}
|
2021-04-01 19:48:25 +02:00
|
|
|
|
|
|
|
@Override
|
|
|
|
public boolean supportsOffset() {
|
|
|
|
return true;
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|