This commit is contained in:
Andrea Cavalli 2021-09-07 11:26:10 +02:00
parent 66fa853272
commit 717d13ef1f
16 changed files with 154 additions and 68 deletions

View File

@ -143,6 +143,7 @@ public class LLLocalDictionary implements LLDictionary {
private final ColumnFamilyHandle cfh; private final ColumnFamilyHandle cfh;
private final String databaseName; private final String databaseName;
private final String columnName; private final String columnName;
private final Scheduler dbScheduler;
private final Function<LLSnapshot, Snapshot> snapshotResolver; private final Function<LLSnapshot, Snapshot> snapshotResolver;
private final Striped<StampedLock> itemsLock = Striped.readWriteStampedLock(STRIPES); private final Striped<StampedLock> itemsLock = Striped.readWriteStampedLock(STRIPES);
private final UpdateMode updateMode; private final UpdateMode updateMode;
@ -157,6 +158,7 @@ public class LLLocalDictionary implements LLDictionary {
@NotNull ColumnFamilyHandle columnFamilyHandle, @NotNull ColumnFamilyHandle columnFamilyHandle,
String databaseName, String databaseName,
String columnName, String columnName,
Scheduler dbScheduler,
Function<LLSnapshot, Snapshot> snapshotResolver, Function<LLSnapshot, Snapshot> snapshotResolver,
UpdateMode updateMode, UpdateMode updateMode,
DatabaseOptions databaseOptions) { DatabaseOptions databaseOptions) {
@ -166,6 +168,7 @@ public class LLLocalDictionary implements LLDictionary {
this.cfh = columnFamilyHandle; this.cfh = columnFamilyHandle;
this.databaseName = databaseName; this.databaseName = databaseName;
this.columnName = columnName; this.columnName = columnName;
this.dbScheduler = dbScheduler;
this.snapshotResolver = snapshotResolver; this.snapshotResolver = snapshotResolver;
this.updateMode = updateMode; this.updateMode = updateMode;
this.getRangeMultiDebugName = databaseName + "(" + columnName + ")" + "::getRangeMulti"; this.getRangeMultiDebugName = databaseName + "(" + columnName + ")" + "::getRangeMulti";
@ -241,7 +244,7 @@ public class LLLocalDictionary implements LLDictionary {
} }
private <T> @NotNull Mono<T> runOnDb(Callable<@Nullable T> callable) { private <T> @NotNull Mono<T> runOnDb(Callable<@Nullable T> callable) {
return Mono.fromCallable(callable); return Mono.fromCallable(callable).subscribeOn(dbScheduler);
} }
@Override @Override
@ -1420,7 +1423,7 @@ public class LLLocalDictionary implements LLDictionary {
rangeSend -> Flux.using( rangeSend -> Flux.using(
() -> new LLLocalEntryReactiveRocksIterator(db, alloc, cfh, rangeSend, () -> new LLLocalEntryReactiveRocksIterator(db, alloc, cfh, rangeSend,
databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), getRangeMultiDebugName), databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), getRangeMultiDebugName),
LLLocalReactiveRocksIterator::flux, llLocalEntryReactiveRocksIterator -> llLocalEntryReactiveRocksIterator.flux().subscribeOn(dbScheduler),
LLLocalReactiveRocksIterator::release LLLocalReactiveRocksIterator::release
).transform(LLUtils::handleDiscard), ).transform(LLUtils::handleDiscard),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
@ -1432,7 +1435,7 @@ public class LLLocalDictionary implements LLDictionary {
rangeSend -> Flux.using( rangeSend -> Flux.using(
() -> new LLLocalGroupedEntryReactiveRocksIterator(db, alloc, cfh, prefixLength, rangeSend, () -> new LLLocalGroupedEntryReactiveRocksIterator(db, alloc, cfh, prefixLength, rangeSend,
databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), "getRangeMultiGrouped"), databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), "getRangeMultiGrouped"),
LLLocalGroupedReactiveRocksIterator::flux, reactiveRocksIterator -> reactiveRocksIterator.flux().subscribeOn(dbScheduler),
LLLocalGroupedReactiveRocksIterator::release LLLocalGroupedReactiveRocksIterator::release
).transform(LLUtils::handleDiscard), ).transform(LLUtils::handleDiscard),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
@ -1463,7 +1466,7 @@ public class LLLocalDictionary implements LLDictionary {
rangeSend -> Flux.using( rangeSend -> Flux.using(
() -> new LLLocalGroupedKeyReactiveRocksIterator(db, alloc, cfh, prefixLength, rangeSend, () -> new LLLocalGroupedKeyReactiveRocksIterator(db, alloc, cfh, prefixLength, rangeSend,
databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), "getRangeKeysGrouped"), databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), "getRangeKeysGrouped"),
LLLocalGroupedReactiveRocksIterator::flux, reactiveRocksIterator -> reactiveRocksIterator.flux().subscribeOn(dbScheduler),
LLLocalGroupedReactiveRocksIterator::release LLLocalGroupedReactiveRocksIterator::release
).transform(LLUtils::handleDiscard), ).transform(LLUtils::handleDiscard),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
@ -1474,7 +1477,7 @@ public class LLLocalDictionary implements LLDictionary {
public Flux<BadBlock> badBlocks(Mono<Send<LLRange>> rangeMono) { public Flux<BadBlock> badBlocks(Mono<Send<LLRange>> rangeMono) {
return Flux.usingWhen(rangeMono, return Flux.usingWhen(rangeMono,
rangeSend -> Flux rangeSend -> Flux
.create(sink -> { .<BadBlock>create(sink -> {
var range = rangeSend.receive(); var range = rangeSend.receive();
sink.onDispose(range::close); sink.onDispose(range::close);
try (var ro = new ReadOptions(getReadOptions(null))) { try (var ro = new ReadOptions(getReadOptions(null))) {
@ -1511,7 +1514,8 @@ public class LLLocalDictionary implements LLDictionary {
} catch (Throwable ex) { } catch (Throwable ex) {
sink.error(ex); sink.error(ex);
} }
}), })
.subscribeOn(dbScheduler),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
); );
} }
@ -1531,9 +1535,10 @@ public class LLLocalDictionary implements LLDictionary {
true, true,
"getRangeKeysGrouped" "getRangeKeysGrouped"
), ),
LLLocalKeyPrefixReactiveRocksIterator::flux, it -> it.flux(),
LLLocalKeyPrefixReactiveRocksIterator::release it -> it.release()
), )
.subscribeOn(dbScheduler),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
); );
} }
@ -1560,7 +1565,7 @@ public class LLLocalDictionary implements LLDictionary {
rangeSend -> Flux.using( rangeSend -> Flux.using(
() -> new LLLocalKeyReactiveRocksIterator(db, alloc, cfh, rangeSend, () -> new LLLocalKeyReactiveRocksIterator(db, alloc, cfh, rangeSend,
databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), getRangeKeysMultiDebugName), databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), getRangeKeysMultiDebugName),
LLLocalReactiveRocksIterator::flux, llLocalKeyReactiveRocksIterator -> llLocalKeyReactiveRocksIterator.flux().subscribeOn(dbScheduler),
LLLocalReactiveRocksIterator::release LLLocalReactiveRocksIterator::release
).transform(LLUtils::handleDiscard), ).transform(LLUtils::handleDiscard),
rangeSend -> Mono.fromRunnable(rangeSend::close) rangeSend -> Mono.fromRunnable(rangeSend::close)
@ -2001,7 +2006,8 @@ public class LLLocalDictionary implements LLDictionary {
return null; return null;
} }
}) })
.onErrorMap(cause -> new IOException("Failed to clear", cause)); .onErrorMap(cause -> new IOException("Failed to clear", cause))
.subscribeOn(dbScheduler);
} }

View File

@ -66,6 +66,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
RocksDB.DEFAULT_COLUMN_FAMILY); RocksDB.DEFAULT_COLUMN_FAMILY);
private final BufferAllocator allocator; private final BufferAllocator allocator;
private final Scheduler dbScheduler;
// Configurations // Configurations
@ -128,6 +129,12 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
// 8 or more // 8 or more
threadCap = Math.max(8, Runtime.getRuntime().availableProcessors()); threadCap = Math.max(8, Runtime.getRuntime().availableProcessors());
} }
this.dbScheduler = Schedulers.newBoundedElastic(threadCap,
Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
"db-" + name,
60,
true
);
this.enableColumnsBug = "true".equals(databaseOptions.extraFlags().getOrDefault("enableColumnBug", "false")); this.enableColumnsBug = "true".equals(databaseOptions.extraFlags().getOrDefault("enableColumnBug", "false"));
createIfNotExists(descriptors, rocksdbOptions, databaseOptions, dbPath, dbPathString); createIfNotExists(descriptors, rocksdbOptions, databaseOptions, dbPath, dbPathString);
@ -458,9 +465,11 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
(snapshot) -> snapshotsHandles.get(snapshot.getSequenceNumber()), (snapshot) -> snapshotsHandles.get(snapshot.getSequenceNumber()),
LLLocalKeyValueDatabase.this.name, LLLocalKeyValueDatabase.this.name,
name, name,
dbScheduler,
defaultValue defaultValue
)) ))
.onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(name), cause)); .onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(name), cause))
.subscribeOn(dbScheduler);
} }
@Override @Override
@ -472,10 +481,12 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
getCfh(columnName), getCfh(columnName),
name, name,
Column.toString(columnName), Column.toString(columnName),
dbScheduler,
(snapshot) -> snapshotsHandles.get(snapshot.getSequenceNumber()), (snapshot) -> snapshotsHandles.get(snapshot.getSequenceNumber()),
updateMode, updateMode,
databaseOptions databaseOptions
)); ))
.subscribeOn(dbScheduler);
} }
private ColumnFamilyHandle getCfh(byte[] columnName) throws RocksDBException { private ColumnFamilyHandle getCfh(byte[] columnName) throws RocksDBException {
@ -491,7 +502,8 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
@Override @Override
public Mono<Long> getProperty(String propertyName) { public Mono<Long> getProperty(String propertyName) {
return Mono.fromCallable(() -> db.getAggregatedLongProperty(propertyName)) return Mono.fromCallable(() -> db.getAggregatedLongProperty(propertyName))
.onErrorMap(cause -> new IOException("Failed to read " + propertyName, cause)); .onErrorMap(cause -> new IOException("Failed to read " + propertyName, cause))
.subscribeOn(dbScheduler);
} }
@Override @Override
@ -502,7 +514,8 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
return null; return null;
}) })
.onErrorMap(cause -> new IOException("Failed to verify checksum of database \"" .onErrorMap(cause -> new IOException("Failed to verify checksum of database \""
+ getDatabaseName() + "\"", cause)); + getDatabaseName() + "\"", cause))
.subscribeOn(dbScheduler);
} }
@Override @Override
@ -518,20 +531,22 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
long currentSnapshotSequenceNumber = nextSnapshotNumbers.getAndIncrement(); long currentSnapshotSequenceNumber = nextSnapshotNumbers.getAndIncrement();
this.snapshotsHandles.put(currentSnapshotSequenceNumber, snapshot); this.snapshotsHandles.put(currentSnapshotSequenceNumber, snapshot);
return new LLSnapshot(currentSnapshotSequenceNumber); return new LLSnapshot(currentSnapshotSequenceNumber);
}); })
.subscribeOn(dbScheduler);
} }
@Override @Override
public Mono<Void> releaseSnapshot(LLSnapshot snapshot) { public Mono<Void> releaseSnapshot(LLSnapshot snapshot) {
return Mono return Mono
.fromCallable(() -> { .<Void>fromCallable(() -> {
Snapshot dbSnapshot = this.snapshotsHandles.remove(snapshot.getSequenceNumber()); Snapshot dbSnapshot = this.snapshotsHandles.remove(snapshot.getSequenceNumber());
if (dbSnapshot == null) { if (dbSnapshot == null) {
throw new IOException("Snapshot " + snapshot.getSequenceNumber() + " not found!"); throw new IOException("Snapshot " + snapshot.getSequenceNumber() + " not found!");
} }
db.releaseSnapshot(dbSnapshot); db.releaseSnapshot(dbSnapshot);
return null; return null;
}); })
.subscribeOn(dbScheduler);
} }
@Override @Override
@ -546,7 +561,8 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
} }
return null; return null;
}) })
.onErrorMap(cause -> new IOException("Failed to close", cause)); .onErrorMap(cause -> new IOException("Failed to close", cause))
.subscribeOn(dbScheduler);
} }
/** /**

View File

@ -76,6 +76,16 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
Integer.MAX_VALUE, Integer.MAX_VALUE,
true true
); );
// Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks
protected static final Scheduler luceneSearcherScheduler = Schedulers.newBoundedElastic(
4,
Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
"lucene-searcher",
60,
true
);
// Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks
private static final Scheduler luceneWriterScheduler = Schedulers.boundedElastic();
private final String luceneIndexName; private final String luceneIndexName;
private final IndexWriter indexWriter; private final IndexWriter indexWriter;
@ -174,9 +184,11 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND); indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
indexWriterConfig.setIndexDeletionPolicy(snapshotter); indexWriterConfig.setIndexDeletionPolicy(snapshotter);
indexWriterConfig.setCommitOnClose(true); indexWriterConfig.setCommitOnClose(true);
int writerSchedulerMaxThreadCount;
MergeScheduler mergeScheduler; MergeScheduler mergeScheduler;
if (lowMemory) { if (lowMemory) {
mergeScheduler = new SerialMergeScheduler(); mergeScheduler = new SerialMergeScheduler();
writerSchedulerMaxThreadCount = 1;
} else { } else {
var concurrentMergeScheduler = new ConcurrentMergeScheduler(); var concurrentMergeScheduler = new ConcurrentMergeScheduler();
concurrentMergeScheduler.setDefaultMaxMergesAndThreads(false); concurrentMergeScheduler.setDefaultMaxMergesAndThreads(false);
@ -185,6 +197,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
} else { } else {
concurrentMergeScheduler.enableAutoIOThrottle(); concurrentMergeScheduler.enableAutoIOThrottle();
} }
writerSchedulerMaxThreadCount = concurrentMergeScheduler.getMaxThreadCount();
mergeScheduler = concurrentMergeScheduler; mergeScheduler = concurrentMergeScheduler;
} }
indexWriterConfig.setMergeScheduler(mergeScheduler); indexWriterConfig.setMergeScheduler(mergeScheduler);
@ -231,18 +244,20 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
activeTasks.register(); activeTasks.register();
try { try {
//noinspection BlockingMethodInNonBlockingContext
indexWriter.addDocument(LLUtils.toDocument(doc)); indexWriter.addDocument(LLUtils.toDocument(doc));
return null; return null;
} finally { } finally {
activeTasks.arriveAndDeregister(); activeTasks.arriveAndDeregister();
} }
}); }).subscribeOn(luceneWriterScheduler);
} }
@Override @Override
public Mono<Void> addDocuments(Flux<Entry<LLTerm, LLDocument>> documents) { public Mono<Void> addDocuments(Flux<Entry<LLTerm, LLDocument>> documents) {
return documents return documents
.collectList() .collectList()
.publishOn(luceneWriterScheduler)
.flatMap(documentsList -> Mono .flatMap(documentsList -> Mono
.fromCallable(() -> { .fromCallable(() -> {
activeTasks.register(); activeTasks.register();
@ -262,12 +277,13 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
activeTasks.register(); activeTasks.register();
try { try {
//noinspection BlockingMethodInNonBlockingContext
indexWriter.deleteDocuments(LLUtils.toTerm(id)); indexWriter.deleteDocuments(LLUtils.toTerm(id));
return null; return null;
} finally { } finally {
activeTasks.arriveAndDeregister(); activeTasks.arriveAndDeregister();
} }
}); }).subscribeOn(luceneWriterScheduler);
} }
@Override @Override
@ -275,12 +291,13 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
activeTasks.register(); activeTasks.register();
try { try {
//noinspection BlockingMethodInNonBlockingContext
indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document)); indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document));
} finally { } finally {
activeTasks.arriveAndDeregister(); activeTasks.arriveAndDeregister();
} }
return null; return null;
}); }).subscribeOn(luceneWriterScheduler);
} }
@Override @Override
@ -296,13 +313,15 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
for (Entry<LLTerm, LLDocument> entry : documentsMap.entrySet()) { for (Entry<LLTerm, LLDocument> entry : documentsMap.entrySet()) {
LLTerm key = entry.getKey(); LLTerm key = entry.getKey();
LLDocument value = entry.getValue(); LLDocument value = entry.getValue();
//noinspection BlockingMethodInNonBlockingContext
indexWriter.updateDocument(LLUtils.toTerm(key), LLUtils.toDocument(value)); indexWriter.updateDocument(LLUtils.toTerm(key), LLUtils.toDocument(value));
} }
return null; return null;
} finally { } finally {
activeTasks.arriveAndDeregister(); activeTasks.arriveAndDeregister();
} }
}); })
.subscribeOn(luceneWriterScheduler);
} }
@Override @Override
@ -333,7 +352,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
.flatMap(indexSearcher -> { .flatMap(indexSearcher -> {
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
return localSearcher return localSearcher
.collect(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, keyFieldName) .collect(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, keyFieldName, luceneSearcherScheduler)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())) .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()))
.onErrorResume(ex -> releaseMono.then(Mono.error(ex))); .onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
}) })
@ -349,7 +368,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
.flatMap(indexSearcher -> { .flatMap(indexSearcher -> {
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
return shardSearcher return shardSearcher
.searchOn(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery) .searchOn(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, luceneSearcherScheduler)
.onErrorResume(ex -> releaseMono.then(Mono.error(ex))); .onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
}) })
); );
@ -426,7 +445,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return searcherManager.captureIndexSearcher(snapshot).flatMap(indexSearcher -> { return searcherManager.captureIndexSearcher(snapshot).flatMap(indexSearcher -> {
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
return localSearcher return localSearcher
.collect(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, keyFieldName) .collect(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, keyFieldName, luceneSearcherScheduler)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())) .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()))
.onErrorResume(ex -> releaseMono.then(Mono.error(ex))); .onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
}); });
@ -439,7 +458,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return searcherManager.captureIndexSearcher(snapshot) return searcherManager.captureIndexSearcher(snapshot)
.flatMap(indexSearcher -> { .flatMap(indexSearcher -> {
Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); Mono<Void> releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher);
return shardSearcher.searchOn(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams) return shardSearcher.searchOn(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, luceneSearcherScheduler)
.onErrorResume(ex -> releaseMono.then(Mono.error(ex))); .onErrorResume(ex -> releaseMono.then(Mono.error(ex)));
}); });
} }

View File

@ -1,5 +1,7 @@
package it.cavallium.dbengine.database.disk; package it.cavallium.dbengine.database.disk;
import static it.cavallium.dbengine.database.disk.LLLocalLuceneIndex.luceneSearcherScheduler;
import com.google.common.cache.Cache; import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader.InvalidCacheLoadException; import com.google.common.cache.CacheLoader.InvalidCacheLoadException;
@ -218,7 +220,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
.flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex() .flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex()
.distributedMoreLikeThis(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, mltDocumentFields, shardSearcher)) .distributedMoreLikeThis(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, mltDocumentFields, shardSearcher))
// Collect all the shards results into a single global result // Collect all the shards results into a single global result
.then(shardSearcher.collect(localQueryParams, keyFieldName)) .then(shardSearcher.collect(localQueryParams, keyFieldName, luceneSearcherScheduler))
) )
// Fix the result type // Fix the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())); .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()));
@ -246,7 +248,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
.flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex() .flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex()
.distributedSearch(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, shardSearcher)) .distributedSearch(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, shardSearcher))
// Collect all the shards results into a single global result // Collect all the shards results into a single global result
.then(shardSearcher.collect(localQueryParams, keyFieldName)) .then(shardSearcher.collect(localQueryParams, keyFieldName, luceneSearcherScheduler))
) )
// Fix the result type // Fix the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())); .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release()));

View File

@ -23,17 +23,20 @@ public class LLLocalSingleton implements LLSingleton {
private final Function<LLSnapshot, Snapshot> snapshotResolver; private final Function<LLSnapshot, Snapshot> snapshotResolver;
private final byte[] name; private final byte[] name;
private final String databaseName; private final String databaseName;
private final Scheduler dbScheduler;
public LLLocalSingleton(RocksDB db, ColumnFamilyHandle singletonListColumn, public LLLocalSingleton(RocksDB db, ColumnFamilyHandle singletonListColumn,
Function<LLSnapshot, Snapshot> snapshotResolver, Function<LLSnapshot, Snapshot> snapshotResolver,
String databaseName, String databaseName,
byte[] name, byte[] name,
Scheduler dbScheduler,
byte[] defaultValue) throws RocksDBException { byte[] defaultValue) throws RocksDBException {
this.db = db; this.db = db;
this.cfh = singletonListColumn; this.cfh = singletonListColumn;
this.databaseName = databaseName; this.databaseName = databaseName;
this.snapshotResolver = snapshotResolver; this.snapshotResolver = snapshotResolver;
this.name = name; this.name = name;
this.dbScheduler = dbScheduler;
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Initialized in a nonblocking thread"); throw new UnsupportedOperationException("Initialized in a nonblocking thread");
} }
@ -59,7 +62,8 @@ public class LLLocalSingleton implements LLSingleton {
} }
return db.get(cfh, resolveSnapshot(snapshot), name); return db.get(cfh, resolveSnapshot(snapshot), name);
}) })
.onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(name), cause)); .onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(name), cause))
.subscribeOn(dbScheduler);
} }
@Override @Override
@ -72,7 +76,8 @@ public class LLLocalSingleton implements LLSingleton {
db.put(cfh, name, value); db.put(cfh, name, value);
return null; return null;
}) })
.onErrorMap(cause -> new IOException("Failed to write " + Arrays.toString(name), cause)); .onErrorMap(cause -> new IOException("Failed to write " + Arrays.toString(name), cause))
.subscribeOn(dbScheduler);
} }
@Override @Override

View File

@ -69,7 +69,6 @@ import org.warp.commonutils.log.LoggerFactory;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
public class LuceneUtils { public class LuceneUtils {
@ -366,6 +365,7 @@ public class LuceneUtils {
public static Flux<LLKeyScore> convertHits(ScoreDoc[] hits, public static Flux<LLKeyScore> convertHits(ScoreDoc[] hits,
IndexSearchers indexSearchers, IndexSearchers indexSearchers,
String keyFieldName, String keyFieldName,
Scheduler scheduler,
boolean preserveOrder) { boolean preserveOrder) {
return Flux return Flux
@ -373,11 +373,12 @@ public class LuceneUtils {
.transform(hitsFlux -> { .transform(hitsFlux -> {
if (preserveOrder) { if (preserveOrder) {
return hitsFlux return hitsFlux
.publishOn(scheduler)
.mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName)); .mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName));
} else { } else {
return hitsFlux return hitsFlux
.parallel() .parallel()
.runOn(Schedulers.boundedElastic()) .runOn(scheduler)
.map(hit -> { .map(hit -> {
var result = mapHitBlocking(hit, indexSearchers, keyFieldName); var result = mapHitBlocking(hit, indexSearchers, keyFieldName);
// The "else" value is an errored key score, to filter out next // The "else" value is an errored key score, to filter out next

View File

@ -17,22 +17,24 @@ public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher {
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher, public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName) { String keyFieldName,
Scheduler scheduler) {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
return releaseIndexSearcher return releaseIndexSearcher
.then(Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread"))); .then(Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread")));
} }
if (queryParams.limit() == 0) { if (queryParams.limit() == 0) {
return countSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName); return countSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName, scheduler);
} else if (!queryParams.isScored() && queryParams.offset() == 0 && queryParams.limit() >= 2147483630 } else if (!queryParams.isScored() && queryParams.offset() == 0 && queryParams.limit() >= 2147483630
&& !queryParams.isSorted()) { && !queryParams.isSorted()) {
return unscoredPagedLuceneLocalSearcher.collect(indexSearcher, return unscoredPagedLuceneLocalSearcher.collect(indexSearcher,
releaseIndexSearcher, releaseIndexSearcher,
queryParams, queryParams,
keyFieldName keyFieldName,
scheduler
); );
} else { } else {
return localSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName); return localSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName, scheduler);
} }
} }
} }

View File

@ -15,7 +15,9 @@ public class CountLuceneLocalSearcher implements LuceneLocalSearcher {
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher, public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName) { String keyFieldName,
Scheduler scheduler) {
//noinspection BlockingMethodInNonBlockingContext
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
@ -26,6 +28,7 @@ public class CountLuceneLocalSearcher implements LuceneLocalSearcher {
Flux.empty(), Flux.empty(),
releaseIndexSearcher); releaseIndexSearcher);
} }
); )
.subscribeOn(scheduler);
} }
} }

View File

@ -25,17 +25,20 @@ public class CountLuceneMultiSearcher implements LuceneMultiSearcher {
@Override @Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, public Mono<Void> searchOn(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams) { LocalQueryParams queryParams,
Scheduler scheduler) {
return Mono return Mono
.<Void>fromCallable(() -> { .<Void>fromCallable(() -> {
//noinspection BlockingMethodInNonBlockingContext
totalHits.addAndGet(indexSearcher.count(queryParams.query())); totalHits.addAndGet(indexSearcher.count(queryParams.query()));
release.add(releaseIndexSearcher); release.add(releaseIndexSearcher);
return null; return null;
}); })
.subscribeOn(scheduler);
} }
@Override @Override
public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName) { public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName, Scheduler scheduler) {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called collect in a nonblocking thread"); throw new UnsupportedOperationException("Called collect in a nonblocking thread");

View File

@ -16,12 +16,13 @@ public class LocalLuceneWrapper implements LuceneLocalSearcher {
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher, public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName) { String keyFieldName,
Scheduler scheduler) {
var shardSearcher = luceneMultiSearcher.createShardSearcher(queryParams); var shardSearcher = luceneMultiSearcher.createShardSearcher(queryParams);
return shardSearcher return shardSearcher
.flatMap(luceneShardSearcher -> luceneShardSearcher .flatMap(luceneShardSearcher -> luceneShardSearcher
.searchOn(indexSearcher, releaseIndexSearcher, queryParams) .searchOn(indexSearcher, releaseIndexSearcher, queryParams, scheduler)
.then(luceneShardSearcher.collect(queryParams, keyFieldName)) .then(luceneShardSearcher.collect(queryParams, keyFieldName, scheduler))
); );
} }
} }

View File

@ -11,9 +11,11 @@ public interface LuceneLocalSearcher {
* @param indexSearcher Lucene index searcher * @param indexSearcher Lucene index searcher
* @param queryParams the query parameters * @param queryParams the query parameters
* @param keyFieldName the name of the key field * @param keyFieldName the name of the key field
* @param scheduler a blocking scheduler
*/ */
Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher, Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName); String keyFieldName,
Scheduler scheduler);
} }

View File

@ -10,14 +10,17 @@ public interface LuceneShardSearcher {
/** /**
* @param indexSearcher the index searcher, which contains all the lucene data * @param indexSearcher the index searcher, which contains all the lucene data
* @param queryParams the query parameters * @param queryParams the query parameters
* @param scheduler a blocking scheduler
*/ */
Mono<Void> searchOn(IndexSearcher indexSearcher, Mono<Void> searchOn(IndexSearcher indexSearcher,
Mono<Void> indexSearcherRelease, Mono<Void> indexSearcherRelease,
LocalQueryParams queryParams); LocalQueryParams queryParams,
Scheduler scheduler);
/** /**
* @param queryParams the query parameters * @param queryParams the query parameters
* @param keyFieldName the name of the key field * @param keyFieldName the name of the key field
* @param collectorScheduler a blocking scheduler
*/ */
Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName); Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName, Scheduler collectorScheduler);
} }

View File

@ -48,25 +48,28 @@ class ScoredSimpleLuceneShardSearcher implements LuceneShardSearcher {
@Override @Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, public Mono<Void> searchOn(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams) { LocalQueryParams queryParams,
Scheduler scheduler) {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called searchOn in a nonblocking thread"); throw new UnsupportedOperationException("Called searchOn in a nonblocking thread");
} }
TopFieldCollector collector; TopFieldCollector collector;
synchronized (lock) { synchronized (lock) {
//noinspection BlockingMethodInNonBlockingContext
collector = firstPageSharedManager.newCollector(); collector = firstPageSharedManager.newCollector();
indexSearchersArray.add(indexSearcher); indexSearchersArray.add(indexSearcher);
indexSearcherReleasersArray.add(releaseIndexSearcher); indexSearcherReleasersArray.add(releaseIndexSearcher);
collectors.add(collector); collectors.add(collector);
} }
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(luceneQuery, collector); indexSearcher.search(luceneQuery, collector);
return null; return null;
}); }).subscribeOn(scheduler);
} }
@Override @Override
public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName) { public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName, Scheduler collectorScheduler) {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
return Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread")); return Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread"));
} }
@ -88,7 +91,7 @@ class ScoredSimpleLuceneShardSearcher implements LuceneShardSearcher {
indexSearchers = IndexSearchers.of(indexSearchersArray); indexSearchers = IndexSearchers.of(indexSearchersArray);
} }
Flux<LLKeyScore> firstPageHits = LuceneUtils Flux<LLKeyScore> firstPageHits = LuceneUtils
.convertHits(result.scoreDocs, indexSearchers, keyFieldName, true); .convertHits(result.scoreDocs, indexSearchers, keyFieldName, collectorScheduler, true);
Flux<LLKeyScore> nextHits = Flux.defer(() -> { Flux<LLKeyScore> nextHits = Flux.defer(() -> {
if (paginationInfo.forceSinglePage() if (paginationInfo.forceSinglePage()
@ -168,8 +171,9 @@ class ScoredSimpleLuceneShardSearcher implements LuceneShardSearcher {
emitter.onCancel(cancelEvent::tryEmitEmpty); emitter.onCancel(cancelEvent::tryEmitEmpty);
}) })
.subscribeOn(collectorScheduler)
.flatMapSequential(topFieldDoc -> LuceneUtils .flatMapSequential(topFieldDoc -> LuceneUtils
.convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, true) .convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, collectorScheduler, true)
); );
}); });
@ -179,7 +183,8 @@ class ScoredSimpleLuceneShardSearcher implements LuceneShardSearcher {
//.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)),
release release
); );
}); })
.subscribeOn(collectorScheduler);
} }
} }

View File

@ -23,7 +23,8 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher, public Mono<LuceneSearchResult> collect(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName) { String keyFieldName,
Scheduler scheduler) {
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
@ -45,6 +46,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
LuceneUtils.totalHitsThreshold(), LuceneUtils.totalHitsThreshold(),
!paginationInfo.forceSinglePage(), !paginationInfo.forceSinglePage(),
queryParams.isScored()); queryParams.isScored());
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(queryParams.query(), firstPageCollector); indexSearcher.search(queryParams.query(), firstPageCollector);
firstPageTopDocs = firstPageCollector.topDocs(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()), firstPageTopDocs = firstPageCollector.topDocs(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()),
LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()) LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit())
@ -55,6 +57,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
firstPageTopDocs.scoreDocs, firstPageTopDocs.scoreDocs,
IndexSearchers.unsharded(indexSearcher), IndexSearchers.unsharded(indexSearcher),
keyFieldName, keyFieldName,
scheduler,
true true
) )
.take(queryParams.limit(), true); .take(queryParams.limit(), true);
@ -97,8 +100,9 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
}, },
s -> {} s -> {}
) )
.subscribeOn(scheduler)
.flatMapSequential(topFieldDoc -> LuceneUtils .flatMapSequential(topFieldDoc -> LuceneUtils
.convertHits(topFieldDoc.scoreDocs, IndexSearchers.unsharded(indexSearcher), keyFieldName, true) .convertHits(topFieldDoc.scoreDocs, IndexSearchers.unsharded(indexSearcher), keyFieldName, scheduler, true)
) )
); );
} }
@ -116,6 +120,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher {
//.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)),
releaseIndexSearcher releaseIndexSearcher
); );
}); })
.subscribeOn(scheduler);
} }
} }

View File

@ -41,25 +41,28 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
@Override @Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, public Mono<Void> searchOn(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams) { LocalQueryParams queryParams,
Scheduler scheduler) {
return Mono.fromCallable(() -> { return Mono.fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called searchOn in a nonblocking thread"); throw new UnsupportedOperationException("Called searchOn in a nonblocking thread");
} }
TopDocsCollector<ScoreDoc> collector; TopDocsCollector<ScoreDoc> collector;
synchronized (lock) { synchronized (lock) {
//noinspection BlockingMethodInNonBlockingContext
collector = firstPageUnsortedCollectorManager.newCollector(); collector = firstPageUnsortedCollectorManager.newCollector();
indexSearchersArray.add(indexSearcher); indexSearchersArray.add(indexSearcher);
indexSearcherReleasersArray.add(releaseIndexSearcher); indexSearcherReleasersArray.add(releaseIndexSearcher);
collectors.add(collector); collectors.add(collector);
} }
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(luceneQuery, collector); indexSearcher.search(luceneQuery, collector);
return null; return null;
}); }).subscribeOn(scheduler);
} }
@Override @Override
public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName) { public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, String keyFieldName, Scheduler scheduler) {
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
@ -68,7 +71,7 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
TopDocs result; TopDocs result;
Mono<Void> release; Mono<Void> release;
synchronized (lock) { synchronized (lock) {
//noinspection BlockingMethodInNonBlockingContext
result = firstPageUnsortedCollectorManager.reduce(collectors); result = firstPageUnsortedCollectorManager.reduce(collectors);
release = Mono.when(indexSearcherReleasersArray); release = Mono.when(indexSearcherReleasersArray);
} }
@ -77,7 +80,7 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
indexSearchers = IndexSearchers.of(indexSearchersArray); indexSearchers = IndexSearchers.of(indexSearchersArray);
} }
Flux<LLKeyScore> firstPageHits = LuceneUtils Flux<LLKeyScore> firstPageHits = LuceneUtils
.convertHits(result.scoreDocs, indexSearchers, keyFieldName, false); .convertHits(result.scoreDocs, indexSearchers, keyFieldName, scheduler, false);
Flux<LLKeyScore> nextHits = Flux.defer(() -> { Flux<LLKeyScore> nextHits = Flux.defer(() -> {
if (paginationInfo.forceSinglePage() || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { if (paginationInfo.forceSinglePage() || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) {
@ -95,19 +98,23 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
() -> TopDocsSearcher.getTopDocsCollector(queryParams.sort(), s.currentPageLimit(), () -> TopDocsSearcher.getTopDocsCollector(queryParams.sort(), s.currentPageLimit(),
s.last(), LuceneUtils.totalHitsThreshold(), true, queryParams.isScored()), s.last(), LuceneUtils.totalHitsThreshold(), true, queryParams.isScored()),
0, s.currentPageLimit(), queryParams.sort()); 0, s.currentPageLimit(), queryParams.sort());
//noinspection BlockingMethodInNonBlockingContext
TopDocs pageTopDocs = Flux TopDocs pageTopDocs = Flux
.fromIterable(indexSearchersArray) .fromIterable(indexSearchersArray)
.flatMapSequential(indexSearcher -> Mono .flatMapSequential(indexSearcher -> Mono
.fromCallable(() -> { .fromCallable(() -> {
//noinspection BlockingMethodInNonBlockingContext
var collector = currentPageUnsortedCollectorManager.newCollector(); var collector = currentPageUnsortedCollectorManager.newCollector();
//noinspection BlockingMethodInNonBlockingContext
indexSearcher.search(luceneQuery, collector); indexSearcher.search(luceneQuery, collector);
return collector; return collector;
}) })
.subscribeOn(scheduler)
) )
.collect(Collectors.toCollection(ObjectArrayList::new)) .collect(Collectors.toCollection(ObjectArrayList::new))
.flatMap(collectors -> Mono .flatMap(collectors -> Mono
.fromCallable(() -> currentPageUnsortedCollectorManager.reduce(collectors)) .fromCallable(() -> currentPageUnsortedCollectorManager.reduce(collectors))
.subscribeOn(scheduler)
) )
.blockOptional().orElseThrow(); .blockOptional().orElseThrow();
@ -122,8 +129,9 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
}, },
s -> {} s -> {}
) )
.subscribeOn(scheduler)
.flatMapSequential(topFieldDoc -> LuceneUtils .flatMapSequential(topFieldDoc -> LuceneUtils
.convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, false) .convertHits(topFieldDoc.scoreDocs, indexSearchers, keyFieldName, scheduler, false)
); );
}); });
@ -132,7 +140,8 @@ class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher {
//.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)),
release release
); );
}); })
.subscribeOn(scheduler);
} }
} }

View File

@ -99,7 +99,8 @@ public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMult
@Override @Override
public Mono<Void> searchOn(IndexSearcher indexSearcher, public Mono<Void> searchOn(IndexSearcher indexSearcher,
Mono<Void> releaseIndexSearcher, Mono<Void> releaseIndexSearcher,
LocalQueryParams queryParams) { LocalQueryParams queryParams,
Scheduler scheduler) {
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
@ -126,12 +127,14 @@ public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMult
} }
}); });
return null; return null;
}); })
.subscribeOn(scheduler);
} }
@Override @Override
public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams, public Mono<LuceneSearchResult> collect(LocalQueryParams queryParams,
String keyFieldName) { String keyFieldName,
Scheduler scheduler) {
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
if (Schedulers.isInNonBlockingThread()) { if (Schedulers.isInNonBlockingThread()) {
@ -166,6 +169,7 @@ public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMult
.flatMap(scoreDocs -> LuceneUtils.convertHits(scoreDocs.toArray(ScoreDoc[]::new), .flatMap(scoreDocs -> LuceneUtils.convertHits(scoreDocs.toArray(ScoreDoc[]::new),
indexSearchers, indexSearchers,
keyFieldName, keyFieldName,
scheduler,
false false
)); ));