Reduce the possibility of leaks

This commit is contained in:
Andrea Cavalli 2022-07-02 11:44:13 +02:00
parent caf55a633e
commit 2a9427f0e4
30 changed files with 388 additions and 411 deletions

View File

@ -80,7 +80,7 @@ public class CodecsExample {
.then(), .then(),
SpeedExample.numRepeats, SpeedExample.numRepeats,
tuple -> tuple.getT1().close() tuple -> tuple.getT1().close()
)).subscribeOn(Schedulers.parallel()).blockOptional(); )).transform(LLUtils::handleDiscard).subscribeOn(Schedulers.parallel()).blockOptional();
} }
private static void testConversion() { private static void testConversion() {
@ -88,6 +88,7 @@ public class CodecsExample {
.then() .then()
.then(readNew()) .then(readNew())
.subscribeOn(Schedulers.parallel()) .subscribeOn(Schedulers.parallel())
.transform(LLUtils::handleDiscard)
.blockOptional(); .blockOptional();
} }

View File

@ -62,6 +62,7 @@ public class IndicizationExample {
.then(index.close()) .then(index.close())
) )
.subscribeOn(Schedulers.parallel()) .subscribeOn(Schedulers.parallel())
.transform(LLUtils::handleDiscard)
.block(); .block();
tempIndex(true) tempIndex(true)
.flatMap(index -> .flatMap(index ->
@ -138,6 +139,7 @@ public class IndicizationExample {
.then(index.close()) .then(index.close())
) )
.subscribeOn(Schedulers.parallel()) .subscribeOn(Schedulers.parallel())
.transform(LLUtils::handleDiscard)
.block(); .block();
} }

View File

@ -56,6 +56,7 @@ public class SpeedExample {
.then(test3LevelPut()) .then(test3LevelPut())
.then(test4LevelPut()) .then(test4LevelPut())
.subscribeOn(Schedulers.parallel()) .subscribeOn(Schedulers.parallel())
.transform(LLUtils::handleDiscard)
.blockOptional(); .blockOptional();
} }

View File

@ -35,6 +35,7 @@ import reactor.core.publisher.SignalType;
public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> { public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
private static final Duration MAX_COUNT_TIME = Duration.ofSeconds(30);
private final LLLuceneIndex luceneIndex; private final LLLuceneIndex luceneIndex;
private final Indicizer<T,U> indicizer; private final Indicizer<T,U> indicizer;
@ -149,13 +150,9 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
@Override @Override
public Mono<TotalHitsCount> count(@Nullable CompositeSnapshot snapshot, Query query) { public Mono<TotalHitsCount> count(@Nullable CompositeSnapshot snapshot, Query query) {
return Mono.usingWhen(this.search(ClientQueryParams return luceneIndex
.builder() .count(resolveSnapshot(snapshot), query, MAX_COUNT_TIME)
.snapshot(snapshot) .doOnDiscard(DiscardingCloseable.class, DiscardingCloseable::close);
.query(query)
.timeout(Duration.ofSeconds(30))
.limit(0)
.build()), searchResultKeys -> Mono.just(searchResultKeys.totalHitsCount()), LLUtils::finalizeResource);
} }
@Override @Override
@ -205,8 +202,7 @@ public class LuceneIndexImpl<T, U> implements LuceneIndex<T, U> {
} }
@Nullable @Nullable
private static LLSearchResultShard mergeResults(ClientQueryParams queryParams, private static LLSearchResultShard mergeResults(ClientQueryParams queryParams, List<LLSearchResultShard> shards) {
List<LLSearchResultShard> shards) {
if (shards.size() == 0) { if (shards.size() == 0) {
return null; return null;
} else if (shards.size() == 1) { } else if (shards.size() == 1) {

View File

@ -7,6 +7,7 @@ import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
import it.cavallium.dbengine.lucene.collector.Buckets; import it.cavallium.dbengine.lucene.collector.Buckets;
import it.cavallium.dbengine.lucene.searcher.BucketParams; import it.cavallium.dbengine.lucene.searcher.BucketParams;
import java.time.Duration;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -60,16 +61,20 @@ public interface LLLuceneIndex extends LLSnapshottable, SafeCloseable {
@Nullable Query normalizationQuery, @Nullable Query normalizationQuery,
BucketParams bucketParams); BucketParams bucketParams);
default Mono<TotalHitsCount> count(@Nullable LLSnapshot snapshot, Query query) { default Mono<TotalHitsCount> count(@Nullable LLSnapshot snapshot, Query query, @Nullable Duration timeout) {
QueryParams params = QueryParams.of(query, 0, 0, NoSort.of(), false, Long.MAX_VALUE); QueryParams params = QueryParams.of(query,
return Mono.from(this.search(snapshot, params, null) 0,
.map(llSearchResultShard -> { 0,
try (llSearchResultShard) { NoSort.of(),
return llSearchResultShard.totalHitsCount(); false,
} timeout == null ? Long.MAX_VALUE : timeout.toMillis()
})
.defaultIfEmpty(TotalHitsCount.of(0, true))
); );
return Mono
.usingWhen(this.search(snapshot, params, null).singleOrEmpty(),
llSearchResultShard -> Mono.just(llSearchResultShard.totalHitsCount()),
LLUtils::finalizeResource
)
.defaultIfEmpty(TotalHitsCount.of(0, true));
} }
boolean isLowMemoryMode(); boolean isLowMemoryMode();

View File

@ -156,14 +156,14 @@ public class LLMultiDatabaseConnection implements LLDatabaseConnection {
var connectionIndexStructure = indexStructure var connectionIndexStructure = indexStructure
.setActiveShards(new IntArrayList(entry.getValue())); .setActiveShards(new IntArrayList(entry.getValue()));
var connIndex = entry.getKey() Flux<LLLuceneIndex> connIndex = entry.getKey()
.getLuceneIndex(clusterName, .getLuceneIndex(clusterName,
connectionIndexStructure, connectionIndexStructure,
indicizerAnalyzers, indicizerAnalyzers,
indicizerSimilarities, indicizerSimilarities,
luceneOptions, luceneOptions,
luceneHacks luceneHacks
).cache().repeat(); ).cast(LLLuceneIndex.class).cache().repeat();
return Flux return Flux
.fromIterable(entry.getValue()) .fromIterable(entry.getValue())
.zipWith(connIndex); .zipWith(connIndex);
@ -171,7 +171,7 @@ public class LLMultiDatabaseConnection implements LLDatabaseConnection {
.collectList() .collectList()
.map(indices -> { .map(indices -> {
var luceneIndices = new LLLuceneIndex[indexStructure.totalShards()]; var luceneIndices = new LLLuceneIndex[indexStructure.totalShards()];
for (Tuple2<Integer, ? extends LLLuceneIndex> index : indices) { for (var index : indices) {
luceneIndices[index.getT1()] = index.getT2(); luceneIndices[index.getT1()] = index.getT2();
} }
return new LLMultiLuceneIndex(clusterName, return new LLMultiLuceneIndex(clusterName,

View File

@ -131,7 +131,7 @@ public class LLMultiLuceneIndex implements LLLuceneIndex {
queryParams, queryParams,
keyFieldName, keyFieldName,
mltDocumentFields mltDocumentFields
)); )).doOnDiscard(DiscardingCloseable.class, SafeCloseable::close);
} }
private Mono<Buckets> mergeShards(List<Buckets> shards) { private Mono<Buckets> mergeShards(List<Buckets> shards) {
@ -167,7 +167,7 @@ public class LLMultiLuceneIndex implements LLLuceneIndex {
return luceneIndicesFlux.flatMap(luceneIndex -> luceneIndex.search(snapshot, return luceneIndicesFlux.flatMap(luceneIndex -> luceneIndex.search(snapshot,
queryParams, queryParams,
keyFieldName keyFieldName
)); )).doOnDiscard(DiscardingCloseable.class, SafeCloseable::close);
} }
@Override @Override
@ -179,12 +179,7 @@ public class LLMultiLuceneIndex implements LLLuceneIndex {
queries, queries,
normalizationQuery, normalizationQuery,
bucketParams bucketParams
)).collectList().flatMap(this::mergeShards); )).collectList().flatMap(this::mergeShards).doOnDiscard(DiscardingCloseable.class, SafeCloseable::close);
}
@Override
public Mono<TotalHitsCount> count(@Nullable LLSnapshot snapshot, Query query) {
return LLLuceneIndex.super.count(snapshot, query);
} }
@Override @Override
@ -195,7 +190,7 @@ public class LLMultiLuceneIndex implements LLLuceneIndex {
@Override @Override
public void close() { public void close() {
Iterable<Mono<Void>> it = () -> luceneIndicesSet.stream().map(e -> Mono.<Void>fromRunnable(e::close)).iterator(); Iterable<Mono<Void>> it = () -> luceneIndicesSet.stream().map(e -> Mono.<Void>fromRunnable(e::close)).iterator();
Mono.whenDelayError(it).block(); Mono.whenDelayError(it).transform(LLUtils::handleDiscard).block();
} }
@Override @Override

View File

@ -647,7 +647,7 @@ public class LLUtils {
return Mono.fromRunnable(() -> LLUtils.finalizeResourceNow(resource)); return Mono.fromRunnable(() -> LLUtils.finalizeResourceNow(resource));
} }
public static Mono<Void> finalizeResource(SimpleResource resource) { public static Mono<Void> finalizeResource(SafeCloseable resource) {
return Mono.fromRunnable(resource::close); return Mono.fromRunnable(resource::close);
} }
@ -657,7 +657,7 @@ public class LLUtils {
} }
} }
public static void finalizeResourceNow(SimpleResource resource) { public static void finalizeResourceNow(SafeCloseable resource) {
resource.close(); resource.close();
} }

View File

@ -74,7 +74,7 @@ public class DatabaseMapDictionaryHashed<T, U, TH> extends
SerializerFixedBinaryLength<TH> keySuffixHashSerializer, SerializerFixedBinaryLength<TH> keySuffixHashSerializer,
Runnable onClose) { Runnable onClose) {
super((Drop<DatabaseMapDictionaryHashed<T, U, TH>>) (Drop) DROP); super((Drop<DatabaseMapDictionaryHashed<T, U, TH>>) (Drop) DROP);
if (dictionary.getUpdateMode().block() != UpdateMode.ALLOW) { if (dictionary.getUpdateMode().transform(LLUtils::handleDiscard).block() != UpdateMode.ALLOW) {
throw new IllegalArgumentException("Hashed maps only works when UpdateMode is ALLOW"); throw new IllegalArgumentException("Hashed maps only works when UpdateMode is ALLOW");
} }
this.alloc = dictionary.getAllocator(); this.alloc = dictionary.getAllocator();

View File

@ -263,7 +263,7 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends
* Value getter doesn't lock data. Please make sure to lock before getting data. * Value getter doesn't lock data. Please make sure to lock before getting data.
*/ */
default ValueGetterBlocking<T, U> getDbValueGetter(@Nullable CompositeSnapshot snapshot) { default ValueGetterBlocking<T, U> getDbValueGetter(@Nullable CompositeSnapshot snapshot) {
return k -> getValue(snapshot, k).block(); return k -> getValue(snapshot, k).transform(LLUtils::handleDiscard).block();
} }
default ValueGetter<T, U> getAsyncDbValueGetter(@Nullable CompositeSnapshot snapshot) { default ValueGetter<T, U> getAsyncDbValueGetter(@Nullable CompositeSnapshot snapshot) {

View File

@ -238,10 +238,6 @@ public class CachedIndexSearcherManager extends SimpleResource implements IndexS
private class SnapshotIndexSearcher extends LLIndexSearcher { private class SnapshotIndexSearcher extends LLIndexSearcher {
public SnapshotIndexSearcher(IndexSearcher indexSearcher, AtomicBoolean closed) {
super(indexSearcher, closed);
}
public SnapshotIndexSearcher(IndexSearcher indexSearcher) { public SnapshotIndexSearcher(IndexSearcher indexSearcher) {
super(indexSearcher); super(indexSearcher);
} }

View File

@ -17,6 +17,7 @@ import io.micrometer.core.instrument.Timer;
import it.cavallium.dbengine.client.query.QueryParser; import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.Query; import it.cavallium.dbengine.client.query.current.data.Query;
import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
import it.cavallium.dbengine.database.LLIndexRequest; import it.cavallium.dbengine.database.LLIndexRequest;
import it.cavallium.dbengine.database.LLLuceneIndex; import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLSearchResultShard; import it.cavallium.dbengine.database.LLSearchResultShard;
@ -37,6 +38,7 @@ import it.cavallium.dbengine.lucene.searcher.BucketParams;
import it.cavallium.dbengine.lucene.searcher.DecimalBucketMultiSearcher; import it.cavallium.dbengine.lucene.searcher.DecimalBucketMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams;
import it.cavallium.dbengine.lucene.searcher.LocalSearcher; import it.cavallium.dbengine.lucene.searcher.LocalSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneSearchResult;
import it.cavallium.dbengine.rpc.current.data.IndicizerAnalyzers; import it.cavallium.dbengine.rpc.current.data.IndicizerAnalyzers;
import it.cavallium.dbengine.rpc.current.data.IndicizerSimilarities; import it.cavallium.dbengine.rpc.current.data.IndicizerSimilarities;
import it.cavallium.dbengine.rpc.current.data.LuceneOptions; import it.cavallium.dbengine.rpc.current.data.LuceneOptions;
@ -508,13 +510,28 @@ public class LLLocalLuceneIndex extends SimpleResource implements LLLuceneIndex
@Override @Override
public Flux<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, public Flux<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, QueryParams queryParams,
@Nullable String keyFieldName) { @Nullable String keyFieldName) {
return searchInternal(snapshot, queryParams, keyFieldName)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.flux();
}
public Mono<LuceneSearchResult> searchInternal(@Nullable LLSnapshot snapshot, QueryParams queryParams,
@Nullable String keyFieldName) {
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams, luceneAnalyzer); LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams, luceneAnalyzer);
var searcher = searcherManager.retrieveSearcher(snapshot); var searcher = searcherManager.retrieveSearcher(snapshot);
return localSearcher return localSearcher.collect(searcher, localQueryParams, keyFieldName, NO_REWRITE);
.collect(searcher, localQueryParams, keyFieldName, NO_REWRITE) }
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.flux(); @Override
public Mono<TotalHitsCount> count(@Nullable LLSnapshot snapshot, Query query, @Nullable Duration timeout) {
var params = LuceneUtils.getCountQueryParams(query);
return Mono
.usingWhen(this.searchInternal(snapshot, params, null),
result -> Mono.just(result.totalHitsCount()),
LLUtils::finalizeResource
)
.defaultIfEmpty(TotalHitsCount.of(0, true));
} }
@Override @Override

View File

@ -6,8 +6,10 @@ import com.google.common.collect.Multimap;
import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.MeterRegistry;
import io.netty5.buffer.api.Send; import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.client.query.QueryParser; import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.NoSort;
import it.cavallium.dbengine.client.query.current.data.Query; import it.cavallium.dbengine.client.query.current.data.Query;
import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
import it.cavallium.dbengine.database.LLIndexRequest; import it.cavallium.dbengine.database.LLIndexRequest;
import it.cavallium.dbengine.database.LLLuceneIndex; import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLSearchResultShard; import it.cavallium.dbengine.database.LLSearchResultShard;
@ -25,6 +27,7 @@ import it.cavallium.dbengine.lucene.searcher.BucketParams;
import it.cavallium.dbengine.lucene.searcher.DecimalBucketMultiSearcher; import it.cavallium.dbengine.lucene.searcher.DecimalBucketMultiSearcher;
import it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite; import it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite;
import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams;
import it.cavallium.dbengine.lucene.searcher.LuceneSearchResult;
import it.cavallium.dbengine.lucene.searcher.MultiSearcher; import it.cavallium.dbengine.lucene.searcher.MultiSearcher;
import it.cavallium.dbengine.rpc.current.data.IndicizerAnalyzers; import it.cavallium.dbengine.rpc.current.data.IndicizerAnalyzers;
import it.cavallium.dbengine.rpc.current.data.IndicizerSimilarities; import it.cavallium.dbengine.rpc.current.data.IndicizerSimilarities;
@ -34,6 +37,7 @@ import it.unimi.dsi.fastutil.ints.IntList;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.io.UncheckedIOException; import java.io.UncheckedIOException;
import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -288,15 +292,31 @@ public class LLLocalMultiLuceneIndex extends SimpleResource implements LLLuceneI
public Flux<LLSearchResultShard> search(@Nullable LLSnapshot snapshot, public Flux<LLSearchResultShard> search(@Nullable LLSnapshot snapshot,
QueryParams queryParams, QueryParams queryParams,
@Nullable String keyFieldName) { @Nullable String keyFieldName) {
return searchInternal(snapshot, queryParams, keyFieldName)
// Transform the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.flux();
}
private Mono<LuceneSearchResult> searchInternal(@Nullable LLSnapshot snapshot,
QueryParams queryParams,
@Nullable String keyFieldName) {
LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams, luceneAnalyzer); LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams, luceneAnalyzer);
var searchers = getIndexSearchers(snapshot); var searchers = getIndexSearchers(snapshot);
// Collect all the shards results into a single global result // Collect all the shards results into a single global result
return multiSearcher return multiSearcher.collectMulti(searchers, localQueryParams, keyFieldName, GlobalQueryRewrite.NO_REWRITE);
.collectMulti(searchers, localQueryParams, keyFieldName, GlobalQueryRewrite.NO_REWRITE) }
// Transform the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close)) @Override
.flux(); public Mono<TotalHitsCount> count(@Nullable LLSnapshot snapshot, Query query, @Nullable Duration timeout) {
var params = LuceneUtils.getCountQueryParams(query);
return Mono
.usingWhen(this.searchInternal(snapshot, params, null),
result -> Mono.just(result.totalHitsCount()),
LLUtils::finalizeResource
)
.defaultIfEmpty(TotalHitsCount.of(0, true));
} }
@Override @Override
@ -336,6 +356,7 @@ public class LLLocalMultiLuceneIndex extends SimpleResource implements LLLuceneI
}).subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))) }).subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())))
.publishOn(Schedulers.parallel()) .publishOn(Schedulers.parallel())
.then() .then()
.transform(LLUtils::handleDiscard)
.block(); .block();
} }

View File

@ -2,6 +2,7 @@ package it.cavallium.dbengine.database.disk;
import static com.google.common.collect.Lists.partition; import static com.google.common.collect.Lists.partition;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.rpc.current.data.Column; import it.cavallium.dbengine.rpc.current.data.Column;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
@ -92,7 +93,7 @@ public class RocksDBUtils {
} }
} }
return null; return null;
}).subscribeOn(Schedulers.boundedElastic())).toList()).block(); }).subscribeOn(Schedulers.boundedElastic())).toList()).transform(LLUtils::handleDiscard).block();
} }
} }
} }

View File

@ -22,6 +22,7 @@ import it.cavallium.dbengine.database.LLSingleton;
import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLTerm; import it.cavallium.dbengine.database.LLTerm;
import it.cavallium.dbengine.database.LLUpdateDocument; import it.cavallium.dbengine.database.LLUpdateDocument;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.RocksDBLongProperty; import it.cavallium.dbengine.database.RocksDBLongProperty;
import it.cavallium.dbengine.database.RocksDBMapProperty; import it.cavallium.dbengine.database.RocksDBMapProperty;
import it.cavallium.dbengine.database.RocksDBStringProperty; import it.cavallium.dbengine.database.RocksDBStringProperty;
@ -510,7 +511,7 @@ public class LLQuicConnection implements LLDatabaseConnection {
@Override @Override
public void close() { public void close() {
sendRequest(new CloseLuceneIndex(id)).then().block(); sendRequest(new CloseLuceneIndex(id)).then().transform(LLUtils::handleDiscard).block();
} }
@Override @Override

View File

@ -69,7 +69,7 @@ public class BufferDataInputShared implements BufferDataInput {
} }
@Override @Override
public int readUnsignedByte() { public int readUnsignedByte() {/* if (StackWalker.getInstance().walk(s -> s.limit(16).anyMatch(frame -> frame.getMethodName().contains("updateAndGetDelta")))) {throw new TempException();}*/
if (buf == null) throw new IndexOutOfBoundsException(); if (buf == null) throw new IndexOutOfBoundsException();
return buf.readUnsignedByte(); return buf.readUnsignedByte();
} }

View File

@ -1,6 +1,8 @@
package it.cavallium.dbengine.lucene; package it.cavallium.dbengine.lucene;
import static it.cavallium.dbengine.client.UninterruptibleScheduler.uninterruptibleScheduler; import static it.cavallium.dbengine.client.UninterruptibleScheduler.uninterruptibleScheduler;
import static it.cavallium.dbengine.database.LLUtils.singleOrClose;
import static it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite.NO_REWRITE;
import com.google.common.collect.HashMultimap; import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap; import com.google.common.collect.Multimap;
@ -9,6 +11,7 @@ import it.cavallium.data.generator.nativedata.Nullableint;
import it.cavallium.data.generator.nativedata.Nullablelong; import it.cavallium.data.generator.nativedata.Nullablelong;
import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.client.query.QueryParser; import it.cavallium.dbengine.client.query.QueryParser;
import it.cavallium.dbengine.client.query.current.data.NoSort;
import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.client.query.current.data.QueryParams;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLKeyScore;
@ -18,7 +21,9 @@ import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep;
import it.cavallium.dbengine.database.collections.DatabaseStageEntry; import it.cavallium.dbengine.database.collections.DatabaseStageEntry;
import it.cavallium.dbengine.database.collections.DatabaseStageMap; import it.cavallium.dbengine.database.collections.DatabaseStageMap;
import it.cavallium.dbengine.database.collections.ValueGetter; import it.cavallium.dbengine.database.collections.ValueGetter;
import it.cavallium.dbengine.database.disk.LLIndexSearcher;
import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.database.disk.LLIndexSearchers;
import it.cavallium.dbengine.database.disk.LLIndexSearchers.UnshardedIndexSearchers;
import it.cavallium.dbengine.lucene.analyzer.LegacyWordAnalyzer; import it.cavallium.dbengine.lucene.analyzer.LegacyWordAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer; import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer;
import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer; import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer;
@ -28,7 +33,12 @@ import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer;
import it.cavallium.dbengine.lucene.directory.RocksdbDirectory; import it.cavallium.dbengine.lucene.directory.RocksdbDirectory;
import it.cavallium.dbengine.lucene.mlt.BigCompositeReader; import it.cavallium.dbengine.lucene.mlt.BigCompositeReader;
import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis; import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis;
import it.cavallium.dbengine.lucene.searcher.AdaptiveLocalSearcher;
import it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite;
import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams;
import it.cavallium.dbengine.lucene.searcher.LocalSearcher;
import it.cavallium.dbengine.lucene.searcher.LuceneSearchResult;
import it.cavallium.dbengine.lucene.searcher.MultiSearcher;
import it.cavallium.dbengine.lucene.similarity.NGramSimilarity; import it.cavallium.dbengine.lucene.similarity.NGramSimilarity;
import it.cavallium.dbengine.rpc.current.data.ByteBuffersDirectory; import it.cavallium.dbengine.rpc.current.data.ByteBuffersDirectory;
import it.cavallium.dbengine.rpc.current.data.DirectIOFSDirectory; import it.cavallium.dbengine.rpc.current.data.DirectIOFSDirectory;
@ -104,7 +114,6 @@ import org.novasearch.lucene.search.similarities.BM25Similarity.BM25Model;
import org.novasearch.lucene.search.similarities.LdpSimilarity; import org.novasearch.lucene.search.similarities.LdpSimilarity;
import org.novasearch.lucene.search.similarities.LtcSimilarity; import org.novasearch.lucene.search.similarities.LtcSimilarity;
import org.novasearch.lucene.search.similarities.RobertsonSimilarity; import org.novasearch.lucene.search.similarities.RobertsonSimilarity;
import org.rocksdb.util.SizeUnit;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers; import reactor.core.scheduler.Schedulers;
@ -743,4 +752,39 @@ public class LuceneUtils {
public static it.cavallium.dbengine.rpc.current.data.TieredMergePolicy getDefaultMergePolicy() { public static it.cavallium.dbengine.rpc.current.data.TieredMergePolicy getDefaultMergePolicy() {
return DEFAULT_MERGE_POLICY; return DEFAULT_MERGE_POLICY;
} }
public static QueryParams getCountQueryParams(it.cavallium.dbengine.client.query.current.data.Query query) {
return QueryParams.of(query, 0, 0, NoSort.of(), false, Long.MAX_VALUE);
}
public static Mono<LuceneSearchResult> rewrite(LocalSearcher localSearcher,
Mono<LLIndexSearcher> indexSearcherMono,
LocalQueryParams queryParams,
String keyFieldName,
GlobalQueryRewrite transformer) {
return Mono.usingWhen(indexSearcherMono, indexSearcher -> {
try (UnshardedIndexSearchers indexSearchers = LLIndexSearchers.unsharded(indexSearcher)) {
return Mono
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))
.flatMap(queryParams2 ->
localSearcher.collect(indexSearcherMono, queryParams2, keyFieldName, NO_REWRITE));
}
}, LLUtils::finalizeResource);
}
public static Mono<LuceneSearchResult> rewriteMulti(MultiSearcher multiSearcher,
Mono<LLIndexSearchers> indexSearchersMono,
LocalQueryParams queryParams,
String keyFieldName,
GlobalQueryRewrite transformer) {
return Mono.usingWhen(indexSearchersMono,
indexSearchers -> Mono
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))
.flatMap(queryParams2 ->
multiSearcher.collectMulti(indexSearchersMono, queryParams2, keyFieldName, NO_REWRITE)),
LLUtils::finalizeResource
);
}
} }

View File

@ -7,7 +7,9 @@ import static it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite.NO_REWRIT
import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearcher;
import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.database.disk.LLIndexSearchers;
import it.cavallium.dbengine.database.disk.LLIndexSearchers.UnshardedIndexSearchers;
import it.cavallium.dbengine.database.disk.LLTempHugePqEnv; import it.cavallium.dbengine.database.disk.LLTempHugePqEnv;
import it.cavallium.dbengine.lucene.LuceneUtils;
import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers; import reactor.core.scheduler.Schedulers;
@ -47,18 +49,10 @@ public class AdaptiveLocalSearcher implements LocalSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearcherMono, indexSearcher -> { if (transformer != NO_REWRITE) {
var indexSearchers = LLIndexSearchers.unsharded(indexSearcher); return LuceneUtils.rewrite(this, indexSearcherMono, queryParams, keyFieldName, transformer);
}
if (transformer == NO_REWRITE) { return transformedCollect(indexSearcherMono, queryParams, keyFieldName, transformer);
return transformedCollect(indexSearcher, queryParams, keyFieldName, transformer);
} else {
return Mono
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))
.flatMap(queryParams2 -> transformedCollect(indexSearcher, queryParams2, keyFieldName, NO_REWRITE));
}
});
} }
@Override @Override
@ -67,7 +61,7 @@ public class AdaptiveLocalSearcher implements LocalSearcher {
} }
// Remember to change also AdaptiveMultiSearcher // Remember to change also AdaptiveMultiSearcher
public Mono<LuceneSearchResult> transformedCollect(LLIndexSearcher indexSearcher, public Mono<LuceneSearchResult> transformedCollect(Mono<LLIndexSearcher> indexSearcherMono,
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName, String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
@ -77,36 +71,36 @@ public class AdaptiveLocalSearcher implements LocalSearcher {
= Math.max(maxInMemoryResultEntries, (long) queryParams.pageLimits().getPageLimit(0)); = Math.max(maxInMemoryResultEntries, (long) queryParams.pageLimits().getPageLimit(0));
if (!FORCE_HUGE_PQ && queryParams.limitLong() == 0) { if (!FORCE_HUGE_PQ && queryParams.limitLong() == 0) {
return countSearcher.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return countSearcher.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} else if (!FORCE_HUGE_PQ && realLimit <= maxInMemoryResultEntries) { } else if (!FORCE_HUGE_PQ && realLimit <= maxInMemoryResultEntries) {
return standardSearcher.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return standardSearcher.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} else if (FORCE_HUGE_PQ || queryParams.isSorted()) { } else if (FORCE_HUGE_PQ || queryParams.isSorted()) {
if (!FORCE_HUGE_PQ && realLimit <= maxAllowedInMemoryLimit) { if (!FORCE_HUGE_PQ && realLimit <= maxAllowedInMemoryLimit) {
return scoredPaged.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return scoredPaged.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} else { } else {
if (queryParams.isSortedByScore()) { if (queryParams.isSortedByScore()) {
if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) { if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) {
throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater"); throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater");
} }
if (sortedByScoreFull != null) { if (sortedByScoreFull != null) {
return sortedByScoreFull.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return sortedByScoreFull.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} else { } else {
return scoredPaged.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return scoredPaged.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} }
} else { } else {
if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) { if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) {
throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater"); throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater");
} }
if (sortedScoredFull != null) { if (sortedScoredFull != null) {
return sortedScoredFull.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return sortedScoredFull.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} else { } else {
return scoredPaged.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return scoredPaged.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} }
} }
} }
} else { } else {
// Run large/unbounded searches using the continuous multi searcher // Run large/unbounded searches using the continuous multi searcher
return unsortedUnscoredContinuous.collect(Mono.just(indexSearcher), queryParams, keyFieldName, transformer); return unsortedUnscoredContinuous.collect(indexSearcherMono, queryParams, keyFieldName, transformer);
} }
} }
} }

View File

@ -9,6 +9,7 @@ import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.database.disk.LLIndexSearchers;
import it.cavallium.dbengine.database.disk.LLTempHugePqEnv; import it.cavallium.dbengine.database.disk.LLTempHugePqEnv;
import it.cavallium.dbengine.lucene.LuceneUtils;
import java.io.IOException; import java.io.IOException;
import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
@ -46,19 +47,14 @@ public class AdaptiveMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != NO_REWRITE) {
if (transformer == NO_REWRITE) { return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
return transformedCollectMulti(indexSearchers, queryParams, keyFieldName, transformer); }
} else { return transformedCollectMulti(indexSearchersMono, queryParams, keyFieldName, transformer);
return Mono.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))
.flatMap(queryParams2 -> transformedCollectMulti(indexSearchers, queryParams2, keyFieldName, NO_REWRITE));
}
});
} }
// Remember to change also AdaptiveLocalSearcher // Remember to change also AdaptiveLocalSearcher
public Mono<LuceneSearchResult> transformedCollectMulti(LLIndexSearchers indexSearchers, public Mono<LuceneSearchResult> transformedCollectMulti(Mono<LLIndexSearchers> indexSearchers,
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
@ -68,36 +64,36 @@ public class AdaptiveMultiSearcher implements MultiSearcher {
= Math.max(maxInMemoryResultEntries, (long) queryParams.pageLimits().getPageLimit(0)); = Math.max(maxInMemoryResultEntries, (long) queryParams.pageLimits().getPageLimit(0));
if (!FORCE_HUGE_PQ && queryParams.limitLong() == 0) { if (!FORCE_HUGE_PQ && queryParams.limitLong() == 0) {
return count.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return count.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} else if (!FORCE_HUGE_PQ && realLimit <= maxInMemoryResultEntries) { } else if (!FORCE_HUGE_PQ && realLimit <= maxInMemoryResultEntries) {
return standardSearcher.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return standardSearcher.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} else if (FORCE_HUGE_PQ || queryParams.isSorted()) { } else if (FORCE_HUGE_PQ || queryParams.isSorted()) {
if (!FORCE_HUGE_PQ && realLimit <= maxAllowedInMemoryLimit) { if (!FORCE_HUGE_PQ && realLimit <= maxAllowedInMemoryLimit) {
return scoredPaged.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return scoredPaged.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} else { } else {
if (queryParams.isSortedByScore()) { if (queryParams.isSortedByScore()) {
if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) { if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) {
throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater"); throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater");
} }
if (sortedByScoreFull != null) { if (sortedByScoreFull != null) {
return sortedByScoreFull.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return sortedByScoreFull.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} else { } else {
return scoredPaged.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return scoredPaged.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} }
} else { } else {
if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) { if (!FORCE_HUGE_PQ && queryParams.limitLong() < maxInMemoryResultEntries) {
throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater"); throw new UnsupportedOperationException("Allowed limit is " + maxInMemoryResultEntries + " or greater");
} }
if (sortedScoredFull != null) { if (sortedScoredFull != null) {
return sortedScoredFull.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return sortedScoredFull.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} else { } else {
return scoredPaged.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return scoredPaged.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} }
} }
} }
} else { } else {
// Run large/unbounded searches using the continuous multi searcher // Run large/unbounded searches using the continuous multi searcher
return unsortedUnscoredContinuous.collectMulti(Mono.just(indexSearchers), queryParams, keyFieldName, transformer); return unsortedUnscoredContinuous.collectMulti(indexSearchers, queryParams, keyFieldName, transformer);
} }
} }

View File

@ -2,6 +2,7 @@ package it.cavallium.dbengine.lucene.searcher;
import static it.cavallium.dbengine.client.UninterruptibleScheduler.uninterruptibleScheduler; import static it.cavallium.dbengine.client.UninterruptibleScheduler.uninterruptibleScheduler;
import static it.cavallium.dbengine.database.LLUtils.singleOrClose; import static it.cavallium.dbengine.database.LLUtils.singleOrClose;
import static it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite.NO_REWRITE;
import io.netty5.buffer.api.Send; import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
@ -9,6 +10,8 @@ import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearcher;
import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.database.disk.LLIndexSearchers;
import it.cavallium.dbengine.database.disk.LLIndexSearchers.UnshardedIndexSearchers;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.utils.SimpleResource; import it.cavallium.dbengine.utils.SimpleResource;
import java.io.IOException; import java.io.IOException;
import java.io.UncheckedIOException; import java.io.UncheckedIOException;
@ -16,6 +19,7 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.lucene.search.IndexSearcher;
import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
@ -30,61 +34,36 @@ public class CountMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName, String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams); if (queryParams.isSorted() && queryParams.limitLong() > 0) {
} else { throw new UnsupportedOperationException(
queryParamsMono = Mono "Sorted queries are not supported by SimpleUnsortedUnscoredLuceneMultiSearcher");
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) }
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); if (queryParams.needsScores() && queryParams.limitLong() > 0) {
} throw new UnsupportedOperationException(
"Scored queries are not supported by SimpleUnsortedUnscoredLuceneMultiSearcher");
}
return queryParamsMono.flatMap(queryParams2 -> { return Mono.usingWhen(indexSearchersMono, searchers -> Flux
var localQueryParams = getLocalQueryParams(queryParams2); .fromIterable(searchers.llShards())
return Mono .flatMap(searcher -> this.collect(Mono.just(searcher), queryParams, keyFieldName, transformer))
.fromRunnable(() -> { .collectList()
if (queryParams2.isSorted() && queryParams2.limitLong() > 0) { .map(results -> {
throw new UnsupportedOperationException( boolean exactTotalHitsCount = true;
"Sorted queries are not supported by SimpleUnsortedUnscoredLuceneMultiSearcher"); long totalHitsCountValue = 0;
} for (LuceneSearchResult result : results) {
if (queryParams2.needsScores() && queryParams2.limitLong() > 0) { exactTotalHitsCount &= result.totalHitsCount().exact();
throw new UnsupportedOperationException( totalHitsCountValue += result.totalHitsCount().value();
"Scored queries are not supported by SimpleUnsortedUnscoredLuceneMultiSearcher"); result.close();
} }
})
.thenMany(Flux.fromIterable(indexSearchers.llShards()))
.flatMap(searcher -> this.collect(Mono.just(searcher), localQueryParams, keyFieldName, transformer))
.collectList()
.map(results -> {
List<LuceneSearchResult> resultsToDrop = new ArrayList<>(results.size());
List<Flux<LLKeyScore>> resultsFluxes = new ArrayList<>(results.size());
boolean exactTotalHitsCount = true;
long totalHitsCountValue = 0;
for (LuceneSearchResult result : results) {
resultsToDrop.add(result);
resultsFluxes.add(result.results());
exactTotalHitsCount &= result.totalHitsCount().exact();
totalHitsCountValue += result.totalHitsCount().value();
}
var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount);
Flux<LLKeyScore> mergedFluxes = Flux
.merge(resultsFluxes)
.skip(queryParams2.offsetLong())
.take(queryParams2.limitLong(), true);
return new LuceneSearchResult(totalHitsCount, mergedFluxes, () -> { return new LuceneSearchResult(totalHitsCount, Flux.empty(), null);
resultsToDrop.forEach(LLUtils::finalizeResourceNow); })
try { .doOnDiscard(LuceneSearchResult.class, SimpleResource::close), LLUtils::finalizeResource);
indexSearchers.close();
} catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
}
});
});
});
});
} }
private LocalQueryParams getLocalQueryParams(LocalQueryParams queryParams) { private LocalQueryParams getLocalQueryParams(LocalQueryParams queryParams) {
@ -103,31 +82,17 @@ public class CountMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearcherMono, indexSearcher -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewrite(this, indexSearcherMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams);
} else {
queryParamsMono = Mono
.fromCallable(() -> transformer.rewrite(LLIndexSearchers.unsharded(indexSearcher), queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()));
}
return queryParamsMono return Mono.usingWhen(indexSearcherMono, indexSearcher -> Mono.fromCallable(() -> {
.flatMap(queryParams2 -> Mono.fromCallable(() -> { LLUtils.ensureBlocking();
LLUtils.ensureBlocking(); return (long) indexSearcher.getIndexSearcher().count(queryParams.query());
return (long) indexSearcher.getIndexSearcher().count(queryParams2.query()); }).subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())), LLUtils::finalizeResource)
}).subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))) .publishOn(Schedulers.parallel())
.publishOn(Schedulers.parallel()) .transform(TimeoutUtil.timeoutMono(queryParams.timeout()))
.transform(TimeoutUtil.timeoutMono(queryParams.timeout())) .map(count -> new LuceneSearchResult(TotalHitsCount.of(count, true), Flux.empty(), null));
.map(count -> new LuceneSearchResult(TotalHitsCount.of(count, true), Flux.empty(), () -> {
try {
indexSearcher.close();
} catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
}
}));
});
} }
@Override @Override

View File

@ -32,8 +32,7 @@ public interface MultiSearcher extends LocalSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
Mono<LLIndexSearchers> searchers = singleOrClose(indexSearcherMono, indexSearcher -> Mono<LLIndexSearchers> searchers = indexSearcherMono.map(LLIndexSearchers::unsharded);
Mono.just(LLIndexSearchers.unsharded(indexSearcher)));
return this.collectMulti(searchers, queryParams, keyFieldName, transformer); return this.collectMulti(searchers, queryParams, keyFieldName, transformer);
} }

View File

@ -39,45 +39,37 @@ public class PagedLocalSearcher implements LocalSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
if (transformer != GlobalQueryRewrite.NO_REWRITE) {
return LuceneUtils.rewrite(this, indexSearcherMono, queryParams, keyFieldName, transformer);
}
PaginationInfo paginationInfo = getPaginationInfo(queryParams); PaginationInfo paginationInfo = getPaginationInfo(queryParams);
return singleOrClose(indexSearcherMono, indexSearcher -> { var indexSearchersMono = indexSearcherMono.map(LLIndexSearchers::unsharded);
var indexSearchers = LLIndexSearchers.unsharded(indexSearcher);
Mono<LocalQueryParams> queryParamsMono; return singleOrClose(indexSearchersMono, indexSearchers -> this
if (transformer == GlobalQueryRewrite.NO_REWRITE) { // Search first page results
queryParamsMono = Mono.just(queryParams); .searchFirstPage(indexSearchers.shards(), queryParams, paginationInfo)
} else { // Compute the results of the first page
queryParamsMono = Mono .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono,
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) indexSearchers.shards(),
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); keyFieldName,
} queryParams
))
return queryParamsMono.flatMap(queryParams2 -> this // Compute other results
// Search first page results .transform(firstResult -> this.computeOtherResults(firstResult,
.searchFirstPage(indexSearchers.shards(), queryParams2, paginationInfo) indexSearchers.shards(),
// Compute the results of the first page queryParams,
.transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, keyFieldName,
indexSearchers.shards(), () -> {
keyFieldName, try {
queryParams2 indexSearchers.close();
)) } catch (UncheckedIOException e) {
// Compute other results LOG.error(e);
.transform(firstResult -> this.computeOtherResults(firstResult,
indexSearchers.shards(),
queryParams2,
keyFieldName,
() -> {
try {
indexSearcher.close();
} catch (UncheckedIOException e) {
LOG.error(e);
}
} }
)) }
// Ensure that one LuceneSearchResult is always returned ))
.single()); // Ensure that one LuceneSearchResult is always returned
}); .single());
} }
@Override @Override

View File

@ -39,45 +39,31 @@ public class ScoredPagedMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams);
} else {
queryParamsMono = Mono
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams))
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()));
}
return queryParamsMono.flatMap(queryParams2 -> { PaginationInfo paginationInfo = getPaginationInfo(queryParams);
PaginationInfo paginationInfo = getPaginationInfo(queryParams2);
return this return singleOrClose(indexSearchersMono, indexSearchers -> this
// Search first page results // Search first page results
.searchFirstPage(indexSearchers.shards(), queryParams2, paginationInfo) .searchFirstPage(indexSearchers.shards(), queryParams, paginationInfo)
// Compute the results of the first page // Compute the results of the first page
.transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, .transform(firstPageTopDocsMono ->
indexSearchers, this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers, keyFieldName, queryParams
keyFieldName, ))
queryParams2 // Compute other results
)) .map(firstResult -> this.computeOtherResults(firstResult, indexSearchers.shards(), queryParams, keyFieldName,
// Compute other results () -> {
.map(firstResult -> this.computeOtherResults(firstResult, try {
indexSearchers.shards(), indexSearchers.close();
queryParams2, } catch (UncheckedIOException e) {
keyFieldName, LOG.error("Can't close index searchers", e);
() -> { }
try { }
indexSearchers.close(); ))
} catch (UncheckedIOException e) { // Ensure that one LuceneSearchResult is always returned
LOG.error("Can't close index searchers", e); .single());
}
}
))
// Ensure that one LuceneSearchResult is always returned
.single();
});
});
} }
private Sort getSort(LocalQueryParams queryParams) { private Sort getSort(LocalQueryParams queryParams) {

View File

@ -37,31 +37,20 @@ public class SortedByScoreFullMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams); if (queryParams.isSorted() && !queryParams.isSortedByScore()) {
} else { throw new IllegalArgumentException(SortedByScoreFullMultiSearcher.this.getClass().getSimpleName()
queryParamsMono = Mono + " doesn't support sorted queries");
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) }
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); return singleOrClose(indexSearchersMono, indexSearchers -> this
} // Search results
.search(indexSearchers.shards(), queryParams)
return queryParamsMono.flatMap(queryParams2 -> { // Compute the results
if (queryParams2.isSorted() && !queryParams2.isSortedByScore()) { .transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams))
throw new IllegalArgumentException(SortedByScoreFullMultiSearcher.this.getClass().getSimpleName() // Ensure that one LuceneSearchResult is always returned
+ " doesn't support sorted queries"); .single());
}
return this
// Search results
.search(indexSearchers.shards(), queryParams2)
// Compute the results
.transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams2))
// Ensure that one LuceneSearchResult is always returned
.single();
});
});
} }
/** /**
@ -71,7 +60,6 @@ public class SortedByScoreFullMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams) { LocalQueryParams queryParams) {
return Mono return Mono
.fromCallable(() -> { .fromCallable(() -> {
LLUtils.ensureBlocking();
var totalHitsThreshold = queryParams.getTotalHitsThresholdLong(); var totalHitsThreshold = queryParams.getTotalHitsThresholdLong();
return HugePqFullScoreDocCollector.createSharedManager(env, queryParams.limitLong(), totalHitsThreshold); return HugePqFullScoreDocCollector.createSharedManager(env, queryParams.limitLong(), totalHitsThreshold);
}) })
@ -91,7 +79,7 @@ public class SortedByScoreFullMultiSearcher implements MultiSearcher {
collector.close(); collector.close();
throw ex; throw ex;
} }
})) }).subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())))
.collectList() .collectList()
.flatMap(collectors -> Mono.fromCallable(() -> { .flatMap(collectors -> Mono.fromCallable(() -> {
try { try {
@ -104,7 +92,8 @@ public class SortedByScoreFullMultiSearcher implements MultiSearcher {
throw ex; throw ex;
} }
})) }))
); )
.publishOn(Schedulers.parallel());
} }
/** /**

View File

@ -37,24 +37,16 @@ public class SortedScoredFullMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams); return singleOrClose(indexSearchersMono, indexSearchers -> this
} else { // Search results
queryParamsMono = Mono .search(indexSearchers.shards(), queryParams)
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) // Compute the results
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); .transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams))
} // Ensure that one LuceneSearchResult is always returned
.single());
return queryParamsMono.flatMap(queryParams2 -> this
// Search results
.search(indexSearchers.shards(), queryParams2)
// Compute the results
.transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams2))
// Ensure that one LuceneSearchResult is always returned
.single());
});
} }
/** /**
@ -98,7 +90,9 @@ public class SortedScoredFullMultiSearcher implements MultiSearcher {
throw ex; throw ex;
} }
})) }))
); )
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic()))
.publishOn(Schedulers.parallel());
} }
/** /**

View File

@ -37,24 +37,16 @@ public class StandardSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams); return singleOrClose(indexSearchersMono, indexSearchers -> this
} else { // Search results
queryParamsMono = Mono .search(indexSearchers.shards(), queryParams)
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) // Compute the results
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); .transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams))
} // Ensure that one LuceneSearchResult is always returned
.single());
return queryParamsMono.flatMap(queryParams2 -> this
// Search results
.search(indexSearchers.shards(), queryParams2)
// Compute the results
.transform(fullDocsMono -> this.computeResults(fullDocsMono, indexSearchers, keyFieldName, queryParams2))
// Ensure that one LuceneSearchResult is always returned
.single());
});
} }
/** /**

View File

@ -33,42 +33,31 @@ public class UnsortedStreamingMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
@Nullable String keyFieldName, @Nullable String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
return singleOrClose(indexSearchersMono, indexSearchers -> { if (transformer != GlobalQueryRewrite.NO_REWRITE) {
Mono<LocalQueryParams> queryParamsMono; return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
if (transformer == GlobalQueryRewrite.NO_REWRITE) { }
queryParamsMono = Mono.just(queryParams); if (queryParams.isSorted() && queryParams.limitLong() > 0) {
} else { throw new UnsupportedOperationException("Sorted queries are not supported" + " by UnsortedContinuousLuceneMultiSearcher");
queryParamsMono = Mono }
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) var localQueryParams = getLocalQueryParams(queryParams);
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); return singleOrClose(indexSearchersMono, indexSearchers -> Mono.fromCallable(() -> {
} var shards = indexSearchers.shards();
return queryParamsMono.map(queryParams2 -> { Flux<ScoreDoc> scoreDocsFlux = getScoreDocs(localQueryParams, shards);
var localQueryParams = getLocalQueryParams(queryParams2);
if (queryParams2.isSorted() && queryParams2.limitLong() > 0) { Flux<LLKeyScore> resultsFlux = LuceneUtils.convertHits(scoreDocsFlux, shards, keyFieldName, false);
throw new UnsupportedOperationException("Sorted queries are not supported"
+ " by UnsortedContinuousLuceneMultiSearcher"); var totalHitsCount = new TotalHitsCount(0, false);
Flux<LLKeyScore> mergedFluxes = resultsFlux.skip(queryParams.offsetLong()).take(queryParams.limitLong(), true);
return new LuceneSearchResult(totalHitsCount, mergedFluxes, () -> {
try {
indexSearchers.close();
} catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
} }
var shards = indexSearchers.shards();
Flux<ScoreDoc> scoreDocsFlux = getScoreDocs(localQueryParams, shards);
Flux<LLKeyScore> resultsFlux = LuceneUtils.convertHits(scoreDocsFlux, shards, keyFieldName, false);
var totalHitsCount = new TotalHitsCount(0, false);
Flux<LLKeyScore> mergedFluxes = resultsFlux
.skip(queryParams2.offsetLong())
.take(queryParams2.limitLong(), true);
return new LuceneSearchResult(totalHitsCount, mergedFluxes, () -> {
try {
indexSearchers.close();
} catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
}
});
}); });
}); }));
} }
private Flux<ScoreDoc> getScoreDocs(LocalQueryParams localQueryParams, List<IndexSearcher> shards) { private Flux<ScoreDoc> getScoreDocs(LocalQueryParams localQueryParams, List<IndexSearcher> shards) {

View File

@ -15,6 +15,7 @@ import it.cavallium.dbengine.client.Sort;
import it.cavallium.dbengine.client.query.current.data.MatchAllDocsQuery; import it.cavallium.dbengine.client.query.current.data.MatchAllDocsQuery;
import it.cavallium.dbengine.database.LLLuceneIndex; import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLScoreMode; import it.cavallium.dbengine.database.LLScoreMode;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.LLTempHugePqEnv; import it.cavallium.dbengine.database.disk.LLTempHugePqEnv;
import it.cavallium.dbengine.lucene.searcher.AdaptiveLocalSearcher; import it.cavallium.dbengine.lucene.searcher.AdaptiveLocalSearcher;
import it.cavallium.dbengine.lucene.searcher.AdaptiveMultiSearcher; import it.cavallium.dbengine.lucene.searcher.AdaptiveMultiSearcher;
@ -136,7 +137,11 @@ public class TestLuceneIndex {
index.updateDocument("test-key-13", "2111").block(); index.updateDocument("test-key-13", "2111").block();
index.updateDocument("test-key-14", "2999").block(); index.updateDocument("test-key-14", "2999").block();
index.updateDocument("test-key-15", "3902").block(); index.updateDocument("test-key-15", "3902").block();
Flux.range(1, 1000).concatMap(i -> index.updateDocument("test-key-" + (15 + i), "" + i)).blockLast(); Flux
.range(1, 1000)
.concatMap(i -> index.updateDocument("test-key-" + (15 + i), "" + i))
.transform(LLUtils::handleDiscard)
.blockLast();
tempDb.swappableLuceneSearcher().setSingle(new CountMultiSearcher()); tempDb.swappableLuceneSearcher().setSingle(new CountMultiSearcher());
tempDb.swappableLuceneSearcher().setMulti(new CountMultiSearcher()); tempDb.swappableLuceneSearcher().setMulti(new CountMultiSearcher());
assertCount(index, 1000 + 15); assertCount(index, 1000 + 15);

View File

@ -10,6 +10,7 @@ import it.cavallium.dbengine.database.LLKeyScore;
import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearcher;
import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.database.disk.LLIndexSearchers;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite; import it.cavallium.dbengine.lucene.searcher.GlobalQueryRewrite;
import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams;
import it.cavallium.dbengine.lucene.searcher.LocalSearcher; import it.cavallium.dbengine.lucene.searcher.LocalSearcher;
@ -42,64 +43,51 @@ public class UnsortedUnscoredSimpleMultiSearcher implements MultiSearcher {
LocalQueryParams queryParams, LocalQueryParams queryParams,
String keyFieldName, String keyFieldName,
GlobalQueryRewrite transformer) { GlobalQueryRewrite transformer) {
if (transformer != NO_REWRITE) {
return LuceneUtils.rewriteMulti(this, indexSearchersMono, queryParams, keyFieldName, transformer);
}
if (queryParams.isSorted() && queryParams.limitLong() > 0) {
throw new UnsupportedOperationException(
"Sorted queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher");
}
if (queryParams.needsScores() && queryParams.limitLong() > 0) {
throw new UnsupportedOperationException(
"Scored queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher");
}
return singleOrClose(indexSearchersMono, indexSearchers -> { return singleOrClose(indexSearchersMono, indexSearchers -> {
Mono<LocalQueryParams> queryParamsMono; var localQueryParams = getLocalQueryParams(queryParams);
if (transformer == NO_REWRITE) { return Flux
queryParamsMono = Mono.just(queryParams); .fromIterable(indexSearchers.llShards())
} else { .flatMap(searcher -> localSearcher.collect(Mono.just(searcher), localQueryParams, keyFieldName, transformer))
queryParamsMono = Mono .collectList()
.fromCallable(() -> transformer.rewrite(indexSearchers, queryParams)) .map(results -> {
.subscribeOn(uninterruptibleScheduler(Schedulers.boundedElastic())); List<LuceneSearchResult> resultsToDrop = new ArrayList<>(results.size());
} List<Flux<LLKeyScore>> resultsFluxes = new ArrayList<>(results.size());
boolean exactTotalHitsCount = true;
long totalHitsCountValue = 0;
for (LuceneSearchResult result : results) {
resultsToDrop.add(result);
resultsFluxes.add(result.results());
exactTotalHitsCount &= result.totalHitsCount().exact();
totalHitsCountValue += result.totalHitsCount().value();
}
return queryParamsMono.flatMap(queryParams2 -> { var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount);
var localQueryParams = getLocalQueryParams(queryParams2); Flux<LLKeyScore> mergedFluxes = Flux
return Flux .merge(resultsFluxes)
.fromIterable(indexSearchers.llShards()) .skip(queryParams.offsetLong())
.flatMap(searcher -> .take(queryParams.limitLong(), true);
localSearcher.collect(Mono.just(searcher), localQueryParams, keyFieldName, transformer))
.collectList()
.map(results -> {
List<LuceneSearchResult> resultsToDrop = new ArrayList<>(results.size());
List<Flux<LLKeyScore>> resultsFluxes = new ArrayList<>(results.size());
boolean exactTotalHitsCount = true;
long totalHitsCountValue = 0;
for (LuceneSearchResult result : results) {
resultsToDrop.add(result);
resultsFluxes.add(result.results());
exactTotalHitsCount &= result.totalHitsCount().exact();
totalHitsCountValue += result.totalHitsCount().value();
}
var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); return new LuceneSearchResult(totalHitsCount, mergedFluxes, () -> {
Flux<LLKeyScore> mergedFluxes = Flux resultsToDrop.forEach(SimpleResource::close);
.merge(resultsFluxes) try {
.skip(queryParams2.offsetLong()) indexSearchers.close();
.take(queryParams2.limitLong(), true); } catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
return new LuceneSearchResult(totalHitsCount, mergedFluxes, () -> { }
resultsToDrop.forEach(SimpleResource::close); });
try { });
indexSearchers.close();
} catch (UncheckedIOException e) {
LOG.error("Can't close index searchers", e);
}
});
})
.doFirst(() -> {
LLUtils.ensureBlocking();
if (queryParams2.isSorted() && queryParams2.limitLong() > 0) {
throw new UnsupportedOperationException("Sorted queries are not supported"
+ " by SimpleUnsortedUnscoredLuceneMultiSearcher");
}
if (queryParams2.needsScores() && queryParams2.limitLong() > 0) {
throw new UnsupportedOperationException("Scored queries are not supported"
+ " by SimpleUnsortedUnscoredLuceneMultiSearcher");
}
});
}
);
}); });
} }

View File

@ -4,6 +4,7 @@ import static org.junit.jupiter.api.Assertions.*;
import it.cavallium.dbengine.client.query.QueryUtils; import it.cavallium.dbengine.client.query.QueryUtils;
import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.IndexSearcherManager; import it.cavallium.dbengine.database.disk.IndexSearcherManager;
import it.cavallium.dbengine.database.disk.LLTempHugePqEnv; import it.cavallium.dbengine.database.disk.LLTempHugePqEnv;
import it.cavallium.dbengine.lucene.LLScoreDoc; import it.cavallium.dbengine.lucene.LLScoreDoc;
@ -78,7 +79,13 @@ public class HugePqFullScoreDocCollectorTest {
.toList(); .toList();
try (var collector = HugePqFullScoreDocCollector.create(env, 20)) { try (var collector = HugePqFullScoreDocCollector.create(env, 20)) {
searcher.search(luceneQuery, collector); searcher.search(luceneQuery, collector);
var docs = collector.fullDocs().iterate().collectList().blockOptional().orElseThrow(); var docs = collector
.fullDocs()
.iterate()
.collectList()
.transform(LLUtils::handleDiscard)
.blockOptional()
.orElseThrow();
System.out.println("Expected docs:"); System.out.println("Expected docs:");
for (LLScoreDoc expectedDoc : expectedDocs) { for (LLScoreDoc expectedDoc : expectedDocs) {
System.out.println(expectedDoc); System.out.println(expectedDoc);
@ -146,23 +153,24 @@ public class HugePqFullScoreDocCollectorTest {
.map(scoreDoc -> new LLScoreDoc(scoreDoc.doc, scoreDoc.score, scoreDoc.shardIndex)) .map(scoreDoc -> new LLScoreDoc(scoreDoc.doc, scoreDoc.score, scoreDoc.shardIndex))
.toList(); .toList();
var collectorManager = HugePqFullScoreDocCollector.createSharedManager(env, 20, Integer.MAX_VALUE); var collectorManager = HugePqFullScoreDocCollector.createSharedManager(env, 20, Integer.MAX_VALUE);
var collector1 = collectorManager.newCollector(); try (var collector1 = collectorManager.newCollector();
var collector2 = collectorManager.newCollector(); var collector2 = collectorManager.newCollector()) {
shardSearcher1.search(luceneQuery, collector1); shardSearcher1.search(luceneQuery, collector1);
shardSearcher2.search(luceneQuery, collector2); shardSearcher2.search(luceneQuery, collector2);
try (var results = collectorManager.reduce(List.of(collector1, collector2))) { try (var results = collectorManager.reduce(List.of(collector1, collector2))) {
var docs = results.iterate().collectList().blockOptional().orElseThrow(); var docs = results.iterate().collectList().blockOptional().orElseThrow();
System.out.println("Expected docs:"); System.out.println("Expected docs:");
for (LLScoreDoc expectedDoc : expectedDocs) { for (LLScoreDoc expectedDoc : expectedDocs) {
System.out.println(expectedDoc); System.out.println(expectedDoc);
}
System.out.println("");
System.out.println("Obtained docs:");
for (LLScoreDoc doc : docs) {
System.out.println(doc);
}
assertEquals(expectedDocs, docs.stream().map(elem -> new LLScoreDoc(elem.doc(), elem.score(), -1)).toList());
assertEquals(expectedTotalHits, new TotalHitsCount(results.totalHits().value, results.totalHits().relation == Relation.EQUAL_TO));
} }
System.out.println("");
System.out.println("Obtained docs:");
for (LLScoreDoc doc : docs) {
System.out.println(doc);
}
assertEquals(expectedDocs, docs.stream().map(elem -> new LLScoreDoc(elem.doc(), elem.score(), -1)).toList());
assertEquals(expectedTotalHits, new TotalHitsCount(results.totalHits().value, results.totalHits().relation == Relation.EQUAL_TO));
} }
} }
} }