From 8bc0284f270303d82d5e1a377b056888e05fb5b3 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Sat, 18 Sep 2021 18:34:21 +0200 Subject: [PATCH 01/23] First safe search commit --- .../dbengine/database/LLLuceneIndex.java | 14 +- .../database/LLSearchResultShard.java | 75 ++++--- .../cavallium/dbengine/database/LLUtils.java | 7 + .../database/disk/CachedIndexSearcher.java | 106 --------- .../disk/CachedIndexSearcherManager.java | 89 +++----- .../database/disk/IndexSearcherManager.java | 25 +++ .../database/disk/LLIndexSearcher.java | 94 ++++++++ .../database/disk/LLLocalLuceneIndex.java | 102 ++++----- .../disk/LLLocalMultiLuceneIndex.java | 24 --- .../dbengine/lucene/LuceneUtils.java | 34 +-- .../searcher/AdaptiveLuceneLocalSearcher.java | 28 +-- .../searcher/AdaptiveLuceneMultiSearcher.java | 14 +- .../searcher/CountLuceneLocalSearcher.java | 37 ++-- .../searcher/CountLuceneMultiSearcher.java | 115 ++++++---- .../lucene/searcher/IndexSearchers.java | 137 ++++++++++-- .../lucene/searcher/LocalLuceneWrapper.java | 28 --- .../lucene/searcher/LuceneLocalSearcher.java | 15 +- .../lucene/searcher/LuceneMultiSearcher.java | 36 ++-- .../lucene/searcher/LuceneSearchResult.java | 65 ++++-- .../lucene/searcher/LuceneShardSearcher.java | 26 --- .../searcher/ScoredLuceneMultiSearcher.java | 3 +- .../ScoredSimpleLuceneShardSearcher.java | 11 +- .../searcher/SimpleLuceneLocalSearcher.java | 203 ++++++++++-------- .../UnscoredPagedLuceneMultiSearcher.java | 2 +- .../UnscoredPagedLuceneShardSearcher.java | 3 +- ...UnsortedContinuousLuceneMultiSearcher.java | 8 +- 26 files changed, 706 insertions(+), 595 deletions(-) delete mode 100644 src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcher.java create mode 100644 src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java create mode 100644 src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/LocalLuceneWrapper.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneShardSearcher.java diff --git a/src/main/java/it/cavallium/dbengine/database/LLLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/LLLuceneIndex.java index 4180025..2d2b58f 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/LLLuceneIndex.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database; +import io.net5.buffer.api.Send; import it.cavallium.data.generator.nativedata.Nullablefloat; import it.cavallium.dbengine.client.query.current.data.NoSort; import it.cavallium.dbengine.client.query.current.data.Query; @@ -40,7 +41,7 @@ public interface LLLuceneIndex extends LLSnapshottable { * The additional query will be used with the moreLikeThis query: "mltQuery AND additionalQuery" * @return the collection has one or more flux */ - Mono moreLikeThis(@Nullable LLSnapshot snapshot, + Mono> moreLikeThis(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName, Flux>> mltDocumentFields); @@ -50,13 +51,18 @@ public interface LLLuceneIndex extends LLSnapshottable { * returned can be at most limit * 15 * @return the collection has one or more flux */ - Mono search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName); + Mono> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName); default Mono count(@Nullable LLSnapshot snapshot, Query query) { QueryParams params = QueryParams.of(query, 0, 0, Nullablefloat.empty(), NoSort.of(), ScoreMode.of(false, false)); return Mono.from(this.search(snapshot, params, null) - .flatMap(llSearchResultShard -> llSearchResultShard.release().thenReturn(llSearchResultShard.totalHitsCount())) - .defaultIfEmpty(TotalHitsCount.of(0, true))); + .map(llSearchResultShardToReceive -> { + try (var llSearchResultShard = llSearchResultShardToReceive.receive()) { + return llSearchResultShard.totalHitsCount(); + } + }) + .defaultIfEmpty(TotalHitsCount.of(0, true)) + ).doOnDiscard(Send.class, Send::close); } boolean isLowMemoryMode(); diff --git a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java index 7291493..f3db927 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java +++ b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java @@ -1,46 +1,41 @@ package it.cavallium.dbengine.database; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; -import it.cavallium.dbengine.lucene.searcher.LuceneSearchResult; import java.util.Objects; import org.warp.commonutils.log.Logger; import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; -public final class LLSearchResultShard { +public final class LLSearchResultShard extends ResourceSupport { private static final Logger logger = LoggerFactory.getLogger(LLSearchResultShard.class); - private volatile boolean releaseCalled; + private Flux results; + private TotalHitsCount totalHitsCount; - private final Flux results; - private final TotalHitsCount totalHitsCount; - private final Mono release; - - public LLSearchResultShard(Flux results, TotalHitsCount totalHitsCount, Mono release) { + public LLSearchResultShard(Flux results, TotalHitsCount totalHitsCount, Drop drop) { + super(new LLSearchResultShard.CloseOnDrop(drop)); this.results = results; this.totalHitsCount = totalHitsCount; - this.release = Mono.fromRunnable(() -> { - if (releaseCalled) { - logger.warn(this.getClass().getName() + "::release has been called twice!"); - } - releaseCalled = true; - }).then(release); } public Flux results() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("LLSearchResultShard must be owned to be used")); + } return results; } public TotalHitsCount totalHitsCount() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("LLSearchResultShard must be owned to be used")); + } return totalHitsCount; } - public Mono release() { - return release; - } - @Override public boolean equals(Object obj) { if (obj == this) @@ -48,28 +43,48 @@ public final class LLSearchResultShard { if (obj == null || obj.getClass() != this.getClass()) return false; var that = (LLSearchResultShard) obj; - return Objects.equals(this.results, that.results) && Objects.equals(this.totalHitsCount, that.totalHitsCount) - && Objects.equals(this.release, that.release); + return Objects.equals(this.results, that.results) && Objects.equals(this.totalHitsCount, that.totalHitsCount); } @Override public int hashCode() { - return Objects.hash(results, totalHitsCount, release); + return Objects.hash(results, totalHitsCount); } @Override public String toString() { - return "LLSearchResultShard[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ", " + "release=" - + release + ']'; + return "LLSearchResultShard[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ']'; } - @SuppressWarnings("deprecation") @Override - protected void finalize() throws Throwable { - if (!releaseCalled) { - logger.warn(this.getClass().getName() + "::release has not been called before class finalization!"); - } - super.finalize(); + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); } + @Override + protected Owned prepareSend() { + var results = this.results; + var totalHitsCount = this.totalHitsCount; + makeInaccessible(); + return drop -> new LLSearchResultShard(results, totalHitsCount, drop); + } + + private void makeInaccessible() { + this.results = null; + this.totalHitsCount = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(LLSearchResultShard obj) { + delegate.drop(obj); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index d4f47b2..2a729f4 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -48,6 +48,7 @@ import org.slf4j.Marker; import org.slf4j.MarkerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; import reactor.util.function.Tuple2; import reactor.util.function.Tuple3; @@ -337,6 +338,12 @@ public class LLUtils { } } + public static void ensureBlocking() { + if (Schedulers.isInNonBlockingThread()) { + throw new UnsupportedOperationException("Called collect in a nonblocking thread"); + } + } + public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull diff --git a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcher.java deleted file mode 100644 index 061826c..0000000 --- a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcher.java +++ /dev/null @@ -1,106 +0,0 @@ -package it.cavallium.dbengine.database.disk; - -import java.io.IOException; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.SearcherManager; -import org.jetbrains.annotations.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class CachedIndexSearcher { - - private static final Logger logger = LoggerFactory.getLogger(CachedIndexSearcher.class); - - private final IndexSearcher indexSearcher; - private final SearcherManager associatedSearcherManager; - private final Runnable afterFinalization; - private boolean inCache = true; - private int usages = 0; - - public CachedIndexSearcher(IndexSearcher indexSearcher, - @Nullable SearcherManager associatedSearcherManager, - @Nullable Runnable afterFinalization) { - this.indexSearcher = indexSearcher; - this.associatedSearcherManager = associatedSearcherManager; - this.afterFinalization = afterFinalization; - } - - public void incUsage() { - synchronized (this) { - usages++; - } - } - - public void decUsage() throws IOException { - synchronized (this) { - if (usages > 0) { - usages--; - if (mustClose()) { - try { - close(); - } finally { - if (afterFinalization != null) afterFinalization.run(); - } - } - } - } - } - - public void removeFromCache() throws IOException { - synchronized (this) { - if (inCache) { - inCache = false; - if (mustClose()) { - try { - close(); - } finally { - if (afterFinalization != null) afterFinalization.run(); - } - } - } - } - } - - private void close() throws IOException { - if (associatedSearcherManager != null) { - associatedSearcherManager.release(indexSearcher); - } - } - - private boolean mustClose() { - return !this.inCache && this.usages == 0; - } - - public IndexReader getIndexReader() { - return indexSearcher.getIndexReader(); - } - - public IndexSearcher getIndexSearcher() { - return indexSearcher; - } - - @SuppressWarnings("deprecation") - @Override - protected void finalize() throws Throwable { - boolean failedToRelease = false; - if (usages > 0) { - failedToRelease = true; - logger.error("A cached index searcher has been garbage collected, but " - + usages + " usages have not been released"); - } - if (inCache) { - failedToRelease = true; - logger.error("A cached index searcher has been garbage collected, but it's marked" - + " as still actively cached"); - } - if (failedToRelease) { - try { - this.close(); - } catch (Throwable ex) { - logger.warn("Error when closing cached index searcher", ex); - } - } - super.finalize(); - } -} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java b/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java index d467947..20f11d9 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java @@ -3,16 +3,14 @@ package it.cavallium.dbengine.database.disk; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.database.LLSnapshot; import java.io.IOException; import java.time.Duration; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.LockSupport; import java.util.function.Function; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.search.IndexSearcher; @@ -30,7 +28,7 @@ import reactor.core.publisher.Sinks; import reactor.core.publisher.Sinks.Empty; import reactor.core.scheduler.Schedulers; -public class CachedIndexSearcherManager { +public class CachedIndexSearcherManager implements IndexSearcherManager { private static final Logger logger = LoggerFactory.getLogger(CachedIndexSearcherManager.class); @@ -41,8 +39,8 @@ public class CachedIndexSearcherManager { private final Phaser activeSearchers = new Phaser(1); private final Phaser activeRefreshes = new Phaser(1); - private final LoadingCache> cachedSnapshotSearchers; - private final Mono cachedMainSearcher; + private final LoadingCache>> cachedSnapshotSearchers; + private final Mono> cachedMainSearcher; private final Empty closeRequested = Sinks.empty(); private final Empty refresherClosed = Sinks.empty(); @@ -84,7 +82,7 @@ public class CachedIndexSearcherManager { .maximumSize(3) .build(new CacheLoader<>() { @Override - public Mono load(@NotNull LLSnapshot snapshot) { + public Mono> load(@NotNull LLSnapshot snapshot) { return CachedIndexSearcherManager.this.generateCachedSearcher(snapshot); } }); @@ -129,7 +127,11 @@ public class CachedIndexSearcherManager { })).cache(); } - private Mono generateCachedSearcher(@Nullable LLSnapshot snapshot) { + private Mono> generateCachedSearcher(@Nullable LLSnapshot snapshot) { + var onClose = this.closeRequested.asMono(); + var onQueryRefresh = Mono.delay(queryRefreshDebounceTime).then(); + var onInvalidateCache = Mono.firstWithSignal(onClose, onQueryRefresh); + return Mono.fromCallable(() -> { activeSearchers.register(); IndexSearcher indexSearcher; @@ -142,34 +144,20 @@ public class CachedIndexSearcherManager { indexSearcher = snapshotsManager.resolveSnapshot(snapshot).getIndexSearcher(); associatedSearcherManager = null; } - AtomicBoolean alreadyDeregistered = new AtomicBoolean(false); - return new CachedIndexSearcher(indexSearcher, associatedSearcherManager, - () -> { - // This shouldn't happen more than once, - // but I put this AtomicBoolean to be sure that this will NEVER happen more than once. - if (alreadyDeregistered.compareAndSet(false, true)) { - activeSearchers.arriveAndDeregister(); - } else { - logger.error("Disposed CachedIndexSearcher twice! This is an implementation bug!"); - } - } - ); + return new LLIndexSearcher(indexSearcher, associatedSearcherManager, this::dropCachedIndexSearcher); }) - .cacheInvalidateWhen(indexSearcher -> Mono - .firstWithSignal( - this.closeRequested.asMono(), - Mono.delay(queryRefreshDebounceTime).then() - ), - indexSearcher -> { - try { - // Mark as removed from cache - indexSearcher.removeFromCache(); - } catch (Exception ex) { - logger.error("Failed to release an old cached IndexSearcher", ex); - } - }); + .cacheInvalidateWhen(indexSearcher -> onInvalidateCache, ResourceSupport::close) + .map(searcher -> searcher.copy(this::dropCachedIndexSearcher).send()) + .takeUntilOther(onClose) + .doOnDiscard(ResourceSupport.class, ResourceSupport::close); } + private void dropCachedIndexSearcher(LLIndexSearcher cachedIndexSearcher) { + // This shouldn't happen more than once per searcher. + activeSearchers.arriveAndDeregister(); + } + + @Override public void maybeRefreshBlocking() throws IOException { try { activeRefreshes.register(); @@ -181,6 +169,7 @@ public class CachedIndexSearcherManager { } } + @Override public void maybeRefresh() throws IOException { try { activeRefreshes.register(); @@ -192,30 +181,26 @@ public class CachedIndexSearcherManager { } } + @Override public Flux searchMany(@Nullable LLSnapshot snapshot, Function> searcherFunction) { return Flux.usingWhen( - this.captureIndexSearcher(snapshot), + this.retrieveSearcher(snapshot).map(Send::receive), indexSearcher -> searcherFunction.apply(indexSearcher.getIndexSearcher()), - this::releaseUsedIndexSearcher + cachedIndexSearcher -> Mono.fromRunnable(cachedIndexSearcher::close) ); } + @Override public Mono search(@Nullable LLSnapshot snapshot, Function> searcherFunction) { return Mono.usingWhen( - this.captureIndexSearcher(snapshot), + this.retrieveSearcher(snapshot).map(Send::receive), indexSearcher -> searcherFunction.apply(indexSearcher.getIndexSearcher()), - this::releaseUsedIndexSearcher + cachedIndexSearcher -> Mono.fromRunnable(cachedIndexSearcher::close) ); } - public Mono captureIndexSearcher(@Nullable LLSnapshot snapshot) { - return this - .retrieveCachedIndexSearcher(snapshot) - // Increment reference count - .doOnNext(CachedIndexSearcher::incUsage); - } - - private Mono retrieveCachedIndexSearcher(LLSnapshot snapshot) { + @Override + public Mono> retrieveSearcher(@Nullable LLSnapshot snapshot) { if (snapshot == null) { return this.cachedMainSearcher; } else { @@ -223,17 +208,7 @@ public class CachedIndexSearcherManager { } } - public Mono releaseUsedIndexSearcher(CachedIndexSearcher indexSearcher) { - return Mono.fromRunnable(() -> { - try { - // Decrement reference count - indexSearcher.decUsage(); - } catch (Exception ex) { - logger.error("Failed to release an used IndexSearcher", ex); - } - }); - } - + @Override public Mono close() { return closeMono; } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java b/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java new file mode 100644 index 0000000..c86f222 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java @@ -0,0 +1,25 @@ +package it.cavallium.dbengine.database.disk; + +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.LLSnapshot; +import java.io.IOException; +import java.util.function.Function; +import org.apache.lucene.search.IndexSearcher; +import org.jetbrains.annotations.Nullable; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public interface IndexSearcherManager { + + void maybeRefreshBlocking() throws IOException; + + void maybeRefresh() throws IOException; + + Flux searchMany(@Nullable LLSnapshot snapshot, Function> searcherFunction); + + Mono search(@Nullable LLSnapshot snapshot, Function> searcherFunction); + + Mono> retrieveSearcher(@Nullable LLSnapshot snapshot); + + Mono close(); +} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java new file mode 100644 index 0000000..7bf4f68 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -0,0 +1,94 @@ +package it.cavallium.dbengine.database.disk; + +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.internal.ResourceSupport; +import java.io.IOException; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SearcherManager; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LLIndexSearcher extends ResourceSupport { + + private static final Logger logger = LoggerFactory.getLogger(LLIndexSearcher.class); + + private IndexSearcher indexSearcher; + private SearcherManager associatedSearcherManager; + + public LLIndexSearcher(IndexSearcher indexSearcher, + @Nullable SearcherManager associatedSearcherManager, + Drop drop) { + super(new LLIndexSearcher.CloseOnDrop(drop)); + this.indexSearcher = indexSearcher; + this.associatedSearcherManager = associatedSearcherManager; + } + + public IndexReader getIndexReader() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); + } + return indexSearcher.getIndexReader(); + } + + public IndexSearcher getIndexSearcher() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); + } + return indexSearcher; + } + + public LLIndexSearcher copy(Drop drop) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); + } + var copyIndexSearcher = this.indexSearcher; + if (associatedSearcherManager != null) { + copyIndexSearcher.getIndexReader().incRef(); + } + return new LLIndexSearcher(copyIndexSearcher, associatedSearcherManager, new CloseOnDrop(drop)); + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + var indexSearcher = this.indexSearcher; + var associatedSearcherManager = this.associatedSearcherManager; + makeInaccessible(); + return drop -> new LLIndexSearcher(indexSearcher, associatedSearcherManager, drop); + } + + private void makeInaccessible() { + this.indexSearcher = null; + this.associatedSearcherManager = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(LLIndexSearcher obj) { + try { + if (obj.associatedSearcherManager != null) { + if (obj.indexSearcher.getIndexReader().getRefCount() > 0) { + obj.associatedSearcherManager.release(obj.indexSearcher); + } + } + } catch (IOException e) { + logger.error("Failed to drop CachedIndexSearcher", e); + } + delegate.drop(obj); + } + } +} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java index 988922a..31d3034 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java @@ -3,6 +3,7 @@ package it.cavallium.dbengine.database.disk; import static it.cavallium.dbengine.database.LLUtils.MARKER_LUCENE; import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.DirectIOOptions; import it.cavallium.dbengine.client.IndicizerAnalyzers; import it.cavallium.dbengine.client.IndicizerSimilarities; @@ -21,10 +22,9 @@ import it.cavallium.dbengine.lucene.LuceneUtils; import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneLocalSearcher; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LuceneLocalSearcher; -import it.cavallium.dbengine.lucene.searcher.LuceneShardSearcher; +import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher; import java.io.IOException; import java.nio.file.Path; -import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; @@ -73,21 +73,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { * There is only a single thread globally to not overwhelm the disk with * concurrent commits or concurrent refreshes. */ - private static final Scheduler luceneHeavyTasksScheduler = Schedulers.newBoundedElastic(1, - Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, - "lucene", - Integer.MAX_VALUE, - true - ); - // Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks - protected final Scheduler luceneSearcherScheduler = LuceneUtils.newLuceneSearcherScheduler(false); - // Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks - private static final Scheduler luceneWriterScheduler = Schedulers.boundedElastic(); + private static final Scheduler luceneHeavyTasksScheduler = Schedulers.single(Schedulers.boundedElastic()); private final String luceneIndexName; private final IndexWriter indexWriter; private final SnapshotsManager snapshotsManager; - private final CachedIndexSearcherManager searcherManager; + private final IndexSearcherManager searcherManager; private final Similarity similarity; private final Directory directory; private final boolean lowMemory; @@ -166,7 +157,8 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { if (luceneOptions.nrtCachingOptions().isPresent()) { NRTCachingOptions nrtCachingOptions = luceneOptions.nrtCachingOptions().get(); - directory = new NRTCachingDirectory(directory, nrtCachingOptions.maxMergeSizeMB(), nrtCachingOptions.maxCachedMB()); + directory = new NRTCachingDirectory(directory, nrtCachingOptions.maxMergeSizeMB(), + nrtCachingOptions.maxCachedMB()); } this.directory = directory; @@ -177,7 +169,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { this.lowMemory = lowMemory; this.similarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities); - IndexWriterConfig indexWriterConfig = new IndexWriterConfig(LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers)); + var indexWriterConfig = new IndexWriterConfig(LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers)); indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND); indexWriterConfig.setIndexDeletionPolicy(snapshotter); indexWriterConfig.setCommitOnClose(true); @@ -241,13 +233,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { return Mono.fromCallable(() -> { activeTasks.register(); try { - //noinspection BlockingMethodInNonBlockingContext indexWriter.addDocument(LLUtils.toDocument(doc)); return null; } finally { activeTasks.arriveAndDeregister(); } - }).subscribeOn(luceneWriterScheduler); + }).subscribeOn(Schedulers.boundedElastic()); } @Override @@ -258,13 +249,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { .fromCallable(() -> { activeTasks.register(); try { - //noinspection BlockingMethodInNonBlockingContext indexWriter.addDocuments(LLUtils.toDocumentsFromEntries(documentsList)); return null; } finally { activeTasks.arriveAndDeregister(); } - }).subscribeOn(luceneWriterScheduler) + }).subscribeOn(Schedulers.boundedElastic()) ); } @@ -274,13 +264,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { return Mono.fromCallable(() -> { activeTasks.register(); try { - //noinspection BlockingMethodInNonBlockingContext indexWriter.deleteDocuments(LLUtils.toTerm(id)); return null; } finally { activeTasks.arriveAndDeregister(); } - }).subscribeOn(luceneWriterScheduler); + }).subscribeOn(Schedulers.boundedElastic()); } @Override @@ -288,13 +277,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { return Mono.fromCallable(() -> { activeTasks.register(); try { - //noinspection BlockingMethodInNonBlockingContext indexWriter.updateDocument(LLUtils.toTerm(id), LLUtils.toDocument(document)); } finally { activeTasks.arriveAndDeregister(); } return null; - }).subscribeOn(luceneWriterScheduler); + }).subscribeOn(Schedulers.boundedElastic()); } @Override @@ -310,7 +298,6 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { for (Entry entry : documentsMap.entrySet()) { LLTerm key = entry.getKey(); LLDocument value = entry.getValue(); - //noinspection BlockingMethodInNonBlockingContext indexWriter.updateDocument(LLUtils.toTerm(key), LLUtils.toDocument(value)); } return null; @@ -318,7 +305,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { activeTasks.arriveAndDeregister(); } }) - .subscribeOn(luceneWriterScheduler); + .subscribeOn(Schedulers.boundedElastic()); } @Override @@ -340,35 +327,32 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { } @Override - public Mono moreLikeThis(@Nullable LLSnapshot snapshot, + public Mono> moreLikeThis(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName, Flux>> mltDocumentFieldsFlux) { return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) - .flatMap(modifiedLocalQuery -> searcherManager.captureIndexSearcher(snapshot) - .flatMap(indexSearcher -> { - Mono releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); - return localSearcher - .collect(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, keyFieldName, luceneSearcherScheduler) - .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())) - .onErrorResume(ex -> releaseMono.then(Mono.error(ex))); - }) - ); + .flatMap(modifiedLocalQuery -> searcherManager + .retrieveSearcher(snapshot) + .flatMap(indexSearcher -> localSearcher.collect(indexSearcher, modifiedLocalQuery, keyFieldName)) + ) + .map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); + }) + .doOnDiscard(Send.class, Send::close); } public Mono distributedMoreLikeThis(@Nullable LLSnapshot snapshot, QueryParams queryParams, Flux>> mltDocumentFieldsFlux, - LuceneShardSearcher shardSearcher) { + LuceneMultiSearcher shardSearcher) { return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) - .flatMap(modifiedLocalQuery -> searcherManager.captureIndexSearcher(snapshot) - .flatMap(indexSearcher -> { - Mono releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); - return shardSearcher - .searchOn(indexSearcher.getIndexSearcher(), releaseMono, modifiedLocalQuery, luceneSearcherScheduler) - .onErrorResume(ex -> releaseMono.then(Mono.error(ex))); - }) - ); + .flatMap(modifiedLocalQuery -> searcherManager + .retrieveSearcher(snapshot) + .flatMap(indexSearcher -> shardSearcher.searchOn(indexSearcher, modifiedLocalQuery)) + ) + .doOnDiscard(Send.class, Send::close); } public Mono getMoreLikeThisQuery(@Nullable LLSnapshot snapshot, @@ -437,27 +421,27 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { } @Override - public Mono search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) { + public Mono> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, + String keyFieldName) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - return searcherManager.captureIndexSearcher(snapshot).flatMap(indexSearcher -> { - Mono releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); - return localSearcher - .collect(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, keyFieldName, luceneSearcherScheduler) - .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())) - .onErrorResume(ex -> releaseMono.then(Mono.error(ex))); - }); + return searcherManager + .retrieveSearcher(snapshot) + .flatMap(indexSearcher -> localSearcher.collect(indexSearcher, localQueryParams, keyFieldName)) + .map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); + }) + .doOnDiscard(Send.class, Send::close); } public Mono distributedSearch(@Nullable LLSnapshot snapshot, QueryParams queryParams, - LuceneShardSearcher shardSearcher) { + LuceneMultiSearcher shardSearcher) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - return searcherManager.captureIndexSearcher(snapshot) - .flatMap(indexSearcher -> { - Mono releaseMono = searcherManager.releaseUsedIndexSearcher(indexSearcher); - return shardSearcher.searchOn(indexSearcher.getIndexSearcher(), releaseMono, localQueryParams, luceneSearcherScheduler) - .onErrorResume(ex -> releaseMono.then(Mono.error(ex))); - }); + return searcherManager + .retrieveSearcher(snapshot) + .flatMap(indexSearcher -> shardSearcher.searchOn(indexSearcher, localQueryParams)) + .doOnDiscard(Send.class, Send::close); } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java index e7035b5..e1f22de 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java @@ -1,29 +1,18 @@ package it.cavallium.dbengine.database.disk; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader.InvalidCacheLoadException; import it.cavallium.dbengine.client.IndicizerAnalyzers; import it.cavallium.dbengine.client.IndicizerSimilarities; -import it.cavallium.dbengine.client.LuceneIndex; import it.cavallium.dbengine.client.LuceneOptions; import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.database.LLDocument; import it.cavallium.dbengine.database.LLLuceneIndex; -import it.cavallium.dbengine.database.LLSearchResult; import it.cavallium.dbengine.database.LLSearchResultShard; import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.LLTerm; import it.cavallium.dbengine.lucene.LuceneUtils; -import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer; -import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity; import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneMultiSearcher; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher; -import it.cavallium.dbengine.lucene.searcher.LuceneShardSearcher; -import it.unimi.dsi.fastutil.longs.Long2ObjectMap; -import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; -import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap; import java.io.IOException; import java.nio.file.Path; import java.time.Duration; @@ -33,28 +22,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.lucene.search.CollectionStatistics; -import org.apache.lucene.search.IndexSearcher; import org.jetbrains.annotations.Nullable; -import org.warp.commonutils.batch.ParallelUtils; -import org.warp.commonutils.functional.IOBiConsumer; import reactor.core.publisher.Flux; -import reactor.core.publisher.GroupedFlux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; -import reactor.core.scheduler.Schedulers; import reactor.util.function.Tuple2; -import reactor.util.function.Tuples; public class LLLocalMultiLuceneIndex implements LLLuceneIndex { diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index ac6e95a..906ceac 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -1,5 +1,7 @@ package it.cavallium.dbengine.lucene; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.client.IndicizerAnalyzers; import it.cavallium.dbengine.client.IndicizerSimilarities; @@ -70,6 +72,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; +import reactor.util.concurrent.Queues; public class LuceneUtils { @@ -363,23 +366,28 @@ public class LuceneUtils { ); } - public static Flux convertHits(Flux hits, + public static Flux convertHits(Flux hitsFlux, IndexSearchers indexSearchers, String keyFieldName, - Scheduler scheduler, boolean preserveOrder) { + if (preserveOrder) { + return hitsFlux + .publishOn(Schedulers.boundedElastic()) + .mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName)); + } else { + // Compute parallelism + var availableProcessors = Runtime.getRuntime().availableProcessors(); + var min = Queues.XS_BUFFER_SIZE; + var maxParallelGroups = Math.max(availableProcessors, min); - return hits.transform(hitsFlux -> { - if (preserveOrder) { - return hitsFlux - .publishOn(scheduler) - .mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName)); - } else { - return hitsFlux - .publishOn(scheduler) - .mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName)); - } - }); + return hitsFlux + .groupBy(hit -> hit.shardIndex % maxParallelGroups) // Max n groups + .flatMap(shardHits -> shardHits + .publishOn(Schedulers.boundedElastic()) + .mapNotNull(hit -> mapHitBlocking(hit, indexSearchers, keyFieldName)), + maxParallelGroups // Max n concurrency. Concurrency must be >= total groups count + ); + } } @Nullable diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java index d06a072..41e63c4 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java @@ -1,5 +1,8 @@ package it.cavallium.dbengine.lucene.searcher; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import org.apache.lucene.search.IndexSearcher; import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; @@ -9,32 +12,23 @@ public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher { private static final LuceneLocalSearcher localSearcher = new SimpleLuceneLocalSearcher(); - private static final LuceneLocalSearcher unscoredPagedLuceneLocalSearcher = new LocalLuceneWrapper(new UnscoredUnsortedContinuousLuceneMultiSearcher()); + private static final LuceneLocalSearcher unscoredPagedLuceneLocalSearcher = new LocalLuceneWrapper(new UnscoredUnsortedContinuousLuceneMultiSearcher(), d -> {}); private static final LuceneLocalSearcher countSearcher = new CountLuceneLocalSearcher(); @Override - public Mono collect(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, + public Mono> collect(Mono> indexSearcher, LocalQueryParams queryParams, - String keyFieldName, - Scheduler scheduler) { - if (Schedulers.isInNonBlockingThread()) { - return releaseIndexSearcher - .then(Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread"))); - } + String keyFieldName) { + Mono> collectionMono; if (queryParams.limit() == 0) { - return countSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName, scheduler); + collectionMono = countSearcher.collect(indexSearcher, queryParams, keyFieldName); } else if (!queryParams.isScored() && queryParams.offset() == 0 && queryParams.limit() >= 2147483630 && !queryParams.isSorted()) { - return unscoredPagedLuceneLocalSearcher.collect(indexSearcher, - releaseIndexSearcher, - queryParams, - keyFieldName, - scheduler - ); + collectionMono = unscoredPagedLuceneLocalSearcher.collect(indexSearcher, queryParams, keyFieldName); } else { - return localSearcher.collect(indexSearcher, releaseIndexSearcher, queryParams, keyFieldName, scheduler); + collectionMono = localSearcher.collect(indexSearcher, queryParams, keyFieldName); } + return Mono.fromRunnable(LLUtils::ensureBlocking).then(collectionMono); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java index c8ec0e1..eca9414 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java @@ -1,5 +1,7 @@ package it.cavallium.dbengine.lucene.searcher; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.LLUtils; import reactor.core.publisher.Mono; public class AdaptiveLuceneMultiSearcher implements LuceneMultiSearcher { @@ -13,15 +15,17 @@ public class AdaptiveLuceneMultiSearcher implements LuceneMultiSearcher { private static final LuceneMultiSearcher countLuceneMultiSearcher = new CountLuceneMultiSearcher(); @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { + public Mono> createShardSearcher(LocalQueryParams queryParams) { + Mono> shardSearcherCreationMono; if (queryParams.limit() <= 0) { - return countLuceneMultiSearcher.createShardSearcher(queryParams); + shardSearcherCreationMono = countLuceneMultiSearcher.createShardSearcher(queryParams); } else if (queryParams.isScored()) { - return scoredLuceneMultiSearcher.createShardSearcher(queryParams); + shardSearcherCreationMono = scoredLuceneMultiSearcher.createShardSearcher(queryParams); } else if (queryParams.offset() == 0 && queryParams.limit() >= 2147483630 && !queryParams.isSorted()) { - return unscoredIterableLuceneMultiSearcher.createShardSearcher(queryParams); + shardSearcherCreationMono = unscoredIterableLuceneMultiSearcher.createShardSearcher(queryParams); } else { - return unscoredPagedLuceneMultiSearcher.createShardSearcher(queryParams); + shardSearcherCreationMono = unscoredPagedLuceneMultiSearcher.createShardSearcher(queryParams); } + return Mono.fromRunnable(LLUtils::ensureBlocking).then(shardSearcherCreationMono); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java index 7bd7887..6343f0f 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java @@ -1,28 +1,33 @@ package it.cavallium.dbengine.lucene.searcher; -import it.cavallium.dbengine.client.query.QueryParser; -import it.cavallium.dbengine.client.query.current.data.QueryParams; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; -import org.apache.lucene.search.IndexSearcher; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; public class CountLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono collect(IndexSearcher indexSearcher, Mono releaseIndexSearcher, - LocalQueryParams queryParams, String keyFieldName, Scheduler scheduler) { - return Mono.fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - //noinspection BlockingMethodInNonBlockingContext - return new LuceneSearchResult(TotalHitsCount.of(indexSearcher.count(queryParams.query()), true), - Flux.empty(), - releaseIndexSearcher - ); - }).subscribeOn(scheduler); + public Mono> collect(Mono> indexSearcherMono, + LocalQueryParams queryParams, + String keyFieldName) { + return Mono + .usingWhen( + indexSearcherMono, + indexSearcher -> Mono.fromCallable(() -> { + try (var is = indexSearcher.receive()) { + LLUtils.ensureBlocking(); + return is.getIndexSearcher().count(queryParams.query()); + } + }).subscribeOn(Schedulers.boundedElastic()), + is -> Mono.empty() + ) + .map(count -> new LuceneSearchResult(TotalHitsCount.of(count, true), Flux.empty(), drop -> {}).send()) + .doOnDiscard(Send.class, Send::close); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java index b7cfe63..97603ca 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java @@ -1,55 +1,88 @@ package it.cavallium.dbengine.lucene.searcher; -import it.cavallium.dbengine.client.query.QueryParser; -import it.cavallium.dbengine.client.query.current.data.QueryParams; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicBoolean; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import java.util.concurrent.atomic.AtomicLong; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; -import reactor.core.scheduler.Schedulers; public class CountLuceneMultiSearcher implements LuceneMultiSearcher { @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { - return Mono - .fromCallable(() -> { - AtomicLong totalHits = new AtomicLong(0); - ConcurrentLinkedQueue> release = new ConcurrentLinkedQueue<>(); - return new LuceneShardSearcher() { - @Override - public Mono searchOn(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, - LocalQueryParams queryParams, - Scheduler scheduler) { - return Mono - .fromCallable(() -> { - //noinspection BlockingMethodInNonBlockingContext - totalHits.addAndGet(indexSearcher.count(queryParams.query())); - release.add(releaseIndexSearcher); - return null; - }) - .subscribeOn(scheduler); - } + public Mono> createShardSearcher(LocalQueryParams queryParams) { + return Mono.fromCallable(() -> new CountLuceneShardSearcher(new AtomicLong(0), d -> {}).send()); + } - @Override - public Mono collect(LocalQueryParams queryParams, String keyFieldName, Scheduler scheduler) { - return Mono.fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - return new LuceneSearchResult(TotalHitsCount.of(totalHits.get(), true), - Flux.empty(), - Mono.when(release) - ); - }).subscribeOn(scheduler); + private static class CountLuceneShardSearcher extends + ResourceSupport implements LuceneMultiSearcher { + + private AtomicLong totalHitsCount; + + public CountLuceneShardSearcher(AtomicLong totalHitsCount, Drop drop) { + super(new CloseOnDrop(drop)); + this.totalHitsCount = totalHitsCount; + } + + @Override + public Mono searchOn(Send indexSearcher, LocalQueryParams queryParams) { + return Mono + .fromCallable(() -> { + try (var is = indexSearcher.receive()) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("CountLuceneMultiSearcher must be owned to be used")); + } + LLUtils.ensureBlocking(); + totalHitsCount.addAndGet(is.getIndexSearcher().count(queryParams.query())); + return null; } - }; - }); + }); + } + + @Override + public Mono> collect(LocalQueryParams queryParams, String keyFieldName) { + return Mono.fromCallable(() -> { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("CountLuceneMultiSearcher must be owned to be used")); + } + LLUtils.ensureBlocking(); + return new LuceneSearchResult(TotalHitsCount.of(totalHitsCount.get(), true), Flux.empty(), d -> {}) + .send(); + }); + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + var totalHitsCount = this.totalHitsCount; + makeInaccessible(); + return drop -> new CountLuceneShardSearcher(totalHitsCount, drop); + } + + private void makeInaccessible() { + this.totalHitsCount = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(CountLuceneShardSearcher obj) { + delegate.drop(obj); + } + } } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java index 012a08f..ea0cb68 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java @@ -1,27 +1,140 @@ package it.cavallium.dbengine.lucene.searcher; +import io.net5.buffer.UnpooledDirectByteBuf; +import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.database.LLRange; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import java.util.ArrayList; import java.util.List; import org.apache.lucene.search.IndexSearcher; -public interface IndexSearchers { +public interface IndexSearchers extends Resource { - static IndexSearchers of(List indexSearchers) { - return shardIndex -> { - if (shardIndex < 0) { - throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid"); - } - return indexSearchers.get(shardIndex); - }; + static IndexSearchers of(List indexSearchers) { + return new ShardedIndexSearchers(indexSearchers, d -> {}); } - static IndexSearchers unsharded(IndexSearcher indexSearcher) { - return shardIndex -> { + static IndexSearchers unsharded(LLIndexSearcher indexSearcher) { + return new UnshardedIndexSearchers(indexSearcher, d -> {}); + } + + LLIndexSearcher shard(int shardIndex); + + class UnshardedIndexSearchers extends ResourceSupport + implements IndexSearchers { + + private LLIndexSearcher indexSearcher; + + public UnshardedIndexSearchers(LLIndexSearcher indexSearcher, Drop drop) { + super(new CloseOnDrop(drop)); + this.indexSearcher = indexSearcher; + } + + @Override + public LLIndexSearcher shard(int shardIndex) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("UnshardedIndexSearchers must be owned to be used")); + } if (shardIndex != -1) { throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid, this is a unsharded index"); } return indexSearcher; - }; + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + LLIndexSearcher indexSearcher = this.indexSearcher; + this.makeInaccessible(); + return drop -> new UnshardedIndexSearchers(indexSearcher, drop); + } + + private void makeInaccessible() { + this.indexSearcher = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(UnshardedIndexSearchers obj) { + try { + if (obj.indexSearcher != null) obj.indexSearcher.close(); + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } } - IndexSearcher shard(int shardIndex); + class ShardedIndexSearchers extends ResourceSupport + implements IndexSearchers { + + private List indexSearchers; + + public ShardedIndexSearchers(List indexSearchers, Drop drop) { + super(new CloseOnDrop(drop)); + this.indexSearchers = indexSearchers; + } + + @Override + public LLIndexSearcher shard(int shardIndex) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); + } + if (shardIndex < 0) { + throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid"); + } + return indexSearchers.get(shardIndex); + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + List indexSearchers = this.indexSearchers; + this.makeInaccessible(); + return drop -> new ShardedIndexSearchers(indexSearchers, drop); + } + + private void makeInaccessible() { + this.indexSearchers = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(ShardedIndexSearchers obj) { + try { + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } + } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalLuceneWrapper.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalLuceneWrapper.java deleted file mode 100644 index 493cd6b..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalLuceneWrapper.java +++ /dev/null @@ -1,28 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import org.apache.lucene.search.IndexSearcher; -import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; - -public class LocalLuceneWrapper implements LuceneLocalSearcher { - - private final LuceneMultiSearcher luceneMultiSearcher; - - public LocalLuceneWrapper(LuceneMultiSearcher luceneMultiSearcher) { - this.luceneMultiSearcher = luceneMultiSearcher; - } - - @Override - public Mono collect(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, - LocalQueryParams queryParams, - String keyFieldName, - Scheduler scheduler) { - var shardSearcher = luceneMultiSearcher.createShardSearcher(queryParams); - return shardSearcher - .flatMap(luceneShardSearcher -> luceneShardSearcher - .searchOn(indexSearcher, releaseIndexSearcher, queryParams, scheduler) - .then(luceneShardSearcher.collect(queryParams, keyFieldName, scheduler)) - ); - } -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java index 32546e0..3346666 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java @@ -1,21 +1,18 @@ package it.cavallium.dbengine.lucene.searcher; -import it.cavallium.dbengine.client.query.current.data.QueryParams; -import org.apache.lucene.search.IndexSearcher; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; public interface LuceneLocalSearcher { /** - * @param indexSearcher Lucene index searcher + * @param indexSearcherMono Lucene index searcher * @param queryParams the query parameters * @param keyFieldName the name of the key field - * @param scheduler a blocking scheduler */ - Mono collect(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, + Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName, - Scheduler scheduler); + String keyFieldName); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java index c27ab4c..16fa9cb 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java @@ -1,28 +1,32 @@ package it.cavallium.dbengine.lucene.searcher; -import it.cavallium.dbengine.client.query.current.data.QueryParams; -import it.cavallium.dbengine.database.LLKeyScore; -import it.cavallium.dbengine.lucene.LuceneUtils; -import java.util.ArrayList; -import java.util.List; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.Sort; -import org.jetbrains.annotations.Nullable; -import org.warp.commonutils.log.Logger; -import org.warp.commonutils.log.LoggerFactory; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import org.jetbrains.annotations.NotNull; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; -public interface LuceneMultiSearcher { +public interface LuceneMultiSearcher extends LuceneLocalSearcher { /** - * Do a lucene query, receiving the single results using a consumer * @param queryParams the query parameters + * @param keyFieldName the name of the key field */ - Mono createShardSearcher(LocalQueryParams queryParams); + Mono> collect(Flux> indexSearchersFlux, + LocalQueryParams queryParams, + String keyFieldName); + /** + * @param indexSearcherMono Lucene index searcher + * @param queryParams the query parameters + * @param keyFieldName the name of the key field + */ + @Override + default Mono> collect(Mono> indexSearcherMono, + LocalQueryParams queryParams, + String keyFieldName) { + return this.collect(indexSearcherMono.flux(), queryParams, keyFieldName); + } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java index 0a786b6..2746c82 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java @@ -1,7 +1,11 @@ package it.cavallium.dbengine.lucene.searcher; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; +import it.cavallium.dbengine.database.LLSearchResultShard; import it.cavallium.dbengine.database.disk.LLLocalKeyValueDatabase; import java.io.IOException; import java.util.Objects; @@ -10,39 +14,33 @@ import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class LuceneSearchResult { +public final class LuceneSearchResult extends ResourceSupport { private static final Logger logger = LoggerFactory.getLogger(LuceneSearchResult.class); - private volatile boolean releaseCalled; + private TotalHitsCount totalHitsCount; + private Flux results; - private final TotalHitsCount totalHitsCount; - private final Flux results; - private final Mono release; - - public LuceneSearchResult(TotalHitsCount totalHitsCount, Flux results, Mono release) { + public LuceneSearchResult(TotalHitsCount totalHitsCount, Flux results, Drop drop) { + super(new LuceneSearchResult.CloseOnDrop(drop)); this.totalHitsCount = totalHitsCount; this.results = results; - this.release = Mono.fromRunnable(() -> { - if (releaseCalled) { - logger.warn(this.getClass().getName() + "::release has been called twice!"); - } - releaseCalled = true; - }).then(release); } public TotalHitsCount totalHitsCount() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("LuceneSearchResult must be owned to be used")); + } return totalHitsCount; } public Flux results() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("LuceneSearchResult must be owned to be used")); + } return results; } - public Mono release() { - return release; - } - @Override public boolean equals(Object obj) { if (obj == this) @@ -63,13 +61,36 @@ public final class LuceneSearchResult { return "LuceneSearchResult[" + "totalHitsCount=" + totalHitsCount + ", " + "results=" + results + ']'; } - @SuppressWarnings("deprecation") @Override - protected void finalize() throws Throwable { - if (!releaseCalled) { - logger.warn(this.getClass().getName() + "::release has not been called before class finalization!"); + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + var totalHitsCount = this.totalHitsCount; + var results = this.results; + makeInaccessible(); + return drop -> new LuceneSearchResult(totalHitsCount, results, drop); + } + + private void makeInaccessible() { + this.totalHitsCount = null; + this.results = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(LuceneSearchResult obj) { + delegate.drop(obj); } - super.finalize(); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneShardSearcher.java deleted file mode 100644 index e9ab9ca..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneShardSearcher.java +++ /dev/null @@ -1,26 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import it.cavallium.dbengine.client.query.current.data.QueryParams; -import org.apache.lucene.search.IndexSearcher; -import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; - -public interface LuceneShardSearcher { - - /** - * @param indexSearcher the index searcher, which contains all the lucene data - * @param queryParams the query parameters - * @param scheduler a blocking scheduler - */ - Mono searchOn(IndexSearcher indexSearcher, - Mono indexSearcherRelease, - LocalQueryParams queryParams, - Scheduler scheduler); - - /** - * @param queryParams the query parameters - * @param keyFieldName the name of the key field - * @param collectorScheduler a blocking scheduler - */ - Mono collect(LocalQueryParams queryParams, String keyFieldName, Scheduler collectorScheduler); -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java index b044055..97064be 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java @@ -8,13 +8,12 @@ import org.apache.lucene.search.CollectorManager; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopFieldDocs; import reactor.core.publisher.Mono; public class ScoredLuceneMultiSearcher implements LuceneMultiSearcher { @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { + public Mono createShardSearcher(LocalQueryParams queryParams) { return Mono .fromCallable(() -> { Sort luceneSort = queryParams.sort(); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java index 85d5a49..2d3fab5 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java @@ -1,7 +1,6 @@ package it.cavallium.dbengine.lucene.searcher; import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS; -import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.TIE_BREAKER; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.lucene.LuceneUtils; @@ -10,27 +9,19 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; import org.apache.lucene.search.CollectorManager; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopFieldDocs; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.publisher.Sinks; -import reactor.core.publisher.Sinks.Empty; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; -class ScoredSimpleLuceneShardSearcher implements LuceneShardSearcher { +class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { private final Object lock = new Object(); private final List indexSearchersArray = new ArrayList<>(); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index fbc2cb4..2030acd 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -4,116 +4,141 @@ import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT; import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; +import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.lucene.LuceneUtils; import java.io.IOException; import java.util.Arrays; import java.util.Objects; +import java.util.Optional; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.search.TotalHits; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import reactor.core.publisher.SignalType; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; +import reactor.util.function.Tuple2; +import reactor.util.function.Tuple3; +import reactor.util.function.Tuples; public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono collect(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, + public Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName, - Scheduler scheduler) { - return Mono - .fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); - PaginationInfo paginationInfo; - if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); - } else { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); - } - TopDocs firstPageTopDocs; - { - TopDocsCollector firstPageCollector = TopDocsSearcher.getTopDocsCollector( - queryParams.sort(), - LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), - null, - LuceneUtils.totalHitsThreshold(), - !paginationInfo.forceSinglePage(), - queryParams.isScored()); - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(queryParams.query(), firstPageCollector); - firstPageTopDocs = firstPageCollector.topDocs(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()), - LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()) - ); - } - Flux firstPageMono = LuceneUtils - .convertHits(Flux.fromArray(firstPageTopDocs.scoreDocs), IndexSearchers.unsharded(indexSearcher), - keyFieldName, scheduler, true) - .take(queryParams.limit(), true); + String keyFieldName) { + + Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); + PaginationInfo paginationInfo; + if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { + paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); + } else { + paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); + } + + return indexSearcherMono + .flatMap(indexSearcherToReceive -> { + var indexSearcher = indexSearcherToReceive.receive(); + var indexSearchers = IndexSearchers.unsharded(indexSearcher); + return Mono + .fromCallable(() -> { + LLUtils.ensureBlocking(); + TopDocsCollector firstPageCollector = TopDocsSearcher.getTopDocsCollector( + queryParams.sort(), + LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), + null, + LuceneUtils.totalHitsThreshold(), + !paginationInfo.forceSinglePage(), + queryParams.isScored()); + + indexSearcher.getIndexSearcher().search(queryParams.query(), firstPageCollector); + return firstPageCollector.topDocs(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()), + LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()) + ); + }) + .map(firstPageTopDocs -> { + Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageTopDocs.scoreDocs), + indexSearchers, keyFieldName, true) + .take(queryParams.limit(), true); + + return Tuples.of(Optional.ofNullable(LuceneUtils.getLastScoreDoc(firstPageTopDocs.scoreDocs)), + LuceneUtils.convertTotalHitsCount(firstPageTopDocs.totalHits), firstPageHitsFlux); + }) + .map(firstResult -> { + var firstPageLastScoreDoc = firstResult.getT1(); + var totalHitsCount = firstResult.getT2(); + var firstPageFlux = firstResult.getT3(); - Flux nextHits; - if (paginationInfo.forceSinglePage() || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { - nextHits = null; - } else { - nextHits = Flux.defer(() -> Flux - .generate( - () -> new CurrentPageInfo(LuceneUtils.getLastScoreDoc(firstPageTopDocs.scoreDocs), paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1), - (s, sink) -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - if (s.last() != null && s.remainingLimit() > 0) { - TopDocs pageTopDocs; - try { - TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), - s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), true, - queryParams.isScored()); - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(queryParams.query(), collector); - pageTopDocs = collector.topDocs(); - } catch (IOException e) { - sink.error(e); - return EMPTY_STATUS; - } - var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); - sink.next(pageTopDocs); - return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), s.pageIndex() + 1); - } else { - sink.complete(); - return EMPTY_STATUS; - } - }, - s -> {} - ) - .subscribeOn(scheduler) - .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) - .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, - IndexSearchers.unsharded(indexSearcher), keyFieldName, scheduler, true)) - ); - } + Flux nextHits; + if (paginationInfo.forceSinglePage() || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { + nextHits = null; + } else { + nextHits = Flux.defer(() -> Flux + .generate( + () -> new CurrentPageInfo(firstPageLastScoreDoc.orElse(null), paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1), + (s, sink) -> { + LLUtils.ensureBlocking(); + if (s.last() != null && s.remainingLimit() > 0) { + TopDocs pageTopDocs; + try { + TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), + s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), true, + queryParams.isScored()); + indexSearcher.getIndexSearcher().search(queryParams.query(), collector); + pageTopDocs = collector.topDocs(); + } catch (IOException e) { + sink.error(e); + return EMPTY_STATUS; + } + var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); + sink.next(pageTopDocs); + return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), s.pageIndex() + 1); + } else { + sink.complete(); + return EMPTY_STATUS; + } + }, + s -> {} + ) + .subscribeOn(Schedulers.boundedElastic()) + .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) + .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, + keyFieldName, true)) + ); + } - Flux combinedFlux; + Flux combinedFlux; - if (nextHits != null) { - combinedFlux = firstPageMono - .concatWith(nextHits); - } else { - combinedFlux = firstPageMono; - } - - return new LuceneSearchResult(LuceneUtils.convertTotalHitsCount(firstPageTopDocs.totalHits), combinedFlux, - //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), - releaseIndexSearcher - ); - }) - .subscribeOn(scheduler); + if (nextHits != null) { + combinedFlux = firstPageFlux + .concatWith(nextHits); + } else { + combinedFlux = firstPageFlux; + } + return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> { + indexSearcher.close(); + indexSearchers.close(); + }).send(); + }) + .doFinally(s -> { + // Close searchers if the search result has not been returned + if (s != SignalType.ON_COMPLETE) { + indexSearcher.close(); + indexSearchers.close(); + } + }); + } + ) + .doOnDiscard(Send.class, Send::close); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java index 8118348..4162b75 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java @@ -9,7 +9,7 @@ import reactor.core.publisher.Mono; public class UnscoredPagedLuceneMultiSearcher implements LuceneMultiSearcher { @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { + public Mono createShardSearcher(LocalQueryParams queryParams) { return Mono .fromCallable(() -> { if (queryParams.isScored()) { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java index e041b6b..917befd 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java @@ -10,7 +10,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; -import java.util.stream.Collectors; import org.apache.lucene.search.CollectorManager; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; @@ -22,7 +21,7 @@ import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; -class UnscoredPagedLuceneShardSearcher implements LuceneShardSearcher { +class UnscoredPagedLuceneShardSearcher implements LuceneMultiSearcher { private final Object lock = new Object(); private final List indexSearchersArray = new ArrayList<>(); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java index 48a24b2..eacd58a 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java @@ -2,10 +2,8 @@ package it.cavallium.dbengine.lucene.searcher; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.lucene.LuceneUtils; -import it.unimi.dsi.fastutil.objects.ObjectArrayList; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -18,14 +16,12 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.SimpleCollector; -import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; import reactor.core.publisher.Sinks.EmitResult; import reactor.core.publisher.Sinks.Many; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; -import reactor.util.concurrent.Queues; public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMultiSearcher { @@ -34,7 +30,7 @@ public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMult .availableProcessors(), Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, "UnscoredUnsortedExecutor"); @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { + public Mono createShardSearcher(LocalQueryParams queryParams) { return Mono .fromCallable(() -> { AtomicBoolean alreadySubscribed = new AtomicBoolean(false); @@ -92,7 +88,7 @@ public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMult } }; - return new LuceneShardSearcher() { + return new LuceneMultiSearcher() { private final Object lock = new Object(); private final List indexSearchersArray = new ArrayList<>(); private final List> indexSearcherReleasersArray = new ArrayList<>(); From d1963a1d6589de38fa7686924b67056bfd4bf1e6 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Sun, 19 Sep 2021 12:01:11 +0200 Subject: [PATCH 02/23] Finish refactoring SimpleLuceneLocalSearcher --- .../cavallium/dbengine/database/LLUtils.java | 43 +++ .../lucene/searcher/IndexSearchers.java | 12 +- .../searcher/SimpleLuceneLocalSearcher.java | 249 ++++++++++-------- 3 files changed, 194 insertions(+), 110 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 2a729f4..651567c 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -5,6 +5,7 @@ import com.google.common.primitives.Longs; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.CompositeBuffer; +import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; import io.net5.util.internal.PlatformDependent; @@ -24,6 +25,7 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import java.util.function.ToIntFunction; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -344,6 +346,47 @@ public class LLUtils { } } + /** + * cleanup resource + * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful + */ + public static > Mono usingSend(Mono> resourceSupplier, + Function, Mono> resourceClosure, + boolean cleanupOnSuccess) { + return Mono.usingWhen(resourceSupplier, resourceClosure, + r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, + (r, ex) -> Mono.fromRunnable(r::close), + r -> Mono.fromRunnable(r::close)) + .doOnDiscard(Send.class, Send::close); + } + + /** + * cleanup resource + * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful + */ + public static , V extends T> Mono usingResource(Mono resourceSupplier, + Function> resourceClosure, + boolean cleanupOnSuccess) { + return Mono.usingWhen(resourceSupplier, resourceClosure, + r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, + (r, ex) -> Mono.fromRunnable(r::close), + r -> Mono.fromRunnable(r::close)) + .doOnDiscard(Resource.class, Resource::close) + .doOnDiscard(Send.class, Send::close); + } + public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java index ea0cb68..05925e8 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java @@ -19,7 +19,7 @@ public interface IndexSearchers extends Resource { return new ShardedIndexSearchers(indexSearchers, d -> {}); } - static IndexSearchers unsharded(LLIndexSearcher indexSearcher) { + static UnshardedIndexSearchers unsharded(Send indexSearcher) { return new UnshardedIndexSearchers(indexSearcher, d -> {}); } @@ -30,9 +30,9 @@ public interface IndexSearchers extends Resource { private LLIndexSearcher indexSearcher; - public UnshardedIndexSearchers(LLIndexSearcher indexSearcher, Drop drop) { + public UnshardedIndexSearchers(Send indexSearcher, Drop drop) { super(new CloseOnDrop(drop)); - this.indexSearcher = indexSearcher; + this.indexSearcher = indexSearcher.receive(); } @Override @@ -46,6 +46,10 @@ public interface IndexSearchers extends Resource { return indexSearcher; } + public LLIndexSearcher shard() { + return this.shard(0); + } + @Override protected RuntimeException createResourceClosedException() { return new IllegalStateException("Closed"); @@ -53,7 +57,7 @@ public interface IndexSearchers extends Resource { @Override protected Owned prepareSend() { - LLIndexSearcher indexSearcher = this.indexSearcher; + Send indexSearcher = this.indexSearcher.send(); this.makeInaccessible(); return drop -> new UnshardedIndexSearchers(indexSearcher, drop); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index 2030acd..284b006 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -4,31 +4,23 @@ import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT; import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; -import io.net5.buffer.api.Buffer; -import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.lucene.LuceneUtils; +import it.cavallium.dbengine.lucene.searcher.IndexSearchers.UnshardedIndexSearchers; import java.io.IOException; import java.util.Arrays; import java.util.Objects; -import java.util.Optional; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TotalHits; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.publisher.SignalType; -import reactor.core.scheduler.Scheduler; +import reactor.core.publisher.SynchronousSink; import reactor.core.scheduler.Schedulers; -import reactor.util.function.Tuple2; -import reactor.util.function.Tuple3; -import reactor.util.function.Tuples; public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { @@ -38,107 +30,152 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { String keyFieldName) { Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); - PaginationInfo paginationInfo; + PaginationInfo paginationInfo = getPaginationInfo(queryParams); + + var indexSearchersMono = indexSearcherMono.map(IndexSearchers::unsharded); + + return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this + // Search first page results + .searchFirstPage(indexSearchers, queryParams, paginationInfo) + // Compute the results of the first page + .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers, + keyFieldName, queryParams)) + // Compute other results + .transform(firstResult -> this.computeOtherResults(firstResult, indexSearchers, queryParams, keyFieldName)) + // Ensure that one LuceneSearchResult is always returned + .single(), + false); + } + + /** + * Get the pagination info + */ + private PaginationInfo getPaginationInfo(LocalQueryParams queryParams) { if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); } else { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); } + } - return indexSearcherMono - .flatMap(indexSearcherToReceive -> { - var indexSearcher = indexSearcherToReceive.receive(); - var indexSearchers = IndexSearchers.unsharded(indexSearcher); - return Mono - .fromCallable(() -> { - LLUtils.ensureBlocking(); - TopDocsCollector firstPageCollector = TopDocsSearcher.getTopDocsCollector( - queryParams.sort(), - LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), - null, - LuceneUtils.totalHitsThreshold(), - !paginationInfo.forceSinglePage(), - queryParams.isScored()); + /** + * Search effectively the raw results of the first page + */ + private Mono searchFirstPage(UnshardedIndexSearchers indexSearchers, + LocalQueryParams queryParams, + PaginationInfo paginationInfo) { + var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); + var pagination = !paginationInfo.forceSinglePage(); + var resultsOffset = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()); + return Mono + .fromSupplier(() -> new CurrentPageInfo(null, limit, 0)) + .handle((s, sink) -> this.searchPageSync(queryParams, indexSearchers, pagination, resultsOffset, s, sink)); + } - indexSearcher.getIndexSearcher().search(queryParams.query(), firstPageCollector); - return firstPageCollector.topDocs(LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()), - LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit()) - ); - }) - .map(firstPageTopDocs -> { - Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageTopDocs.scoreDocs), - indexSearchers, keyFieldName, true) - .take(queryParams.limit(), true); - - return Tuples.of(Optional.ofNullable(LuceneUtils.getLastScoreDoc(firstPageTopDocs.scoreDocs)), - LuceneUtils.convertTotalHitsCount(firstPageTopDocs.totalHits), firstPageHitsFlux); - }) - .map(firstResult -> { - var firstPageLastScoreDoc = firstResult.getT1(); - var totalHitsCount = firstResult.getT2(); - var firstPageFlux = firstResult.getT3(); - - - Flux nextHits; - if (paginationInfo.forceSinglePage() || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { - nextHits = null; - } else { - nextHits = Flux.defer(() -> Flux - .generate( - () -> new CurrentPageInfo(firstPageLastScoreDoc.orElse(null), paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1), - (s, sink) -> { - LLUtils.ensureBlocking(); - if (s.last() != null && s.remainingLimit() > 0) { - TopDocs pageTopDocs; - try { - TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), - s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), true, - queryParams.isScored()); - indexSearcher.getIndexSearcher().search(queryParams.query(), collector); - pageTopDocs = collector.topDocs(); - } catch (IOException e) { - sink.error(e); - return EMPTY_STATUS; - } - var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); - sink.next(pageTopDocs); - return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), s.pageIndex() + 1); - } else { - sink.complete(); - return EMPTY_STATUS; - } - }, - s -> {} - ) - .subscribeOn(Schedulers.boundedElastic()) - .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) - .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, - keyFieldName, true)) - ); - } - - Flux combinedFlux; - - if (nextHits != null) { - combinedFlux = firstPageFlux - .concatWith(nextHits); - } else { - combinedFlux = firstPageFlux; - } - return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> { - indexSearcher.close(); - indexSearchers.close(); - }).send(); - }) - .doFinally(s -> { - // Close searchers if the search result has not been returned - if (s != SignalType.ON_COMPLETE) { - indexSearcher.close(); - indexSearchers.close(); - } - }); - } + /** + * Search effectively the merged raw results of the next pages + */ + private Flux searchOtherPages(UnshardedIndexSearchers indexSearchers, + LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { + return Flux + .generate( + () -> secondPageInfo, + (s, sink) -> searchPageSync(queryParams, indexSearchers, true, 0, s, sink), + s -> {} ) - .doOnDiscard(Send.class, Send::close); + .subscribeOn(Schedulers.boundedElastic()) + .map(PageData::topDocs) + .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) + .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, + keyFieldName, true)); + } + + private static record FirstPageResults(TotalHitsCount totalHitsCount, Flux firstPageHitsFlux, + CurrentPageInfo nextPageInfo) {} + + /** + * Compute the results of the first page, extracting useful data + */ + private Mono computeFirstPageResults(Mono firstPageDataMono, + IndexSearchers indexSearchers, + String keyFieldName, + LocalQueryParams queryParams) { + return firstPageDataMono.map(firstPageData -> { + var totalHitsCount = LuceneUtils.convertTotalHitsCount(firstPageData.topDocs().totalHits); + + Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageData.topDocs().scoreDocs), + indexSearchers, keyFieldName, true) + .take(queryParams.limit(), true); + + CurrentPageInfo nextPageInfo = firstPageData.nextPageInfo(); + + return new FirstPageResults(totalHitsCount, firstPageHitsFlux, nextPageInfo); + }); + } + + private Mono> computeOtherResults(Mono firstResultMono, + UnshardedIndexSearchers indexSearchers, + LocalQueryParams queryParams, + String keyFieldName) { + return firstResultMono.map(firstResult -> { + var totalHitsCount = firstResult.totalHitsCount(); + var firstPageHitsFlux = firstResult.firstPageHitsFlux(); + var secondPageInfo = firstResult.nextPageInfo(); + + Flux nextHitsFlux = searchOtherPages(indexSearchers, queryParams, keyFieldName, secondPageInfo); + + Flux combinedFlux = firstPageHitsFlux.concatWith(nextHitsFlux); + return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> indexSearchers.close()).send(); + }); + } + + private static record PageData(TopDocs topDocs, CurrentPageInfo nextPageInfo) {} + + /** + * + * @param resultsOffset offset of the resulting topDocs. Useful if you want to + * skip the first n results in the first page + */ + private CurrentPageInfo searchPageSync(LocalQueryParams queryParams, + UnshardedIndexSearchers indexSearchers, + boolean allowPagination, + int resultsOffset, + CurrentPageInfo s, + SynchronousSink sink) { + LLUtils.ensureBlocking(); + if (resultsOffset < 0) { + throw new IndexOutOfBoundsException(resultsOffset); + } + if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { + TopDocs pageTopDocs; + try { + TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), + s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), allowPagination, + queryParams.isScored()); + indexSearchers.shard().getIndexSearcher().search(queryParams.query(), collector); + if (resultsOffset > 0) { + pageTopDocs = collector.topDocs(resultsOffset, s.currentPageLimit()); + } else { + pageTopDocs = collector.topDocs(); + } + } catch (IOException e) { + sink.error(e); + return EMPTY_STATUS; + } + var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); + long nextRemainingLimit; + if (allowPagination) { + nextRemainingLimit = s.remainingLimit() - s.currentPageLimit(); + } else { + nextRemainingLimit = 0L; + } + var nextPageIndex = s.pageIndex() + 1; + var nextPageInfo = new CurrentPageInfo(pageLastDoc, nextRemainingLimit, nextPageIndex); + sink.next(new PageData(pageTopDocs, nextPageInfo)); + return nextPageInfo; + } else { + sink.complete(); + return EMPTY_STATUS; + } } } From 31422847c2fe341f9524e2b633af77725ab9043c Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Sun, 19 Sep 2021 12:35:48 +0200 Subject: [PATCH 03/23] Add SimpleUnsortedUnscoredLuceneMultiSearcher --- .../cavallium/dbengine/database/LLUtils.java | 34 ++++------ ...leUnsortedUnscoredLuceneMultiSearcher.java | 67 +++++++++++++++++++ 2 files changed, 81 insertions(+), 20 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 651567c..d32b0e2 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -353,16 +353,13 @@ public class LLUtils { public static > Mono usingSend(Mono> resourceSupplier, Function, Mono> resourceClosure, boolean cleanupOnSuccess) { - return Mono.usingWhen(resourceSupplier, resourceClosure, - r -> { - if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); - } else { - return Mono.empty(); - } - }, - (r, ex) -> Mono.fromRunnable(r::close), - r -> Mono.fromRunnable(r::close)) + return Mono.usingWhen(resourceSupplier, resourceClosure, r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) .doOnDiscard(Send.class, Send::close); } @@ -373,16 +370,13 @@ public class LLUtils { public static , V extends T> Mono usingResource(Mono resourceSupplier, Function> resourceClosure, boolean cleanupOnSuccess) { - return Mono.usingWhen(resourceSupplier, resourceClosure, - r -> { - if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); - } else { - return Mono.empty(); - } - }, - (r, ex) -> Mono.fromRunnable(r::close), - r -> Mono.fromRunnable(r::close)) + return Mono.usingWhen(resourceSupplier, resourceClosure, r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) .doOnDiscard(Resource.class, Resource::close) .doOnDiscard(Send.class, Send::close); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java new file mode 100644 index 0000000..ef99dde --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -0,0 +1,67 @@ +package it.cavallium.dbengine.lucene.searcher; + +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; +import it.cavallium.dbengine.database.LLKeyScore; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSearcher { + + private final LuceneLocalSearcher localSearcher; + + public SimpleUnsortedUnscoredLuceneMultiSearcher(LuceneLocalSearcher localSearcher) { + this.localSearcher = localSearcher; + } + + @Override + public Mono> collect(Flux> indexSearchersFlux, + LocalQueryParams queryParams, + String keyFieldName) { + return Mono + .fromRunnable(() -> { + LLUtils.ensureBlocking(); + if (!queryParams.isSorted()) { + throw new UnsupportedOperationException("Sorted queries are not supported" + + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); + } + if (!queryParams.isScored()) { + throw new UnsupportedOperationException("Scored queries are not supported" + + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); + } + }) + .thenMany(indexSearchersFlux) + .flatMap(resSend -> localSearcher.collect(Mono.just(resSend), queryParams, keyFieldName)) + .collectList() + .map(results -> { + List resultsToDrop = new ArrayList<>(results.size()); + List> resultsFluxes = new ArrayList<>(results.size()); + boolean exactTotalHitsCount = true; + long totalHitsCountValue = 0; + for (Send resultToReceive : results) { + LuceneSearchResult result = resultToReceive.receive(); + resultsToDrop.add(result); + resultsFluxes.add(result.results()); + exactTotalHitsCount &= result.totalHitsCount().exact(); + totalHitsCountValue += result.totalHitsCount().value(); + } + + var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); + Flux mergedFluxes = Flux.merge(resultsFluxes); + + return new LuceneSearchResult(totalHitsCount, mergedFluxes, d -> { + for (LuceneSearchResult luceneSearchResult : resultsToDrop) { + luceneSearchResult.close(); + } + }).send(); + }) + .doOnDiscard(Send.class, Send::close); + } +} From 65db1711b5eb56156da547fc844a1213bb52d691 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Sun, 19 Sep 2021 19:59:37 +0200 Subject: [PATCH 04/23] Missing: moreLikeThis --- .../dbengine/client/LuceneIndex.java | 17 +- .../dbengine/client/LuceneIndexImpl.java | 120 +++++--- .../dbengine/client/SearchResult.java | 89 +++--- .../dbengine/client/SearchResultKeys.java | 88 +++--- .../cavallium/dbengine/database/LLUtils.java | 83 +++++ .../database/disk/LLIndexContext.java | 81 +++++ .../database/disk/LLIndexContexts.java | 199 ++++++++++++ .../database/disk/LLIndexSearcher.java | 4 +- .../database/disk/LLLocalLuceneIndex.java | 101 +----- .../disk/LLLocalMultiLuceneIndex.java | 90 ++++-- .../dbengine/lucene/LuceneUtils.java | 17 +- .../searcher/AdaptiveLuceneLocalSearcher.java | 18 +- .../searcher/AdaptiveLuceneMultiSearcher.java | 33 +- .../searcher/CountLuceneLocalSearcher.java | 5 +- .../searcher/CountLuceneMultiSearcher.java | 88 ------ .../lucene/searcher/FirstPageResults.java | 8 + .../lucene/searcher/IndexSearchers.java | 144 --------- .../lucene/searcher/LLSearchTransformer.java | 10 + .../lucene/searcher/LuceneLocalSearcher.java | 4 +- .../lucene/searcher/LuceneMultiSearcher.java | 8 +- .../dbengine/lucene/searcher/PageData.java | 5 + .../searcher/ScoredLuceneMultiSearcher.java | 37 --- .../ScoredSimpleLuceneShardSearcher.java | 290 ++++++++++-------- .../searcher/SimpleLuceneLocalSearcher.java | 51 ++- ...leUnsortedUnscoredLuceneMultiSearcher.java | 5 +- .../UnscoredPagedLuceneMultiSearcher.java | 34 -- .../UnscoredPagedLuceneShardSearcher.java | 151 --------- .../UnscoredTopDocsCollectorManager.java | 70 ----- ...UnsortedContinuousLuceneMultiSearcher.java | 179 ----------- 29 files changed, 847 insertions(+), 1182 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java create mode 100644 src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/FirstPageResults.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/PageData.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredTopDocsCollectorManager.java delete mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java diff --git a/src/main/java/it/cavallium/dbengine/client/LuceneIndex.java b/src/main/java/it/cavallium/dbengine/client/LuceneIndex.java index e313868..19832a1 100644 --- a/src/main/java/it/cavallium/dbengine/client/LuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/client/LuceneIndex.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.client; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.query.ClientQueryParams; import it.cavallium.dbengine.client.query.current.data.Query; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; @@ -52,9 +53,9 @@ public interface LuceneIndex extends LLSnapshottable { Mono deleteAll(); - Mono> moreLikeThis(ClientQueryParams> queryParams, T key, U mltDocumentValue); + Mono>> moreLikeThis(ClientQueryParams> queryParams, T key, U mltDocumentValue); - default Mono> moreLikeThisWithValues(ClientQueryParams> queryParams, + default Mono>> moreLikeThisWithValues(ClientQueryParams> queryParams, T key, U mltDocumentValue, ValueGetter valueGetter) { @@ -64,21 +65,19 @@ public interface LuceneIndex extends LLSnapshottable { getValueGetterTransformer(valueGetter)); } - Mono> moreLikeThisWithTransformer(ClientQueryParams> queryParams, + Mono>> moreLikeThisWithTransformer(ClientQueryParams> queryParams, T key, U mltDocumentValue, ValueTransformer valueTransformer); - Mono> search(ClientQueryParams> queryParams); + Mono>> search(ClientQueryParams> queryParams); - default Mono> searchWithValues(ClientQueryParams> queryParams, + default Mono>> searchWithValues(ClientQueryParams> queryParams, ValueGetter valueGetter) { - return this.searchWithTransformer(queryParams, - getValueGetterTransformer(valueGetter) - ); + return this.searchWithTransformer(queryParams, getValueGetterTransformer(valueGetter)); } - Mono> searchWithTransformer(ClientQueryParams> queryParams, + Mono>> searchWithTransformer(ClientQueryParams> queryParams, ValueTransformer valueTransformer); Mono count(@Nullable CompositeSnapshot snapshot, Query query); diff --git a/src/main/java/it/cavallium/dbengine/client/LuceneIndexImpl.java b/src/main/java/it/cavallium/dbengine/client/LuceneIndexImpl.java index b242a9b..e3d6ef7 100644 --- a/src/main/java/it/cavallium/dbengine/client/LuceneIndexImpl.java +++ b/src/main/java/it/cavallium/dbengine/client/LuceneIndexImpl.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.client; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.query.ClientQueryParams; import it.cavallium.dbengine.client.query.current.data.Query; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; @@ -84,58 +85,73 @@ public class LuceneIndexImpl implements LuceneIndex { return luceneIndex.deleteAll(); } - private Mono> transformLuceneResultWithTransformer(LLSearchResultShard llSearchResult) { - return Mono.just(new SearchResultKeys<>(llSearchResult.results() - .map(signal -> new SearchResultKey<>(Mono.fromCallable(signal::key).map(indicizer::getKey), signal.score())), - llSearchResult.totalHitsCount(), - llSearchResult.release() - )); + private Mono>> transformLuceneResultWithTransformer( + Mono> llSearchResultMono) { + return llSearchResultMono.map(llSearchResultToReceive -> { + var llSearchResult = llSearchResultToReceive.receive(); + return new SearchResultKeys<>(llSearchResult.results() + .map(signal -> new SearchResultKey<>(Mono + .fromCallable(signal::key) + .map(indicizer::getKey), signal.score())), + llSearchResult.totalHitsCount(), + d -> llSearchResult.close() + ).send(); + }); } - private Mono> transformLuceneResultWithValues(LLSearchResultShard llSearchResult, + private Mono>> transformLuceneResultWithValues( + Mono> llSearchResultMono, ValueGetter valueGetter) { - return Mono.fromCallable(() -> new SearchResult<>(llSearchResult.results().map(signal -> { - var key = Mono.fromCallable(signal::key).map(indicizer::getKey); - return new SearchResultItem<>(key, key.flatMap(valueGetter::get), signal.score()); - }), llSearchResult.totalHitsCount(), llSearchResult.release())); + return llSearchResultMono.map(llSearchResultToReceive -> { + var llSearchResult = llSearchResultToReceive.receive(); + return new SearchResult<>(llSearchResult.results().map(signal -> { + var key = Mono.fromCallable(signal::key).map(indicizer::getKey); + return new SearchResultItem<>(key, key.flatMap(valueGetter::get), signal.score()); + }), llSearchResult.totalHitsCount(), d -> llSearchResult.close()).send(); + }); } - private Mono> transformLuceneResultWithTransformer(LLSearchResultShard llSearchResult, + private Mono>> transformLuceneResultWithTransformer( + Mono> llSearchResultMono, ValueTransformer valueTransformer) { - var scoresWithKeysFlux = llSearchResult - .results() - .flatMapSequential(signal -> Mono - .fromCallable(signal::key) - .map(indicizer::getKey) - .map(key -> Tuples.of(signal.score(), key)) - ); - var resultItemsFlux = valueTransformer - .transform(scoresWithKeysFlux) - .filter(tuple3 -> tuple3.getT3().isPresent()) - .map(tuple3 -> new SearchResultItem<>(Mono.just(tuple3.getT2()), - Mono.just(tuple3.getT3().orElseThrow()), - tuple3.getT1() - )); - return Mono.fromCallable(() -> new SearchResult<>(resultItemsFlux, - llSearchResult.totalHitsCount(), - llSearchResult.release() - )); + return llSearchResultMono + .map(llSearchResultToReceive -> { + var llSearchResult = llSearchResultToReceive.receive(); + var scoresWithKeysFlux = llSearchResult + .results() + .flatMapSequential(signal -> Mono + .fromCallable(signal::key) + .map(indicizer::getKey) + .map(key -> Tuples.of(signal.score(), key)) + ); + var resultItemsFlux = valueTransformer + .transform(scoresWithKeysFlux) + .filter(tuple3 -> tuple3.getT3().isPresent()) + .map(tuple3 -> new SearchResultItem<>(Mono.just(tuple3.getT2()), + Mono.just(tuple3.getT3().orElseThrow()), + tuple3.getT1() + )); + return new SearchResult<>(resultItemsFlux, + llSearchResult.totalHitsCount(), + d -> llSearchResult.close() + ).send(); + }); } @Override - public Mono> moreLikeThis(ClientQueryParams> queryParams, + public Mono>> moreLikeThis(ClientQueryParams> queryParams, T key, U mltDocumentValue) { Flux>> mltDocumentFields = indicizer.getMoreLikeThisDocumentFields(key, mltDocumentValue); return luceneIndex .moreLikeThis(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName(), mltDocumentFields) - .flatMap(this::transformLuceneResultWithTransformer); + .transform(this::transformLuceneResultWithTransformer); } @Override - public Mono> moreLikeThisWithValues(ClientQueryParams> queryParams, + public Mono>> moreLikeThisWithValues(ClientQueryParams> queryParams, T key, U mltDocumentValue, ValueGetter valueGetter) { @@ -147,13 +163,12 @@ public class LuceneIndexImpl implements LuceneIndex { indicizer.getKeyFieldName(), mltDocumentFields ) - .flatMap(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, - valueGetter - )); + .transform(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, + valueGetter)); } @Override - public Mono> moreLikeThisWithTransformer(ClientQueryParams> queryParams, + public Mono>> moreLikeThisWithTransformer(ClientQueryParams> queryParams, T key, U mltDocumentValue, ValueTransformer valueTransformer) { @@ -165,40 +180,51 @@ public class LuceneIndexImpl implements LuceneIndex { indicizer.getKeyFieldName(), mltDocumentFields ) - .flatMap(llSearchResult -> this.transformLuceneResultWithTransformer(llSearchResult, valueTransformer)); + .transform(llSearchResult -> this.transformLuceneResultWithTransformer(llSearchResult, + valueTransformer)); } @Override - public Mono> search(ClientQueryParams> queryParams) { + public Mono>> search(ClientQueryParams> queryParams) { return luceneIndex .search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName() ) - .flatMap(this::transformLuceneResultWithTransformer); + .transform(this::transformLuceneResultWithTransformer); } @Override - public Mono> searchWithValues(ClientQueryParams> queryParams, + public Mono>> searchWithValues( + ClientQueryParams> queryParams, ValueGetter valueGetter) { return luceneIndex - .search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName()) - .flatMap(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, valueGetter)); + .search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), + indicizer.getKeyFieldName()) + .transform(llSearchResult -> this.transformLuceneResultWithValues(llSearchResult, + valueGetter)); } @Override - public Mono> searchWithTransformer(ClientQueryParams> queryParams, + public Mono>> searchWithTransformer( + ClientQueryParams> queryParams, ValueTransformer valueTransformer) { return luceneIndex - .search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), indicizer.getKeyFieldName()) - .flatMap(llSearchResult -> this.transformLuceneResultWithTransformer(llSearchResult, valueTransformer)); + .search(resolveSnapshot(queryParams.snapshot()), queryParams.toQueryParams(), + indicizer.getKeyFieldName()) + .transform(llSearchResult -> this.transformLuceneResultWithTransformer(llSearchResult, + valueTransformer)); } @Override public Mono count(@Nullable CompositeSnapshot snapshot, Query query) { return this .search(ClientQueryParams.>builder().snapshot(snapshot).query(query).limit(0).build()) - .flatMap(tSearchResultKeys -> tSearchResultKeys.release().thenReturn(tSearchResultKeys.totalHitsCount())); + .map(searchResultKeysSend -> { + try (var searchResultKeys = searchResultKeysSend.receive()) { + return searchResultKeys.totalHitsCount(); + } + }); } @Override diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResult.java b/src/main/java/it/cavallium/dbengine/client/SearchResult.java index 342e0a7..40e6ba7 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResult.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResult.java @@ -1,5 +1,8 @@ package it.cavallium.dbengine.client; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLSearchResultShard; import java.util.Objects; @@ -8,35 +11,20 @@ import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SearchResult { +public final class SearchResult extends ResourceSupport, SearchResult> { - private static final Logger logger = LoggerFactory.getLogger(SearchResult.class); + private Flux> results; + private TotalHitsCount totalHitsCount; - private volatile boolean releaseCalled; - - private final Flux> results; - private final TotalHitsCount totalHitsCount; - private final Mono release; - - public SearchResult(Flux> results, TotalHitsCount totalHitsCount, Mono release) { + public SearchResult(Flux> results, TotalHitsCount totalHitsCount, + Drop> drop) { + super(new SearchResult.CloseOnDrop<>(drop)); this.results = results; this.totalHitsCount = totalHitsCount; - this.release = Mono.fromRunnable(() -> { - if (releaseCalled) { - logger.warn(this.getClass().getName() + "::release has been called twice!"); - } - releaseCalled = true; - }).then(release); } public static SearchResult empty() { - var sr = new SearchResult(Flux.empty(), TotalHitsCount.of(0, true), Mono.empty()); - sr.releaseCalled = true; - return sr; - } - - public Flux> resultsThenRelease() { - return Flux.usingWhen(Mono.just(true), _unused -> results, _unused -> release); + return new SearchResult(Flux.empty(), TotalHitsCount.of(0, true), d -> {}); } public Flux> results() { @@ -47,39 +35,40 @@ public final class SearchResult { return totalHitsCount; } - public Mono release() { - return release; - } - - @Override - public boolean equals(Object obj) { - if (obj == this) - return true; - if (obj == null || obj.getClass() != this.getClass()) - return false; - var that = (SearchResult) obj; - return Objects.equals(this.results, that.results) && Objects.equals(this.totalHitsCount, that.totalHitsCount) - && Objects.equals(this.release, that.release); - } - - @Override - public int hashCode() { - return Objects.hash(results, totalHitsCount, release); - } - @Override public String toString() { - return "SearchResult[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ", " + "release=" - + release + ']'; + return "SearchResult[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ']'; } - @SuppressWarnings("deprecation") @Override - protected void finalize() throws Throwable { - if (!releaseCalled) { - logger.warn(this.getClass().getName() + "::release has not been called before class finalization!"); - } - super.finalize(); + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); } + @Override + protected Owned> prepareSend() { + var results = this.results; + var totalHitsCount = this.totalHitsCount; + makeInaccessible(); + return drop -> new SearchResult<>(results, totalHitsCount, drop); + } + + private void makeInaccessible() { + this.results = null; + this.totalHitsCount = null; + } + + private static class CloseOnDrop implements Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(SearchResult obj) { + delegate.drop(obj); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java index adabd99..d784a14 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java @@ -1,5 +1,8 @@ package it.cavallium.dbengine.client; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLSearchResultShard; import it.cavallium.dbengine.database.collections.ValueGetter; @@ -11,42 +14,29 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @SuppressWarnings("unused") -public final class SearchResultKeys { +public final class SearchResultKeys extends ResourceSupport, SearchResultKeys> { private static final Logger logger = LoggerFactory.getLogger(SearchResultKeys.class); - private volatile boolean releaseCalled; + private Flux> results; + private TotalHitsCount totalHitsCount; - private final Flux> results; - private final TotalHitsCount totalHitsCount; - private final Mono release; - - public SearchResultKeys(Flux> results, TotalHitsCount totalHitsCount, Mono release) { + public SearchResultKeys(Flux> results, TotalHitsCount totalHitsCount, + Drop> drop) { + super(new SearchResultKeys.CloseOnDrop<>(drop)); this.results = results; this.totalHitsCount = totalHitsCount; - this.release = Mono.fromRunnable(() -> { - if (releaseCalled) { - logger.warn(this.getClass().getName() + "::release has been called twice!"); - } - releaseCalled = true; - }).then(release); } public static SearchResultKeys empty() { - var sr = new SearchResultKeys(Flux.empty(), TotalHitsCount.of(0, true), Mono.empty()); - sr.releaseCalled = true; - return sr; + return new SearchResultKeys(Flux.empty(), TotalHitsCount.of(0, true), d -> {}); } public SearchResult withValues(ValueGetter valuesGetter) { return new SearchResult<>(results.map(item -> new SearchResultItem<>(item.key(), item.key().flatMap(valuesGetter::get), item.score() - )), totalHitsCount, release); - } - - public Flux> resultsThenRelease() { - return Flux.usingWhen(Mono.just(true), _unused -> results, _unused -> release); + )), totalHitsCount, d -> this.close()); } public Flux> results() { @@ -57,39 +47,41 @@ public final class SearchResultKeys { return totalHitsCount; } - public Mono release() { - return release; - } - - @Override - public boolean equals(Object obj) { - if (obj == this) - return true; - if (obj == null || obj.getClass() != this.getClass()) - return false; - var that = (SearchResultKeys) obj; - return Objects.equals(this.results, that.results) && Objects.equals(this.totalHitsCount, that.totalHitsCount) - && Objects.equals(this.release, that.release); - } - - @Override - public int hashCode() { - return Objects.hash(results, totalHitsCount, release); - } - @Override public String toString() { - return "SearchResultKeys[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ", " + "release=" - + release + ']'; + return "SearchResultKeys[" + "results=" + results + ", " + "totalHitsCount=" + totalHitsCount + ']'; } - @SuppressWarnings("deprecation") @Override - protected void finalize() throws Throwable { - if (!releaseCalled) { - logger.warn(this.getClass().getName() + "::release has not been called before class finalization!"); + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned> prepareSend() { + var results = this.results; + var totalHitsCount = this.totalHitsCount; + makeInaccessible(); + return drop -> new SearchResultKeys<>(results, totalHitsCount, drop); + } + + private void makeInaccessible() { + this.results = null; + this.totalHitsCount = null; + } + + private static class CloseOnDrop implements Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(SearchResultKeys obj) { + delegate.drop(obj); } - super.finalize(); } } diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index d32b0e2..200f4e4 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -10,19 +10,25 @@ import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; import io.net5.util.internal.PlatformDependent; import it.cavallium.dbengine.database.collections.DatabaseStage; +import it.cavallium.dbengine.database.disk.LLIndexContext; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLLocalLuceneIndex; import it.cavallium.dbengine.database.disk.MemorySegmentUtils; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializationFunction; import it.cavallium.dbengine.lucene.RandomSortField; +import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; @@ -37,10 +43,19 @@ import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; +import org.apache.lucene.queries.mlt.MoreLikeThis; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.rocksdb.RocksDB; @@ -381,6 +396,74 @@ public class LLUtils { .doOnDiscard(Send.class, Send::close); } + public static Mono getMoreLikeThisQuery( + LLIndexSearcher indexSearcher, + @Nullable LLSnapshot snapshot, + LocalQueryParams localQueryParams, + Flux>> mltDocumentFieldsFlux) { + Query luceneAdditionalQuery; + try { + luceneAdditionalQuery = localQueryParams.query(); + } catch (Exception e) { + return Mono.error(e); + } + return mltDocumentFieldsFlux + .collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new) + .flatMap(mltDocumentFields -> { + mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty()); + if (mltDocumentFields.isEmpty()) { + return Mono.just(new LocalQueryParams(new MatchNoDocsQuery(), + localQueryParams.offset(), + localQueryParams.limit(), + localQueryParams.minCompetitiveScore(), + localQueryParams.sort(), + localQueryParams.scoreMode() + )); + } + new IndexSearcher + return indexSearcher.getIndexSearcher().search(snapshot, indexSearcher -> Mono.fromCallable(() -> { + var mlt = new MoreLikeThis(indexSearcher.getIndexReader()); + mlt.setAnalyzer(llLocalLuceneIndex.indexWriter.getAnalyzer()); + mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); + mlt.setMinTermFreq(1); + mlt.setMinDocFreq(3); + mlt.setMaxDocFreqPct(20); + mlt.setBoost(localQueryParams.scoreMode().needsScores()); + mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); + var similarity = llLocalLuceneIndex.getSimilarity(); + if (similarity instanceof TFIDFSimilarity) { + mlt.setSimilarity((TFIDFSimilarity) similarity); + } else { + LLLocalLuceneIndex.logger.trace(MARKER_ROCKSDB, "Using an unsupported similarity algorithm for MoreLikeThis:" + + " {}. You must use a similarity instance based on TFIDFSimilarity!", similarity); + } + + // Get the reference docId and apply it to MoreLikeThis, to generate the query + @SuppressWarnings({"unchecked", "rawtypes"}) + var mltQuery = mlt.like((Map) mltDocumentFields); + Query luceneQuery; + if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { + luceneQuery = new BooleanQuery.Builder() + .add(mltQuery, Occur.MUST) + .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) + .build(); + } else { + luceneQuery = mltQuery; + } + + return luceneQuery; + }) + .subscribeOn(Schedulers.boundedElastic()) + .map(luceneQuery -> new LocalQueryParams(luceneQuery, + localQueryParams.offset(), + localQueryParams.limit(), + localQueryParams.minCompetitiveScore(), + localQueryParams.sort(), + localQueryParams.scoreMode() + ))); + }); + } + public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java new file mode 100644 index 0000000..032c5d3 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java @@ -0,0 +1,81 @@ +package it.cavallium.dbengine.database.disk; + +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.lucene.searcher.LLSearchTransformer; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.IndexSearcher; + +public class LLIndexContext extends ResourceSupport { + + private LLIndexSearcher indexSearcher; + private LLSearchTransformer indexQueryTransformer; + + protected LLIndexContext(Send indexSearcher, + LLSearchTransformer indexQueryTransformer, + Drop drop) { + super(new CloseOnDrop(drop)); + this.indexSearcher = indexSearcher.receive(); + this.indexQueryTransformer = indexQueryTransformer; + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + var indexSearcher = this.indexSearcher.send(); + var indexQueryTransformer = this.indexQueryTransformer; + makeInaccessible(); + return drop -> new LLIndexContext(indexSearcher, indexQueryTransformer, drop); + } + + private void makeInaccessible() { + this.indexSearcher = null; + this.indexQueryTransformer = null; + } + + public IndexSearcher getIndexSearcher() { + if (!isOwned()) { + throw new UnsupportedOperationException("Closed"); + } + return indexSearcher.getIndexSearcher(); + } + + public IndexReader getIndexReader() { + if (!isOwned()) { + throw new UnsupportedOperationException("Closed"); + } + return indexSearcher.getIndexReader(); + } + + public LLSearchTransformer getIndexQueryTransformer() { + if (!isOwned()) { + throw new UnsupportedOperationException("Closed"); + } + return indexQueryTransformer; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(LLIndexContext obj) { + try { + if (obj.indexSearcher != null) obj.indexSearcher.close(); + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } +} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java new file mode 100644 index 0000000..97a795b --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java @@ -0,0 +1,199 @@ +package it.cavallium.dbengine.database.disk; + +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import org.apache.lucene.index.Fields; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReaderContext; +import org.apache.lucene.index.MultiReader; +import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.index.Term; + +public interface LLIndexContexts extends Resource { + + static LLIndexContexts of(List> indexSearchers) { + return new ShardedIndexSearchers(indexSearchers, d -> {}); + } + + static UnshardedIndexSearchers unsharded(Send indexSearcher) { + return new UnshardedIndexSearchers(indexSearcher, d -> {}); + } + + Iterable shards(); + + LLIndexContext shard(int shardIndex); + + IndexReader allShards(); + + class UnshardedIndexSearchers extends ResourceSupport + implements LLIndexContexts { + + private LLIndexContext indexSearcher; + + public UnshardedIndexSearchers(Send indexSearcher, Drop drop) { + super(new CloseOnDrop(drop)); + this.indexSearcher = indexSearcher.receive(); + } + + @Override + public Iterable shards() { + return Collections.singleton(indexSearcher); + } + + @Override + public LLIndexContext shard(int shardIndex) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("UnshardedIndexSearchers must be owned to be used")); + } + if (shardIndex != -1) { + throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid, this is a unsharded index"); + } + return indexSearcher; + } + + @Override + public IndexReader allShards() { + return indexSearcher.getIndexReader(); + } + + public LLIndexContext shard() { + return this.shard(0); + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + Send indexSearcher = this.indexSearcher.send(); + this.makeInaccessible(); + return drop -> new UnshardedIndexSearchers(indexSearcher, drop); + } + + private void makeInaccessible() { + this.indexSearcher = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(UnshardedIndexSearchers obj) { + try { + if (obj.indexSearcher != null) obj.indexSearcher.close(); + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } + } + + class ShardedIndexSearchers extends ResourceSupport + implements LLIndexContexts { + + private List indexSearchers; + + public ShardedIndexSearchers(List> indexSearchers, Drop drop) { + super(new CloseOnDrop(drop)); + this.indexSearchers = new ArrayList<>(indexSearchers.size()); + for (Send indexSearcher : indexSearchers) { + this.indexSearchers.add(indexSearcher.receive()); + } + } + + @Override + public Iterable shards() { + return Collections.unmodifiableList(indexSearchers); + } + + @Override + public LLIndexContext shard(int shardIndex) { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); + } + if (shardIndex < 0) { + throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid"); + } + return indexSearchers.get(shardIndex); + } + + @Override + public IndexReader allShards() { + var irs = new IndexReader[indexSearchers.size()]; + for (int i = 0, s = indexSearchers.size(); i < s; i++) { + irs[i] = indexSearchers.get(i).getIndexReader(); + } + Object2IntOpenHashMap indexes = new Object2IntOpenHashMap<>(); + for (int i = 0; i < irs.length; i++) { + indexes.put(irs[i], i); + } + try { + return new MultiReader(irs, Comparator.comparingInt(indexes::getInt), true); + } catch (IOException ex) { + // This shouldn't happen + throw new UncheckedIOException(ex); + } + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + List> indexSearchers = new ArrayList<>(this.indexSearchers.size()); + for (LLIndexContext indexSearcher : this.indexSearchers) { + indexSearchers.add(indexSearcher.send()); + } + this.makeInaccessible(); + return drop -> new ShardedIndexSearchers(indexSearchers, drop); + } + + private void makeInaccessible() { + this.indexSearchers = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(ShardedIndexSearchers obj) { + try { + if (obj.indexSearchers != null) { + for (LLIndexContext indexSearcher : obj.indexSearchers) { + indexSearcher.close(); + } + } + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } + } +} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java index 7bf4f68..e534357 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -85,10 +85,12 @@ public class LLIndexSearcher extends ResourceSupport>> mltDocumentFieldsFlux) { - return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) + return LLUtils + .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) .flatMap(modifiedLocalQuery -> searcherManager .retrieveSearcher(snapshot) - .flatMap(indexSearcher -> localSearcher.collect(indexSearcher, modifiedLocalQuery, keyFieldName)) + .transform(indexSearcher -> localSearcher.collect(indexSearcher, modifiedLocalQuery, keyFieldName)) ) .map(resultToReceive -> { var result = resultToReceive.receive(); @@ -343,11 +335,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { .doOnDiscard(Send.class, Send::close); } - public Mono distributedMoreLikeThis(@Nullable LLSnapshot snapshot, + public Mono getMoreLikeThisTransformer(@Nullable LLSnapshot snapshot, QueryParams queryParams, Flux>> mltDocumentFieldsFlux, LuceneMultiSearcher shardSearcher) { - return getMoreLikeThisQuery(snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) + return LLUtils + .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) .flatMap(modifiedLocalQuery -> searcherManager .retrieveSearcher(snapshot) .flatMap(indexSearcher -> shardSearcher.searchOn(indexSearcher, modifiedLocalQuery)) @@ -355,78 +348,14 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { .doOnDiscard(Send.class, Send::close); } - public Mono getMoreLikeThisQuery(@Nullable LLSnapshot snapshot, - LocalQueryParams localQueryParams, - Flux>> mltDocumentFieldsFlux) { - Query luceneAdditionalQuery; - try { - luceneAdditionalQuery = localQueryParams.query(); - } catch (Exception e) { - return Mono.error(e); - } - return mltDocumentFieldsFlux - .collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new) - .flatMap(mltDocumentFields -> { - mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty()); - if (mltDocumentFields.isEmpty()) { - return Mono.just(new LocalQueryParams(new MatchNoDocsQuery(), - localQueryParams.offset(), - localQueryParams.limit(), - localQueryParams.minCompetitiveScore(), - localQueryParams.sort(), - localQueryParams.scoreMode() - )); - } - return this.searcherManager.search(snapshot, indexSearcher -> Mono.fromCallable(() -> { - var mlt = new MoreLikeThis(indexSearcher.getIndexReader()); - mlt.setAnalyzer(indexWriter.getAnalyzer()); - mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); - mlt.setMinTermFreq(1); - mlt.setMinDocFreq(3); - mlt.setMaxDocFreqPct(20); - mlt.setBoost(localQueryParams.scoreMode().needsScores()); - mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); - var similarity = getSimilarity(); - if (similarity instanceof TFIDFSimilarity) { - mlt.setSimilarity((TFIDFSimilarity) similarity); - } else { - logger.trace(MARKER_ROCKSDB, "Using an unsupported similarity algorithm for MoreLikeThis:" - + " {}. You must use a similarity instance based on TFIDFSimilarity!", similarity); - } - - // Get the reference docId and apply it to MoreLikeThis, to generate the query - @SuppressWarnings({"unchecked", "rawtypes"}) - var mltQuery = mlt.like((Map) mltDocumentFields); - Query luceneQuery; - if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { - luceneQuery = new BooleanQuery.Builder() - .add(mltQuery, Occur.MUST) - .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) - .build(); - } else { - luceneQuery = mltQuery; - } - - return luceneQuery; - }) - .subscribeOn(Schedulers.boundedElastic()) - .map(luceneQuery -> new LocalQueryParams(luceneQuery, - localQueryParams.offset(), - localQueryParams.limit(), - localQueryParams.minCompetitiveScore(), - localQueryParams.sort(), - localQueryParams.scoreMode() - ))); - }); - } - @Override public Mono> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); return searcherManager .retrieveSearcher(snapshot) - .flatMap(indexSearcher -> localSearcher.collect(indexSearcher, localQueryParams, keyFieldName)) + .transform(indexSearcher -> localSearcher.collect(indexSearcher, localQueryParams, + LLSearchTransformer.NO_TRANSFORMATION, keyFieldName)) .map(resultToReceive -> { var result = resultToReceive.receive(); return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); @@ -434,13 +363,13 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { .doOnDiscard(Send.class, Send::close); } - public Mono distributedSearch(@Nullable LLSnapshot snapshot, - QueryParams queryParams, - LuceneMultiSearcher shardSearcher) { - LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); + public Mono> retrieveContext(@Nullable LLSnapshot snapshot, + @Nullable LLSearchTransformer indexQueryTransformer) { return searcherManager .retrieveSearcher(snapshot) - .flatMap(indexSearcher -> shardSearcher.searchOn(indexSearcher, localQueryParams)) + .map(indexSearcherToReceive -> new LLIndexContext(indexSearcherToReceive, + Objects.requireNonNullElse(indexQueryTransformer, LLSearchTransformer.NO_TRANSFORMATION), + d -> {}).send()) .doOnDiscard(Send.class, Send::close); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java index e1f22de..da10690 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database.disk; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.IndicizerAnalyzers; import it.cavallium.dbengine.client.IndicizerSimilarities; import it.cavallium.dbengine.client.LuceneOptions; @@ -11,6 +12,7 @@ import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.LLTerm; import it.cavallium.dbengine.lucene.LuceneUtils; import it.cavallium.dbengine.lucene.searcher.AdaptiveLuceneMultiSearcher; +import it.cavallium.dbengine.lucene.searcher.LLSearchTransformer; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher; import java.io.IOException; @@ -26,6 +28,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -86,6 +89,20 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { return luceneIndices[0].getLuceneIndexName(); } + private Flux> getIndexContexts(LLSnapshot snapshot, + Function indexQueryTransformers) { + return Flux + .fromArray(luceneIndices) + .index() + // Resolve the snapshot of each shard + .flatMap(tuple -> Mono + .fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1())) + .flatMap(luceneSnapshot -> tuple.getT2().retrieveContext( + luceneSnapshot.orElse(null), indexQueryTransformers.apply(tuple.getT2())) + ) + ); + } + @Override public Mono addDocument(LLTerm id, LLDocument doc) { return getLuceneIndex(id).addDocument(id, doc); @@ -176,12 +193,23 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { } @Override - public Mono moreLikeThis(@Nullable LLSnapshot snapshot, + public Mono> moreLikeThis(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName, Flux>> mltDocumentFields) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - record LuceneIndexWithSnapshot(LLLocalLuceneIndex luceneIndex, Optional snapshot) {} + Flux> serchers = this + .getIndexContexts(snapshot, luceneIndex -> LLSearchTransformer.NO_TRANSFORMATION); + + // Collect all the shards results into a single global result + return multiSearcher + .collect(serchers, localQueryParams, keyFieldName) + // Transform the result type + .map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), + d -> result.close()).send(); + }); return multiSearcher // Create shard searcher @@ -205,31 +233,21 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { } @Override - public Mono search(@Nullable LLSnapshot snapshot, + public Mono> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - record LuceneIndexWithSnapshot(LLLocalLuceneIndex luceneIndex, Optional snapshot) {} + Flux> serchers = getIndexContexts(snapshot); + // Collect all the shards results into a single global result return multiSearcher - // Create shard searcher - .createShardSearcher(localQueryParams) - .flatMap(shardSearcher -> Flux - // Iterate the indexed shards - .fromArray(luceneIndices).index() - // Resolve the snapshot of each shard - .flatMap(tuple -> Mono - .fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1())) - .map(luceneSnapshot -> new LuceneIndexWithSnapshot(tuple.getT2(), luceneSnapshot)) - ) - // Execute the query and collect it using the shard searcher - .flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex() - .distributedSearch(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, shardSearcher)) - // Collect all the shards results into a single global result - .then(shardSearcher.collect(localQueryParams, keyFieldName, luceneSearcherScheduler)) - ) - // Fix the result type - .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())); + .collect(serchers, localQueryParams, keyFieldName) + // Transform the result type + .map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), + d -> result.close()).send(); + }); } @Override @@ -289,4 +307,32 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { public boolean isLowMemoryMode() { return luceneIndices[0].isLowMemoryMode(); } + + private class MoreLikeThisTransformer implements LLSearchTransformer { + + private final LLLocalLuceneIndex luceneIndex; + private final LLSnapshot snapshot; + private final String keyFieldName; + private final Flux>> mltDocumentFields; + + public MoreLikeThisTransformer(LLLocalLuceneIndex luceneIndex, + @Nullable LLSnapshot snapshot, + String keyFieldName, + Flux>> mltDocumentFields) { + this.luceneIndex = luceneIndex; + this.snapshot = snapshot; + this.keyFieldName = keyFieldName; + this.mltDocumentFields = mltDocumentFields; + } + + @Override + public Mono transform(Mono queryParamsMono) { + return queryParamsMono + .flatMap(queryParams -> { + luceneIndex.getMoreLikeThisTransformer(snapshot, queryParams, mltDocumentFields, ); + }); + LLLocalMultiLuceneIndex.this. + return null; + } + } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index 906ceac..773a166 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -1,41 +1,32 @@ package it.cavallium.dbengine.lucene; -import io.net5.buffer.api.Resource; -import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.client.IndicizerAnalyzers; import it.cavallium.dbengine.client.IndicizerSimilarities; -import it.cavallium.dbengine.client.query.BasicType; import it.cavallium.dbengine.client.query.QueryParser; import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; -import it.cavallium.dbengine.database.LLScoreMode; import it.cavallium.dbengine.database.collections.DatabaseMapDictionary; import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep; import it.cavallium.dbengine.database.collections.ValueGetter; +import it.cavallium.dbengine.database.disk.LLIndexContexts; import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer; import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity; import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer; -import it.cavallium.dbengine.lucene.searcher.IndexSearchers; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; -import it.cavallium.dbengine.lucene.searcher.LuceneMultiSearcher; import it.cavallium.dbengine.lucene.similarity.NGramSimilarity; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Set; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.LowerCaseFilter; @@ -65,11 +56,9 @@ import org.novasearch.lucene.search.similarities.BM25Similarity.BM25Model; import org.novasearch.lucene.search.similarities.LdpSimilarity; import org.novasearch.lucene.search.similarities.LtcSimilarity; import org.novasearch.lucene.search.similarities.RobertsonSimilarity; -import org.reactivestreams.Publisher; import org.warp.commonutils.log.Logger; import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; import reactor.util.concurrent.Queues; @@ -367,7 +356,7 @@ public class LuceneUtils { } public static Flux convertHits(Flux hitsFlux, - IndexSearchers indexSearchers, + LLIndexContexts indexSearchers, String keyFieldName, boolean preserveOrder) { if (preserveOrder) { @@ -392,7 +381,7 @@ public class LuceneUtils { @Nullable private static LLKeyScore mapHitBlocking(ScoreDoc hit, - IndexSearchers indexSearchers, + LLIndexContexts indexSearchers, String keyFieldName) { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called mapHitBlocking in a nonblocking thread"); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java index 41e63c4..50a92d5 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java @@ -1,34 +1,24 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; -import org.apache.lucene.search.IndexSearcher; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; -import reactor.core.scheduler.Schedulers; public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher { private static final LuceneLocalSearcher localSearcher = new SimpleLuceneLocalSearcher(); - private static final LuceneLocalSearcher unscoredPagedLuceneLocalSearcher = new LocalLuceneWrapper(new UnscoredUnsortedContinuousLuceneMultiSearcher(), d -> {}); - private static final LuceneLocalSearcher countSearcher = new CountLuceneLocalSearcher(); @Override - public Mono> collect(Mono> indexSearcher, + public Mono> collect(Mono> indexSearcher, LocalQueryParams queryParams, String keyFieldName) { - Mono> collectionMono; if (queryParams.limit() == 0) { - collectionMono = countSearcher.collect(indexSearcher, queryParams, keyFieldName); - } else if (!queryParams.isScored() && queryParams.offset() == 0 && queryParams.limit() >= 2147483630 - && !queryParams.isSorted()) { - collectionMono = unscoredPagedLuceneLocalSearcher.collect(indexSearcher, queryParams, keyFieldName); + return countSearcher.collect(indexSearcher, queryParams, keyFieldName); } else { - collectionMono = localSearcher.collect(indexSearcher, queryParams, keyFieldName); + return localSearcher.collect(indexSearcher, queryParams, keyFieldName); } - return Mono.fromRunnable(LLUtils::ensureBlocking).then(collectionMono); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java index eca9414..f26170f 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java @@ -1,31 +1,32 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexContext; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; public class AdaptiveLuceneMultiSearcher implements LuceneMultiSearcher { - private static final LuceneMultiSearcher scoredLuceneMultiSearcher = new ScoredLuceneMultiSearcher(); + private static final LuceneMultiSearcher countLuceneMultiSearcher + = new SimpleUnsortedUnscoredLuceneMultiSearcher(new CountLuceneLocalSearcher()); - private static final LuceneMultiSearcher unscoredPagedLuceneMultiSearcher = new UnscoredPagedLuceneMultiSearcher(); + private static final LuceneMultiSearcher scoredSimpleLuceneShardSearcher + = new ScoredSimpleLuceneShardSearcher(); - private static final LuceneMultiSearcher unscoredIterableLuceneMultiSearcher = new UnscoredUnsortedContinuousLuceneMultiSearcher(); - - private static final LuceneMultiSearcher countLuceneMultiSearcher = new CountLuceneMultiSearcher(); + private static final LuceneMultiSearcher unscoredPagedLuceneMultiSearcher + = new SimpleUnsortedUnscoredLuceneMultiSearcher(new SimpleLuceneLocalSearcher()); @Override - public Mono> createShardSearcher(LocalQueryParams queryParams) { - Mono> shardSearcherCreationMono; - if (queryParams.limit() <= 0) { - shardSearcherCreationMono = countLuceneMultiSearcher.createShardSearcher(queryParams); - } else if (queryParams.isScored()) { - shardSearcherCreationMono = scoredLuceneMultiSearcher.createShardSearcher(queryParams); - } else if (queryParams.offset() == 0 && queryParams.limit() >= 2147483630 && !queryParams.isSorted()) { - shardSearcherCreationMono = unscoredIterableLuceneMultiSearcher.createShardSearcher(queryParams); + public Mono> collect(Flux> indexSearchersFlux, + LocalQueryParams queryParams, + String keyFieldName) { + if (queryParams.limit() == 0) { + return countLuceneMultiSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); + } else if (queryParams.isSorted() || queryParams.isScored()) { + return scoredSimpleLuceneShardSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); } else { - shardSearcherCreationMono = unscoredPagedLuceneMultiSearcher.createShardSearcher(queryParams); + return unscoredPagedLuceneMultiSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); } - return Mono.fromRunnable(LLUtils::ensureBlocking).then(shardSearcherCreationMono); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java index 6343f0f..2d4ba85 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java @@ -1,10 +1,9 @@ package it.cavallium.dbengine.lucene.searcher; -import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; -import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -13,7 +12,7 @@ import reactor.core.scheduler.Schedulers; public class CountLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono> collect(Mono> indexSearcherMono, + public Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, String keyFieldName) { return Mono diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java deleted file mode 100644 index 97603ca..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneMultiSearcher.java +++ /dev/null @@ -1,88 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import io.net5.buffer.api.Drop; -import io.net5.buffer.api.Owned; -import io.net5.buffer.api.Send; -import io.net5.buffer.api.internal.ResourceSupport; -import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; -import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexSearcher; -import java.util.concurrent.atomic.AtomicLong; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public class CountLuceneMultiSearcher implements LuceneMultiSearcher { - - @Override - public Mono> createShardSearcher(LocalQueryParams queryParams) { - return Mono.fromCallable(() -> new CountLuceneShardSearcher(new AtomicLong(0), d -> {}).send()); - } - - private static class CountLuceneShardSearcher extends - ResourceSupport implements LuceneMultiSearcher { - - private AtomicLong totalHitsCount; - - public CountLuceneShardSearcher(AtomicLong totalHitsCount, Drop drop) { - super(new CloseOnDrop(drop)); - this.totalHitsCount = totalHitsCount; - } - - @Override - public Mono searchOn(Send indexSearcher, LocalQueryParams queryParams) { - return Mono - .fromCallable(() -> { - try (var is = indexSearcher.receive()) { - if (!isOwned()) { - throw attachTrace(new IllegalStateException("CountLuceneMultiSearcher must be owned to be used")); - } - LLUtils.ensureBlocking(); - totalHitsCount.addAndGet(is.getIndexSearcher().count(queryParams.query())); - return null; - } - }); - } - - @Override - public Mono> collect(LocalQueryParams queryParams, String keyFieldName) { - return Mono.fromCallable(() -> { - if (!isOwned()) { - throw attachTrace(new IllegalStateException("CountLuceneMultiSearcher must be owned to be used")); - } - LLUtils.ensureBlocking(); - return new LuceneSearchResult(TotalHitsCount.of(totalHitsCount.get(), true), Flux.empty(), d -> {}) - .send(); - }); - } - - @Override - protected RuntimeException createResourceClosedException() { - return new IllegalStateException("Closed"); - } - - @Override - protected Owned prepareSend() { - var totalHitsCount = this.totalHitsCount; - makeInaccessible(); - return drop -> new CountLuceneShardSearcher(totalHitsCount, drop); - } - - private void makeInaccessible() { - this.totalHitsCount = null; - } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(CountLuceneShardSearcher obj) { - delegate.drop(obj); - } - } - } -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/FirstPageResults.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/FirstPageResults.java new file mode 100644 index 0000000..365233e --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/FirstPageResults.java @@ -0,0 +1,8 @@ +package it.cavallium.dbengine.lucene.searcher; + +import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; +import it.cavallium.dbengine.database.LLKeyScore; +import reactor.core.publisher.Flux; + +record FirstPageResults(TotalHitsCount totalHitsCount, Flux firstPageHitsFlux, + CurrentPageInfo nextPageInfo) {} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java deleted file mode 100644 index 05925e8..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/IndexSearchers.java +++ /dev/null @@ -1,144 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import io.net5.buffer.UnpooledDirectByteBuf; -import io.net5.buffer.api.Buffer; -import io.net5.buffer.api.Drop; -import io.net5.buffer.api.Owned; -import io.net5.buffer.api.Resource; -import io.net5.buffer.api.Send; -import io.net5.buffer.api.internal.ResourceSupport; -import it.cavallium.dbengine.database.LLRange; -import it.cavallium.dbengine.database.disk.LLIndexSearcher; -import java.util.ArrayList; -import java.util.List; -import org.apache.lucene.search.IndexSearcher; - -public interface IndexSearchers extends Resource { - - static IndexSearchers of(List indexSearchers) { - return new ShardedIndexSearchers(indexSearchers, d -> {}); - } - - static UnshardedIndexSearchers unsharded(Send indexSearcher) { - return new UnshardedIndexSearchers(indexSearcher, d -> {}); - } - - LLIndexSearcher shard(int shardIndex); - - class UnshardedIndexSearchers extends ResourceSupport - implements IndexSearchers { - - private LLIndexSearcher indexSearcher; - - public UnshardedIndexSearchers(Send indexSearcher, Drop drop) { - super(new CloseOnDrop(drop)); - this.indexSearcher = indexSearcher.receive(); - } - - @Override - public LLIndexSearcher shard(int shardIndex) { - if (!isOwned()) { - throw attachTrace(new IllegalStateException("UnshardedIndexSearchers must be owned to be used")); - } - if (shardIndex != -1) { - throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid, this is a unsharded index"); - } - return indexSearcher; - } - - public LLIndexSearcher shard() { - return this.shard(0); - } - - @Override - protected RuntimeException createResourceClosedException() { - return new IllegalStateException("Closed"); - } - - @Override - protected Owned prepareSend() { - Send indexSearcher = this.indexSearcher.send(); - this.makeInaccessible(); - return drop -> new UnshardedIndexSearchers(indexSearcher, drop); - } - - private void makeInaccessible() { - this.indexSearcher = null; - } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(UnshardedIndexSearchers obj) { - try { - if (obj.indexSearcher != null) obj.indexSearcher.close(); - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } - } - - class ShardedIndexSearchers extends ResourceSupport - implements IndexSearchers { - - private List indexSearchers; - - public ShardedIndexSearchers(List indexSearchers, Drop drop) { - super(new CloseOnDrop(drop)); - this.indexSearchers = indexSearchers; - } - - @Override - public LLIndexSearcher shard(int shardIndex) { - if (!isOwned()) { - throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); - } - if (shardIndex < 0) { - throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid"); - } - return indexSearchers.get(shardIndex); - } - - @Override - protected RuntimeException createResourceClosedException() { - return new IllegalStateException("Closed"); - } - - @Override - protected Owned prepareSend() { - List indexSearchers = this.indexSearchers; - this.makeInaccessible(); - return drop -> new ShardedIndexSearchers(indexSearchers, drop); - } - - private void makeInaccessible() { - this.indexSearchers = null; - } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(ShardedIndexSearchers obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } - } -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java new file mode 100644 index 0000000..0383297 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java @@ -0,0 +1,10 @@ +package it.cavallium.dbengine.lucene.searcher; + +import reactor.core.publisher.Mono; + +public interface LLSearchTransformer { + + LLSearchTransformer NO_TRANSFORMATION = queryParamsMono -> queryParamsMono; + + Mono transform(Mono queryParamsMono); +} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java index 3346666..767b241 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java @@ -1,7 +1,7 @@ package it.cavallium.dbengine.lucene.searcher; -import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Mono; @@ -12,7 +12,7 @@ public interface LuceneLocalSearcher { * @param queryParams the query parameters * @param keyFieldName the name of the key field */ - Mono> collect(Mono> indexSearcherMono, + Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, String keyFieldName); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java index 16fa9cb..cf75cac 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java @@ -1,12 +1,10 @@ package it.cavallium.dbengine.lucene.searcher; -import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; +import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; -import org.jetbrains.annotations.NotNull; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; public interface LuceneMultiSearcher extends LuceneLocalSearcher { @@ -14,7 +12,7 @@ public interface LuceneMultiSearcher extends LuceneLocalSearcher { * @param queryParams the query parameters * @param keyFieldName the name of the key field */ - Mono> collect(Flux> indexSearchersFlux, + Mono> collect(Flux> indexSearchersFlux, LocalQueryParams queryParams, String keyFieldName); @@ -24,7 +22,7 @@ public interface LuceneMultiSearcher extends LuceneLocalSearcher { * @param keyFieldName the name of the key field */ @Override - default Mono> collect(Mono> indexSearcherMono, + default Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, String keyFieldName) { return this.collect(indexSearcherMono.flux(), queryParams, keyFieldName); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/PageData.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageData.java new file mode 100644 index 0000000..f5be5ee --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageData.java @@ -0,0 +1,5 @@ +package it.cavallium.dbengine.lucene.searcher; + +import org.apache.lucene.search.TopDocs; + +record PageData(TopDocs topDocs, CurrentPageInfo nextPageInfo) {} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java deleted file mode 100644 index 97064be..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredLuceneMultiSearcher.java +++ /dev/null @@ -1,37 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT; -import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; - -import it.cavallium.dbengine.lucene.LuceneUtils; -import org.apache.lucene.search.CollectorManager; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopFieldCollector; -import reactor.core.publisher.Mono; - -public class ScoredLuceneMultiSearcher implements LuceneMultiSearcher { - - @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { - return Mono - .fromCallable(() -> { - Sort luceneSort = queryParams.sort(); - if (luceneSort == null) { - luceneSort = Sort.RELEVANCE; - } - PaginationInfo paginationInfo; - if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); - } else { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); - } - CollectorManager sharedManager = new ScoringShardsCollectorManager(luceneSort, - LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), - null, LuceneUtils.totalHitsThreshold(), LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()), - LuceneUtils.safeLongToInt(paginationInfo.firstPageLimit())); - return new ScoredSimpleLuceneShardSearcher(sharedManager, queryParams.query(), paginationInfo); - }); - } - -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java index 2d3fab5..97fe3f9 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java @@ -1,159 +1,185 @@ package it.cavallium.dbengine.lucene.searcher; -import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS; +import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT; +import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLKeyScore; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexContext; +import it.cavallium.dbengine.database.disk.LLIndexContexts; import it.cavallium.dbengine.lucene.LuceneUtils; -import it.unimi.dsi.fastutil.objects.ObjectArrayList; -import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; -import org.apache.lucene.search.CollectorManager; -import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopFieldCollector; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; -class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { +public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { - private final Object lock = new Object(); - private final List indexSearchersArray = new ArrayList<>(); - private final List> indexSearcherReleasersArray = new ArrayList<>(); - private final List collectors = new ArrayList<>(); - private final CollectorManager firstPageSharedManager; - private final Query luceneQuery; - private final PaginationInfo paginationInfo; - - public ScoredSimpleLuceneShardSearcher(CollectorManager firstPageSharedManager, - Query luceneQuery, PaginationInfo paginationInfo) { - this.firstPageSharedManager = firstPageSharedManager; - this.luceneQuery = luceneQuery; - this.paginationInfo = paginationInfo; + public ScoredSimpleLuceneShardSearcher() { } @Override - public Mono searchOn(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, + public Mono> collect(Flux> indexSearchersFlux, LocalQueryParams queryParams, - Scheduler scheduler) { - return Mono.fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called searchOn in a nonblocking thread"); - } - TopFieldCollector collector; - synchronized (lock) { - //noinspection BlockingMethodInNonBlockingContext - collector = firstPageSharedManager.newCollector(); - indexSearchersArray.add(indexSearcher); - indexSearcherReleasersArray.add(releaseIndexSearcher); - collectors.add(collector); - } - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(luceneQuery, collector); - return null; - }).subscribeOn(scheduler); + String keyFieldName) { + Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); + PaginationInfo paginationInfo = getPaginationInfo(queryParams); + + var indexSearchersMono = indexSearchersFlux.collectList().map(LLIndexContexts::of); + + return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this + // Search first page results + .searchFirstPage(indexSearchers, queryParams, paginationInfo) + // Compute the results of the first page + .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers, + keyFieldName, queryParams)) + // Compute other results + .transform(firstResult -> this.computeOtherResults(firstResult, indexSearchers, queryParams, keyFieldName)) + // Ensure that one LuceneSearchResult is always returned + .single(), + false); } - @Override - public Mono collect(LocalQueryParams queryParams, String keyFieldName, Scheduler collectorScheduler) { - if (Schedulers.isInNonBlockingThread()) { - return Mono.error(() -> new UnsupportedOperationException("Called collect in a nonblocking thread")); + private Sort getSort(LocalQueryParams queryParams) { + Sort luceneSort = queryParams.sort(); + if (luceneSort == null) { + luceneSort = Sort.RELEVANCE; } - if (!queryParams.isScored()) { - return Mono.error(() -> new UnsupportedOperationException("Can't execute an unscored query" - + " with a scored lucene shard searcher")); + return luceneSort; + } + + /** + * Get the pagination info + */ + private PaginationInfo getPaginationInfo(LocalQueryParams queryParams) { + if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); + } else { + return new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); } + } + + /** + * Search effectively the raw results of the first page + */ + private Mono searchFirstPage(LLIndexContexts indexSearchers, + LocalQueryParams queryParams, + PaginationInfo paginationInfo) { + var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); + var pagination = !paginationInfo.forceSinglePage(); + var resultsOffset = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()); + return Mono + .fromSupplier(() -> new CurrentPageInfo(null, limit, 0)) + .flatMap(s -> this.searchPage(queryParams, indexSearchers, pagination, resultsOffset, s)); + } + + /** + * Compute the results of the first page, extracting useful data + */ + private Mono computeFirstPageResults(Mono firstPageDataMono, + LLIndexContexts indexSearchers, + String keyFieldName, + LocalQueryParams queryParams) { + return firstPageDataMono.map(firstPageData -> { + var totalHitsCount = LuceneUtils.convertTotalHitsCount(firstPageData.topDocs().totalHits); + + Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageData.topDocs().scoreDocs), + indexSearchers, keyFieldName, true) + .take(queryParams.limit(), true); + + CurrentPageInfo nextPageInfo = firstPageData.nextPageInfo(); + + return new FirstPageResults(totalHitsCount, firstPageHitsFlux, nextPageInfo); + }); + } + + private Mono> computeOtherResults(Mono firstResultMono, + LLIndexContexts indexSearchers, + LocalQueryParams queryParams, + String keyFieldName) { + return firstResultMono.map(firstResult -> { + var totalHitsCount = firstResult.totalHitsCount(); + var firstPageHitsFlux = firstResult.firstPageHitsFlux(); + var secondPageInfo = firstResult.nextPageInfo(); + + Flux nextHitsFlux = searchOtherPages(indexSearchers, queryParams, keyFieldName, secondPageInfo); + + Flux combinedFlux = firstPageHitsFlux.concatWith(nextHitsFlux); + return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> indexSearchers.close()).send(); + }); + } + + /** + * Search effectively the merged raw results of the next pages + */ + private Flux searchOtherPages(LLIndexContexts indexSearchers, + LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { + return Flux + .defer(() -> { + AtomicReference currentPageInfoRef = new AtomicReference<>(secondPageInfo); + return Flux + .defer(() -> searchPage(queryParams, indexSearchers, true, 0, currentPageInfoRef.get())) + .doOnNext(s -> currentPageInfoRef.set(s.nextPageInfo())) + .repeatWhen(s -> s.takeWhile(n -> n > 0)); + }) + .subscribeOn(Schedulers.boundedElastic()) + .map(PageData::topDocs) + .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) + .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, + keyFieldName, true)); + } + + /** + * + * @param resultsOffset offset of the resulting topDocs. Useful if you want to + * skip the first n results in the first page + */ + private Mono searchPage(LocalQueryParams queryParams, + LLIndexContexts indexSearchers, + boolean allowPagination, + int resultsOffset, + CurrentPageInfo s) { return Mono .fromCallable(() -> { - TopDocs result; - Mono release; - synchronized (lock) { - //noinspection BlockingMethodInNonBlockingContext - result = firstPageSharedManager.reduce(collectors); - release = Mono.when(indexSearcherReleasersArray); + LLUtils.ensureBlocking(); + if (resultsOffset < 0) { + throw new IndexOutOfBoundsException(resultsOffset); } - IndexSearchers indexSearchers; - synchronized (lock) { - indexSearchers = IndexSearchers.of(indexSearchersArray); + if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { + var sort = getSort(queryParams); + var limit = s.currentPageLimit(); + var totalHitsThreshold = LuceneUtils.totalHitsThreshold(); + return new ScoringShardsCollectorManager(sort, limit, null, + totalHitsThreshold, resultsOffset, s.currentPageLimit()); + } else { + return null; } - Flux firstPageHits = LuceneUtils - .convertHits(Flux.fromArray(result.scoreDocs), indexSearchers, keyFieldName, collectorScheduler, true); - - Flux nextHits; - nextHits = Flux - .generate( - () -> new CurrentPageInfo(LuceneUtils.getLastFieldDoc(result.scoreDocs), - paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1), - (s, emitter) -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - - if (s.last() != null && s.remainingLimit() > 0) { - Sort luceneSort = queryParams.sort(); - if (luceneSort == null) { - luceneSort = Sort.RELEVANCE; - } - CollectorManager sharedManager - = new ScoringShardsCollectorManager(luceneSort, s.currentPageLimit(), - (FieldDoc) s.last(), LuceneUtils.totalHitsThreshold(), 0, s.currentPageLimit()); - - try { - var collectors = new ObjectArrayList(indexSearchersArray.size()); - for (IndexSearcher indexSearcher : indexSearchersArray) { - //noinspection BlockingMethodInNonBlockingContext - TopFieldCollector collector = sharedManager.newCollector(); - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(luceneQuery, collector); - - collectors.add(collector); - } - - //noinspection BlockingMethodInNonBlockingContext - var pageTopDocs = sharedManager.reduce(collectors); - var pageLastDoc = LuceneUtils.getLastFieldDoc(pageTopDocs.scoreDocs); - emitter.next(pageTopDocs); - - s = new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), - s.pageIndex() + 1); - } catch (IOException ex) { - emitter.error(ex); - s = EMPTY_STATUS; - } - } else { - emitter.complete(); - s = EMPTY_STATUS; - } - return s; - }) - .subscribeOn(collectorScheduler) - .transform(flux -> { - if (paginationInfo.forceSinglePage() - || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { - return Flux.empty(); - } else { - return flux; - } - }) - .flatMapIterable(topFieldDoc -> Arrays.asList(topFieldDoc.scoreDocs)) - .transform(scoreDocs -> LuceneUtils.convertHits(scoreDocs, - indexSearchers, keyFieldName, collectorScheduler, true)); - - return new LuceneSearchResult(LuceneUtils.convertTotalHitsCount(result.totalHits), - firstPageHits - .concatWith(nextHits), - //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), - release - ); }) - .subscribeOn(collectorScheduler); + .flatMap(sharedManager -> Flux + .fromIterable(indexSearchers.shards()) + .flatMap(shard -> Mono.fromCallable(() -> { + var collector = sharedManager.newCollector(); + shard.getIndexSearcher().search(queryParams.query(), collector); + return collector; + })) + .collectList() + .flatMap(collectors -> Mono.fromCallable(() -> { + var pageTopDocs = sharedManager.reduce(collectors); + var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); + long nextRemainingLimit; + if (allowPagination) { + nextRemainingLimit = s.remainingLimit() - s.currentPageLimit(); + } else { + nextRemainingLimit = 0L; + } + var nextPageIndex = s.pageIndex() + 1; + var nextPageInfo = new CurrentPageInfo(pageLastDoc, nextRemainingLimit, nextPageIndex); + return new PageData(pageTopDocs, nextPageInfo); + })) + ); } - } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index 284b006..7c1526a 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -5,12 +5,12 @@ import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LI import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexContext; +import it.cavallium.dbengine.database.disk.LLIndexContexts; import it.cavallium.dbengine.lucene.LuceneUtils; -import it.cavallium.dbengine.lucene.searcher.IndexSearchers.UnshardedIndexSearchers; +import it.cavallium.dbengine.database.disk.LLIndexContexts.UnshardedIndexSearchers; import java.io.IOException; import java.util.Arrays; import java.util.Objects; @@ -25,14 +25,14 @@ import reactor.core.scheduler.Schedulers; public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono> collect(Mono> indexSearcherMono, + public Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, String keyFieldName) { Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); PaginationInfo paginationInfo = getPaginationInfo(queryParams); - var indexSearchersMono = indexSearcherMono.map(IndexSearchers::unsharded); + var indexSearchersMono = indexSearcherMono.map(LLIndexContexts::unsharded); return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this // Search first page results @@ -72,32 +72,11 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { .handle((s, sink) -> this.searchPageSync(queryParams, indexSearchers, pagination, resultsOffset, s, sink)); } - /** - * Search effectively the merged raw results of the next pages - */ - private Flux searchOtherPages(UnshardedIndexSearchers indexSearchers, - LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { - return Flux - .generate( - () -> secondPageInfo, - (s, sink) -> searchPageSync(queryParams, indexSearchers, true, 0, s, sink), - s -> {} - ) - .subscribeOn(Schedulers.boundedElastic()) - .map(PageData::topDocs) - .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) - .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, - keyFieldName, true)); - } - - private static record FirstPageResults(TotalHitsCount totalHitsCount, Flux firstPageHitsFlux, - CurrentPageInfo nextPageInfo) {} - /** * Compute the results of the first page, extracting useful data */ private Mono computeFirstPageResults(Mono firstPageDataMono, - IndexSearchers indexSearchers, + LLIndexContexts indexSearchers, String keyFieldName, LocalQueryParams queryParams) { return firstPageDataMono.map(firstPageData -> { @@ -129,7 +108,23 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { }); } - private static record PageData(TopDocs topDocs, CurrentPageInfo nextPageInfo) {} + /** + * Search effectively the merged raw results of the next pages + */ + private Flux searchOtherPages(UnshardedIndexSearchers indexSearchers, + LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { + return Flux + .generate( + () -> secondPageInfo, + (s, sink) -> searchPageSync(queryParams, indexSearchers, true, 0, s, sink), + s -> {} + ) + .subscribeOn(Schedulers.boundedElastic()) + .map(PageData::topDocs) + .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) + .transform(topFieldDocFlux -> LuceneUtils.convertHits(topFieldDocFlux, indexSearchers, + keyFieldName, true)); + } /** * diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index ef99dde..045c1d2 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -6,6 +6,7 @@ import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import java.util.ArrayList; import java.util.Comparator; @@ -22,7 +23,7 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } @Override - public Mono> collect(Flux> indexSearchersFlux, + public Mono> collect(Flux> indexSearchersFlux, LocalQueryParams queryParams, String keyFieldName) { return Mono @@ -38,7 +39,7 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } }) .thenMany(indexSearchersFlux) - .flatMap(resSend -> localSearcher.collect(Mono.just(resSend), queryParams, keyFieldName)) + .flatMap(resSend -> localSearcher.collect(Mono.just(resSend).share(), queryParams, keyFieldName)) .collectList() .map(results -> { List resultsToDrop = new ArrayList<>(results.size()); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java deleted file mode 100644 index 4162b75..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneMultiSearcher.java +++ /dev/null @@ -1,34 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.FIRST_PAGE_LIMIT; -import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SEARCH_LIMIT; - -import it.cavallium.dbengine.lucene.LuceneUtils; -import reactor.core.publisher.Mono; - -public class UnscoredPagedLuceneMultiSearcher implements LuceneMultiSearcher { - - @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { - return Mono - .fromCallable(() -> { - if (queryParams.isScored()) { - throw new UnsupportedOperationException("Can't use the unscored searcher to do a scored or sorted query"); - } - PaginationInfo paginationInfo; - if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); - } else { - paginationInfo = new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); - } - UnscoredTopDocsCollectorManager unsortedCollectorManager = new UnscoredTopDocsCollectorManager(() -> TopDocsSearcher.getTopDocsCollector(queryParams.sort(), - LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()), - null, - LuceneUtils.totalHitsThreshold(), - !paginationInfo.forceSinglePage(), - queryParams.isScored() - ), queryParams.offset(), queryParams.limit(), queryParams.sort()); - return new UnscoredPagedLuceneShardSearcher(unsortedCollectorManager, queryParams.query(), paginationInfo); - }); - } -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java deleted file mode 100644 index 917befd..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredPagedLuceneShardSearcher.java +++ /dev/null @@ -1,151 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.EMPTY_STATUS; - -import it.cavallium.dbengine.database.LLKeyScore; -import it.cavallium.dbengine.lucene.LuceneUtils; -import it.unimi.dsi.fastutil.objects.ObjectArrayList; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import org.apache.lucene.search.CollectorManager; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopDocsCollector; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; -import reactor.core.scheduler.Schedulers; - -class UnscoredPagedLuceneShardSearcher implements LuceneMultiSearcher { - - private final Object lock = new Object(); - private final List indexSearchersArray = new ArrayList<>(); - private final List> indexSearcherReleasersArray = new ArrayList<>(); - private final List> collectors = new ArrayList<>(); - private final CollectorManager, TopDocs> firstPageUnsortedCollectorManager; - private final Query luceneQuery; - private final PaginationInfo paginationInfo; - - public UnscoredPagedLuceneShardSearcher( - CollectorManager, TopDocs> firstPagensortedCollectorManager, - Query luceneQuery, - PaginationInfo paginationInfo) { - this.firstPageUnsortedCollectorManager = firstPagensortedCollectorManager; - this.luceneQuery = luceneQuery; - this.paginationInfo = paginationInfo; - } - - @Override - public Mono searchOn(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, - LocalQueryParams queryParams, - Scheduler scheduler) { - return Mono.fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called searchOn in a nonblocking thread"); - } - TopDocsCollector collector; - synchronized (lock) { - //noinspection BlockingMethodInNonBlockingContext - collector = firstPageUnsortedCollectorManager.newCollector(); - indexSearchersArray.add(indexSearcher); - indexSearcherReleasersArray.add(releaseIndexSearcher); - collectors.add(collector); - } - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(luceneQuery, collector); - return null; - }).subscribeOn(scheduler); - } - - @Override - public Mono collect(LocalQueryParams queryParams, String keyFieldName, Scheduler scheduler) { - return Mono - .fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - TopDocs result; - Mono release; - synchronized (lock) { - //noinspection BlockingMethodInNonBlockingContext - result = firstPageUnsortedCollectorManager.reduce(collectors); - release = Mono.when(indexSearcherReleasersArray); - } - IndexSearchers indexSearchers; - synchronized (lock) { - indexSearchers = IndexSearchers.of(indexSearchersArray); - } - Flux firstPageHits = LuceneUtils - .convertHits(Flux.fromArray(result.scoreDocs), indexSearchers, keyFieldName, scheduler, false); - - Flux nextHits = Flux - .generate( - () -> new CurrentPageInfo(LuceneUtils.getLastScoreDoc(result.scoreDocs), - paginationInfo.totalLimit() - paginationInfo.firstPageLimit(), 1), - (s, sink) -> { - if (s.last() != null && s.remainingLimit() > 0 && s.currentPageLimit() > 0) { - Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); - Query luceneQuery = queryParams.query(); - int perShardCollectorLimit = s.currentPageLimit() / indexSearchersArray.size(); - UnscoredTopDocsCollectorManager currentPageUnsortedCollectorManager - = new UnscoredTopDocsCollectorManager( - () -> TopDocsSearcher.getTopDocsCollector(queryParams.sort(), perShardCollectorLimit, - s.last(), LuceneUtils.totalHitsThreshold(), true, queryParams.isScored()), - 0, s.currentPageLimit(), queryParams.sort()); - - try { - var collectors = new ObjectArrayList>(indexSearchersArray.size()); - for (IndexSearcher indexSearcher : indexSearchersArray) { - //noinspection BlockingMethodInNonBlockingContext - var collector = currentPageUnsortedCollectorManager.newCollector(); - //noinspection BlockingMethodInNonBlockingContext - indexSearcher.search(luceneQuery, collector); - - collectors.add(collector); - } - //noinspection BlockingMethodInNonBlockingContext - TopDocs pageTopDocs = currentPageUnsortedCollectorManager.reduce(collectors); - var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); - - sink.next(pageTopDocs); - return new CurrentPageInfo(pageLastDoc, s.remainingLimit() - s.currentPageLimit(), - s.pageIndex() + 1); - } catch (IOException ex) { - sink.error(ex); - return EMPTY_STATUS; - } - } else { - sink.complete(); - return EMPTY_STATUS; - } - } - ) - .subscribeOn(scheduler) - .flatMapIterable(topDocs -> Arrays.asList(topDocs.scoreDocs)) - .transform(scoreDocsFlux -> LuceneUtils.convertHits(scoreDocsFlux, - indexSearchers, keyFieldName, scheduler, false)) - .transform(flux -> { - if (paginationInfo.forceSinglePage() - || paginationInfo.totalLimit() - paginationInfo.firstPageLimit() <= 0) { - return Flux.empty(); - } else { - return flux; - } - }); - - return new LuceneSearchResult(LuceneUtils.convertTotalHitsCount(result.totalHits), firstPageHits - .concatWith(nextHits), - //.transform(flux -> LuceneUtils.filterTopDoc(flux, queryParams)), - release - ); - }) - .subscribeOn(scheduler); - } - -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredTopDocsCollectorManager.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredTopDocsCollectorManager.java deleted file mode 100644 index f158c73..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredTopDocsCollectorManager.java +++ /dev/null @@ -1,70 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.TIE_BREAKER; -import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.ALLOW_UNSCORED_PAGINATION_MODE; - -import it.cavallium.dbengine.lucene.LuceneUtils; -import java.io.IOException; -import java.util.Collection; -import java.util.function.Supplier; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.CollectorManager; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TopFieldDocs; -import org.jetbrains.annotations.Nullable; -import reactor.core.scheduler.Schedulers; - -public class UnscoredTopDocsCollectorManager implements - CollectorManager, TopDocs> { - - private final Supplier> collectorSupplier; - private final long offset; - private final long limit; - private final Sort sort; - - public UnscoredTopDocsCollectorManager(Supplier> collectorSupplier, - long offset, - long limit, - @Nullable Sort sort) { - this.collectorSupplier = collectorSupplier; - this.offset = offset; - this.limit = limit; - this.sort = sort; - } - - @Override - public TopDocsCollector newCollector() throws IOException { - return collectorSupplier.get(); - } - - @Override - public TopDocs reduce(Collection> collection) throws IOException { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called reduce in a nonblocking thread"); - } - int i = 0; - TopDocs[] topDocsArray; - if (sort != null) { - topDocsArray = new TopFieldDocs[collection.size()]; - } else { - topDocsArray = new TopDocs[collection.size()]; - } - for (TopDocsCollector topDocsCollector : collection) { - var topDocs = topDocsCollector.topDocs(); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - scoreDoc.shardIndex = i; - } - topDocsArray[i] = topDocs; - i++; - } - return LuceneUtils.mergeTopDocs(sort, - LuceneUtils.safeLongToInt(offset), - LuceneUtils.safeLongToInt(limit), - topDocsArray, - TIE_BREAKER - ); - } -} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java deleted file mode 100644 index eacd58a..0000000 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/UnscoredUnsortedContinuousLuceneMultiSearcher.java +++ /dev/null @@ -1,179 +0,0 @@ -package it.cavallium.dbengine.lucene.searcher; - -import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; -import it.cavallium.dbengine.lucene.LuceneUtils; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.LockSupport; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.CollectorManager; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.SimpleCollector; -import reactor.core.publisher.Mono; -import reactor.core.publisher.Sinks; -import reactor.core.publisher.Sinks.EmitResult; -import reactor.core.publisher.Sinks.Many; -import reactor.core.scheduler.Scheduler; -import reactor.core.scheduler.Schedulers; - -public class UnscoredUnsortedContinuousLuceneMultiSearcher implements LuceneMultiSearcher { - - private static final Scheduler UNSCORED_UNSORTED_EXECUTOR = Schedulers.newBoundedElastic(Runtime - .getRuntime() - .availableProcessors(), Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, "UnscoredUnsortedExecutor"); - - @Override - public Mono createShardSearcher(LocalQueryParams queryParams) { - return Mono - .fromCallable(() -> { - AtomicBoolean alreadySubscribed = new AtomicBoolean(false); - Many scoreDocsSink = Sinks.many().unicast().onBackpressureBuffer(); - // 1 is the collect phase - AtomicInteger remainingCollectors = new AtomicInteger(1); - - if (queryParams.isScored()) { - throw new UnsupportedOperationException("Can't use the unscored searcher to do a scored or sorted query"); - } - - var cm = new CollectorManager() { - - class IterableCollector extends SimpleCollector { - - private int shardIndex; - private LeafReaderContext context; - - @Override - public void collect(int i) { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - var scoreDoc = new ScoreDoc(context.docBase + i, 0, shardIndex); - synchronized (scoreDocsSink) { - while (scoreDocsSink.tryEmitNext(scoreDoc) == EmitResult.FAIL_OVERFLOW) { - LockSupport.parkNanos(10); - } - } - } - - @Override - protected void doSetNextReader(LeafReaderContext context) { - this.context = context; - } - - @Override - public ScoreMode scoreMode() { - return ScoreMode.COMPLETE_NO_SCORES; - } - - public void setShardIndex(int shardIndex) { - this.shardIndex = shardIndex; - } - } - - @Override - public IterableCollector newCollector() { - return new IterableCollector(); - } - - @Override - public Void reduce(Collection collection) { - throw new UnsupportedOperationException(); - } - }; - - return new LuceneMultiSearcher() { - private final Object lock = new Object(); - private final List indexSearchersArray = new ArrayList<>(); - private final List> indexSearcherReleasersArray = new ArrayList<>(); - @Override - public Mono searchOn(IndexSearcher indexSearcher, - Mono releaseIndexSearcher, - LocalQueryParams queryParams, - Scheduler scheduler) { - return Mono - .fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called searchOn in a nonblocking thread"); - } - //noinspection BlockingMethodInNonBlockingContext - var collector = cm.newCollector(); - int collectorShardIndex; - synchronized (lock) { - collectorShardIndex = indexSearchersArray.size(); - indexSearchersArray.add(indexSearcher); - indexSearcherReleasersArray.add(releaseIndexSearcher); - } - collector.setShardIndex(collectorShardIndex); - remainingCollectors.incrementAndGet(); - UNSCORED_UNSORTED_EXECUTOR.schedule(() -> { - try { - indexSearcher.search(queryParams.query(), collector); - - synchronized (scoreDocsSink) { - decrementRemainingCollectors(scoreDocsSink, remainingCollectors); - } - } catch (IOException e) { - scoreDocsSink.tryEmitError(e); - } - }); - return null; - }) - .subscribeOn(scheduler); - } - - @Override - public Mono collect(LocalQueryParams queryParams, - String keyFieldName, - Scheduler scheduler) { - return Mono - .fromCallable(() -> { - if (Schedulers.isInNonBlockingThread()) { - throw new UnsupportedOperationException("Called collect in a nonblocking thread"); - } - synchronized (scoreDocsSink) { - decrementRemainingCollectors(scoreDocsSink, remainingCollectors); - } - - if (!alreadySubscribed.compareAndSet(false, true)) { - throw new UnsupportedOperationException("Already subscribed!"); - } - - IndexSearchers indexSearchers; - Mono release; - synchronized (lock) { - indexSearchers = IndexSearchers.of(indexSearchersArray); - release = Mono.when(indexSearcherReleasersArray); - } - - AtomicBoolean resultsAlreadySubscribed = new AtomicBoolean(false); - - var scoreDocsFlux = Mono.fromCallable(() -> { - if (!resultsAlreadySubscribed.compareAndSet(false, true)) { - throw new UnsupportedOperationException("Already subscribed!"); - } - return null; - }).thenMany(scoreDocsSink.asFlux()); - var resultsFlux = LuceneUtils - .convertHits(scoreDocsFlux, indexSearchers, keyFieldName, scheduler, false); - - return new LuceneSearchResult(TotalHitsCount.of(0, false), resultsFlux, release); - }) - .subscribeOn(scheduler); - } - }; - }); - } - - private static void decrementRemainingCollectors(Many scoreDocsSink, AtomicInteger remainingCollectors) { - if (remainingCollectors.decrementAndGet() <= 0) { - scoreDocsSink.tryEmitComplete(); - } - } -} From 5cfb5f49cd1339d5d1dc6ab63acdd7cd4105dde6 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 00:22:22 +0200 Subject: [PATCH 05/23] Add custom MoreLikeThis with sharding support --- .../cavallium/dbengine/database/LLUtils.java | 89 +- .../lucene/mlt/MultiMoreLikeThis.java | 1045 +++++++++++++++++ 2 files changed, 1093 insertions(+), 41 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/lucene/mlt/MultiMoreLikeThis.java diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 200f4e4..1589748 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -10,13 +10,13 @@ import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; import io.net5.util.internal.PlatformDependent; import it.cavallium.dbengine.database.collections.DatabaseStage; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLLocalLuceneIndex; import it.cavallium.dbengine.database.disk.MemorySegmentUtils; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializationFunction; import it.cavallium.dbengine.lucene.RandomSortField; +import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import java.nio.ByteBuffer; import java.nio.charset.Charset; @@ -33,6 +33,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.function.ToIntFunction; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FloatPoint; @@ -41,9 +42,10 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; -import org.apache.lucene.queries.mlt.MoreLikeThis; +import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.ConstantScoreQuery; @@ -51,10 +53,11 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.similarities.ClassicSimilarity; +import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -397,9 +400,11 @@ public class LLUtils { } public static Mono getMoreLikeThisQuery( - LLIndexSearcher indexSearcher, + List indexSearchers, @Nullable LLSnapshot snapshot, LocalQueryParams localQueryParams, + Analyzer analyzer, + Similarity similarity, Flux>> mltDocumentFieldsFlux) { Query luceneAdditionalQuery; try { @@ -409,59 +414,61 @@ public class LLUtils { } return mltDocumentFieldsFlux .collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new) - .flatMap(mltDocumentFields -> { + .flatMap(mltDocumentFields -> Mono.fromCallable(() -> { mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty()); if (mltDocumentFields.isEmpty()) { - return Mono.just(new LocalQueryParams(new MatchNoDocsQuery(), + return new LocalQueryParams(new MatchNoDocsQuery(), localQueryParams.offset(), localQueryParams.limit(), localQueryParams.minCompetitiveScore(), localQueryParams.sort(), localQueryParams.scoreMode() - )); + ); } - new IndexSearcher - return indexSearcher.getIndexSearcher().search(snapshot, indexSearcher -> Mono.fromCallable(() -> { - var mlt = new MoreLikeThis(indexSearcher.getIndexReader()); - mlt.setAnalyzer(llLocalLuceneIndex.indexWriter.getAnalyzer()); - mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); - mlt.setMinTermFreq(1); - mlt.setMinDocFreq(3); - mlt.setMaxDocFreqPct(20); - mlt.setBoost(localQueryParams.scoreMode().needsScores()); - mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); - var similarity = llLocalLuceneIndex.getSimilarity(); - if (similarity instanceof TFIDFSimilarity) { - mlt.setSimilarity((TFIDFSimilarity) similarity); - } else { - LLLocalLuceneIndex.logger.trace(MARKER_ROCKSDB, "Using an unsupported similarity algorithm for MoreLikeThis:" - + " {}. You must use a similarity instance based on TFIDFSimilarity!", similarity); + MultiMoreLikeThis mlt; + if (indexSearchers.size() == 1) { + mlt = new MultiMoreLikeThis(indexSearchers.get(0).getIndexReader(), null); + } else { + IndexReader[] indexReaders = new IndexReader[indexSearchers.size()]; + for (int i = 0, size = indexSearchers.size(); i < size; i++) { + indexReaders[i] = indexSearchers.get(i).getIndexReader(); } + mlt = new MultiMoreLikeThis(indexReaders, null); + } + mlt.setAnalyzer(analyzer); + mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); + mlt.setMinTermFreq(1); + mlt.setMinDocFreq(3); + mlt.setMaxDocFreqPct(20); + mlt.setBoost(localQueryParams.scoreMode().needsScores()); + mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); + if (similarity instanceof TFIDFSimilarity tfidfSimilarity) { + mlt.setSimilarity(tfidfSimilarity); + } else { + mlt.setSimilarity(new ClassicSimilarity()); + } - // Get the reference docId and apply it to MoreLikeThis, to generate the query - @SuppressWarnings({"unchecked", "rawtypes"}) - var mltQuery = mlt.like((Map) mltDocumentFields); - Query luceneQuery; - if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { - luceneQuery = new BooleanQuery.Builder() - .add(mltQuery, Occur.MUST) - .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) - .build(); - } else { - luceneQuery = mltQuery; - } + // Get the reference docId and apply it to MoreLikeThis, to generate the query + @SuppressWarnings({"unchecked", "rawtypes"}) + var mltQuery = mlt.like((Map) mltDocumentFields); + Query luceneQuery; + if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { + luceneQuery = new BooleanQuery.Builder() + .add(mltQuery, Occur.MUST) + .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) + .build(); + } else { + luceneQuery = mltQuery; + } - return luceneQuery; - }) - .subscribeOn(Schedulers.boundedElastic()) - .map(luceneQuery -> new LocalQueryParams(luceneQuery, + return new LocalQueryParams(luceneQuery, localQueryParams.offset(), localQueryParams.limit(), localQueryParams.minCompetitiveScore(), localQueryParams.sort(), localQueryParams.scoreMode() - ))); - }); + ); + }).subscribeOn(Schedulers.boundedElastic())); } public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} diff --git a/src/main/java/it/cavallium/dbengine/lucene/mlt/MultiMoreLikeThis.java b/src/main/java/it/cavallium/dbengine/lucene/mlt/MultiMoreLikeThis.java new file mode 100644 index 0000000..6892155 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/mlt/MultiMoreLikeThis.java @@ -0,0 +1,1045 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package it.cavallium.dbengine.lucene.mlt; + +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.Fields; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.BoostQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.similarities.ClassicSimilarity; +import org.apache.lucene.search.similarities.TFIDFSimilarity; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CharsRefBuilder; +import org.apache.lucene.util.PriorityQueue; + +/** + * Generate "more like this" similarity queries. Based on this mail: + * + *

+ * Lucene does let you access the document frequency of terms, with IndexReader.docFreq().
+ * Term frequencies can be computed by re-tokenizing the text, which, for a single document,
+ * is usually fast enough.  But looking up the docFreq() of every term in the document is
+ * probably too slow.
+ *
+ * You can use some heuristics to prune the set of terms, to avoid calling docFreq() too much,
+ * or at all.  Since you're trying to maximize a tf*idf score, you're probably most interested
+ * in terms with a high tf. Choosing a tf threshold even as low as two or three will radically
+ * reduce the number of terms under consideration.  Another heuristic is that terms with a
+ * high idf (i.e., a low df) tend to be longer.  So you could threshold the terms by the
+ * number of characters, not selecting anything less than, e.g., six or seven characters.
+ * With these sorts of heuristics you can usually find small set of, e.g., ten or fewer terms
+ * that do a pretty good job of characterizing a document.
+ *
+ * It all depends on what you're trying to do.  If you're trying to eek out that last percent
+ * of precision and recall regardless of computational difficulty so that you can win a TREC
+ * competition, then the techniques I mention above are useless.  But if you're trying to
+ * provide a "more like this" button on a search results page that does a decent job and has
+ * good performance, such techniques might be useful.
+ *
+ * An efficient, effective "more-like-this" query generator would be a great contribution, if
+ * anyone's interested.  I'd imagine that it would take a Reader or a String (the document's
+ * text), analyzer Analyzer, and return a set of representative terms using heuristics like those
+ * above.  The frequency and length thresholds could be parameters, etc.
+ *
+ * Doug
+ * 
+ * + *

Initial Usage

+ * + *

This class has lots of options to try to make it efficient and flexible. The simplest possible + * usage is as follows. The bold fragment is specific to this class.
+ * + *

+ * IndexReader ir = ...
+ * IndexSearcher is = ...
+ *
+ * MoreLikeThis mlt = new MoreLikeThis(ir);
+ * Reader target = ... // orig source of doc you want to find similarities to
+ * Query query = mlt.like( target);
+ *
+ * Hits hits = is.search(query);
+ * // now the usual iteration thru 'hits' - the only thing to watch for is to make sure
+ * //you ignore the doc if it matches your 'target' document, as it should be similar to itself
+ *
+ * 
+ * + *

Thus you: + * + *

    + *
  1. do your normal, Lucene setup for searching, + *
  2. create a MoreLikeThis, + *
  3. get the text of the doc you want to find similarities to + *
  4. then call one of the like() calls to generate a similarity query + *
  5. call the searcher to find the similar docs + *
+ * + *
+ * + *

More Advanced Usage

+ * + *

You may want to use {@link #setFieldNames setFieldNames(...)} so you can examine multiple + * fields (e.g. body and title) for similarity. + * + *

Depending on the size of your index and the size and makeup of your documents you may want to + * call the other set methods to control how the similarity queries are generated: + * + *

    + *
  • {@link #setMinTermFreq setMinTermFreq(...)} + *
  • {@link #setMinDocFreq setMinDocFreq(...)} + *
  • {@link #setMaxDocFreq setMaxDocFreq(...)} + *
  • {@link #setMaxDocFreqPct setMaxDocFreqPct(...)} + *
  • {@link #setMinWordLen setMinWordLen(...)} + *
  • {@link #setMaxWordLen setMaxWordLen(...)} + *
  • {@link #setMaxQueryTerms setMaxQueryTerms(...)} + *
  • {@link #setMaxNumTokensParsed setMaxNumTokensParsed(...)} + *
  • {@link #setStopWords setStopWord(...)} + *
+ * + *
+ *
+ * + *
+ * Changes: Mark Harwood 29/02/04
+ * Some bugfixing, some refactoring, some optimisation.
+ * - bugfix: retrieveTerms(int docNum) was not working for indexes without a termvector -added missing code
+ * - bugfix: No significant terms being created for fields with a termvector - because
+ * was only counting one occurrence per term/field pair in calculations(ie not including frequency info from TermVector)
+ * - refactor: moved common code into isNoiseWord()
+ * - optimise: when no termvector support available - used maxNumTermsParsed to limit amount of tokenization
+ * 
+ */ +@SuppressWarnings("unused") +public final class MultiMoreLikeThis { + + /** + * Default maximum number of tokens to parse in each example doc field that is not stored with + * TermVector support. + * + * @see #getMaxNumTokensParsed + */ + public static final int DEFAULT_MAX_NUM_TOKENS_PARSED = 5000; + + /** + * Ignore terms with less than this frequency in the source doc. + * + * @see #getMinTermFreq + * @see #setMinTermFreq + */ + public static final int DEFAULT_MIN_TERM_FREQ = 2; + + /** + * Ignore words which do not occur in at least this many docs. + * + * @see #getMinDocFreq + * @see #setMinDocFreq + */ + public static final long DEFAULT_MIN_DOC_FREQ = 5; + + /** + * Ignore words which occur in more than this many docs. + * + * @see #getMaxDocFreq + * @see #setMaxDocFreq + * @see #setMaxDocFreqPct + */ + public static final long DEFAULT_MAX_DOC_FREQ = Long.MAX_VALUE; + + /** + * Boost terms in query based on score. + * + * @see #isBoost + * @see #setBoost + */ + public static final boolean DEFAULT_BOOST = false; + + /** + * Default field names. Null is used to specify that the field names should be looked up at + * runtime from the provided reader. + */ + public static final String[] DEFAULT_FIELD_NAMES = new String[] {"contents"}; + + /** + * Ignore words less than this length or if 0 then this has no effect. + * + * @see #getMinWordLen + * @see #setMinWordLen + */ + public static final int DEFAULT_MIN_WORD_LENGTH = 0; + + /** + * Ignore words greater than this length or if 0 then this has no effect. + * + * @see #getMaxWordLen + * @see #setMaxWordLen + */ + public static final int DEFAULT_MAX_WORD_LENGTH = 0; + + /** + * Default set of stopwords. If null means to allow stop words. + * + * @see #setStopWords + * @see #getStopWords + */ + public static final Set DEFAULT_STOP_WORDS = null; + + /** Current set of stop words. */ + private Set stopWords = DEFAULT_STOP_WORDS; + + /** + * Return a Query with no more than this many terms. + * + * @see IndexSearcher#getMaxClauseCount + * @see #getMaxQueryTerms + * @see #setMaxQueryTerms + */ + public static final int DEFAULT_MAX_QUERY_TERMS = 25; + + /** Analyzer that will be used to parse the doc. */ + private Analyzer analyzer = null; + + /** Ignore words less frequent that this. */ + private int minTermFreq = DEFAULT_MIN_TERM_FREQ; + + /** Ignore words which do not occur in at least this many docs. */ + private long minDocFreq = DEFAULT_MIN_DOC_FREQ; + + /** Ignore words which occur in more than this many docs. */ + private long maxDocFreq = DEFAULT_MAX_DOC_FREQ; + + /** Should we apply a boost to the Query based on the scores? */ + private boolean boost = DEFAULT_BOOST; + + /** Field name we'll analyze. */ + private String[] fieldNames = DEFAULT_FIELD_NAMES; + + /** + * The maximum number of tokens to parse in each example doc field that is not stored with + * TermVector support + */ + private int maxNumTokensParsed = DEFAULT_MAX_NUM_TOKENS_PARSED; + + /** Ignore words if less than this len. */ + private int minWordLen = DEFAULT_MIN_WORD_LENGTH; + + /** Ignore words if greater than this len. */ + private int maxWordLen = DEFAULT_MAX_WORD_LENGTH; + + /** Don't return a query longer than this. */ + private int maxQueryTerms = DEFAULT_MAX_QUERY_TERMS; + + /** For idf() calculations. */ + private TFIDFSimilarity similarity; // = new DefaultSimilarity(); + + /** IndexReader to use */ + private final IndexReader ir; + + /** + * IndexReader array to use when multi-searchers are used. + */ + private final IndexReader[] irArray; + + /** Boost factor to use when boosting the terms */ + private float boostFactor = 1; + + /** + * Returns the boost factor used when boosting terms + * + * @return the boost factor used when boosting terms + * @see #setBoostFactor(float) + */ + public float getBoostFactor() { + return boostFactor; + } + + /** + * Sets the boost factor to use when boosting terms + * + * @see #getBoostFactor() + */ + public void setBoostFactor(float boostFactor) { + this.boostFactor = boostFactor; + } + + /** Constructor requiring an IndexReader. */ + public MultiMoreLikeThis(IndexReader ir) { + this(ir, new ClassicSimilarity()); + } + + public MultiMoreLikeThis(IndexReader ir, TFIDFSimilarity sim) { + this(ir, null, sim); + } + + public MultiMoreLikeThis(IndexReader[] irArray) { + this(irArray, new ClassicSimilarity()); + } + + public MultiMoreLikeThis(IndexReader[] irArray, TFIDFSimilarity sim) { + this(null, irArray, sim); + } + + private MultiMoreLikeThis(IndexReader ir, IndexReader[] irArray, TFIDFSimilarity sim) { + if ((ir == null) == (irArray == null)) { + throw new IllegalArgumentException(); + } + this.irArray = irArray; + this.ir = ir; + this.similarity = sim; + } + + public TFIDFSimilarity getSimilarity() { + return similarity; + } + + public void setSimilarity(TFIDFSimilarity similarity) { + this.similarity = similarity; + } + + /** + * Returns an analyzer that will be used to parse source doc with. The default analyzer is not + * set. + * + * @return the analyzer that will be used to parse source doc with. + */ + public Analyzer getAnalyzer() { + return analyzer; + } + + /** + * Sets the analyzer to use. An analyzer is not required for generating a query with the {@link + * #like(int)} method, all other 'like' methods require an analyzer. + * + * @param analyzer the analyzer to use to tokenize text. + */ + public void setAnalyzer(Analyzer analyzer) { + this.analyzer = analyzer; + } + + /** + * Returns the frequency below which terms will be ignored in the source doc. The default + * frequency is the {@link #DEFAULT_MIN_TERM_FREQ}. + * + * @return the frequency below which terms will be ignored in the source doc. + */ + public int getMinTermFreq() { + return minTermFreq; + } + + /** + * Sets the frequency below which terms will be ignored in the source doc. + * + * @param minTermFreq the frequency below which terms will be ignored in the source doc. + */ + public void setMinTermFreq(int minTermFreq) { + this.minTermFreq = minTermFreq; + } + + /** + * Returns the frequency at which words will be ignored which do not occur in at least this many + * docs. The default frequency is {@link #DEFAULT_MIN_DOC_FREQ}. + * + * @return the frequency at which words will be ignored which do not occur in at least this many + * docs. + */ + public long getMinDocFreq() { + return minDocFreq; + } + + /** + * Sets the frequency at which words will be ignored which do not occur in at least this many + * docs. + * + * @param minDocFreq the frequency at which words will be ignored which do not occur in at least + * this many docs. + */ + public void setMinDocFreq(long minDocFreq) { + this.minDocFreq = minDocFreq; + } + + /** + * Returns the maximum frequency in which words may still appear. Words that appear in more than + * this many docs will be ignored. The default frequency is {@link #DEFAULT_MAX_DOC_FREQ}. + * + * @return get the maximum frequency at which words are still allowed, words which occur in more + * docs than this are ignored. + */ + public long getMaxDocFreq() { + return maxDocFreq; + } + + /** + * Set the maximum frequency in which words may still appear. Words that appear in more than this + * many docs will be ignored. + * + * @param maxFreq the maximum count of documents that a term may appear in to be still considered + * relevant + */ + public void setMaxDocFreq(long maxFreq) { + this.maxDocFreq = maxFreq; + } + + /** + * Set the maximum percentage in which words may still appear. Words that appear in more than this + * many percent of all docs will be ignored. + * + *

This method calls {@link #setMaxDocFreq(long)} internally (both conditions cannot be used at + * the same time). + * + * @param maxPercentage the maximum percentage of documents (0-100) that a term may appear in to + * be still considered relevant. + */ + public void setMaxDocFreqPct(int maxPercentage) { + long maxDoc; + if (irArray == null) { + maxDoc = ir.maxDoc(); + } else { + maxDoc = 0L; + for (IndexReader ir : irArray) { + maxDoc += ir.maxDoc(); + } + } + setMaxDocFreq(Math.toIntExact((long) maxPercentage * maxDoc / 100L)); + } + + /** + * Returns whether to boost terms in query based on "score" or not. The default is {@link + * #DEFAULT_BOOST}. + * + * @return whether to boost terms in query based on "score" or not. + * @see #setBoost + */ + public boolean isBoost() { + return boost; + } + + /** + * Sets whether to boost terms in query based on "score" or not. + * + * @param boost true to boost terms in query based on "score", false otherwise. + * @see #isBoost + */ + public void setBoost(boolean boost) { + this.boost = boost; + } + + /** + * Returns the field names that will be used when generating the 'More Like This' query. The + * default field names that will be used is {@link #DEFAULT_FIELD_NAMES}. + * + * @return the field names that will be used when generating the 'More Like This' query. + */ + public String[] getFieldNames() { + return fieldNames; + } + + /** + * Sets the field names that will be used when generating the 'More Like This' query. Set this to + * null for the field names to be determined at runtime from the IndexReader provided in the + * constructor. + * + * @param fieldNames the field names that will be used when generating the 'More Like This' query. + */ + public void setFieldNames(String[] fieldNames) { + this.fieldNames = fieldNames; + } + + /** + * Returns the minimum word length below which words will be ignored. Set this to 0 for no minimum + * word length. The default is {@link #DEFAULT_MIN_WORD_LENGTH}. + * + * @return the minimum word length below which words will be ignored. + */ + public int getMinWordLen() { + return minWordLen; + } + + /** + * Sets the minimum word length below which words will be ignored. + * + * @param minWordLen the minimum word length below which words will be ignored. + */ + public void setMinWordLen(int minWordLen) { + this.minWordLen = minWordLen; + } + + /** + * Returns the maximum word length above which words will be ignored. Set this to 0 for no maximum + * word length. The default is {@link #DEFAULT_MAX_WORD_LENGTH}. + * + * @return the maximum word length above which words will be ignored. + */ + public int getMaxWordLen() { + return maxWordLen; + } + + /** + * Sets the maximum word length above which words will be ignored. + * + * @param maxWordLen the maximum word length above which words will be ignored. + */ + public void setMaxWordLen(int maxWordLen) { + this.maxWordLen = maxWordLen; + } + + /** + * Set the set of stopwords. Any word in this set is considered "uninteresting" and ignored. Even + * if your Analyzer allows stopwords, you might want to tell the MoreLikeThis code to ignore them, + * as for the purposes of document similarity it seems reasonable to assume that "a stop word is + * never interesting". + * + * @param stopWords set of stopwords, if null it means to allow stop words + * @see #getStopWords + */ + public void setStopWords(Set stopWords) { + this.stopWords = stopWords; + } + + /** + * Get the current stop words being used. + * + * @see #setStopWords + */ + public Set getStopWords() { + return stopWords; + } + + /** + * Returns the maximum number of query terms that will be included in any generated query. The + * default is {@link #DEFAULT_MAX_QUERY_TERMS}. + * + * @return the maximum number of query terms that will be included in any generated query. + */ + public int getMaxQueryTerms() { + return maxQueryTerms; + } + + /** + * Sets the maximum number of query terms that will be included in any generated query. + * + * @param maxQueryTerms the maximum number of query terms that will be included in any generated + * query. + */ + public void setMaxQueryTerms(int maxQueryTerms) { + this.maxQueryTerms = maxQueryTerms; + } + + /** + * @return The maximum number of tokens to parse in each example doc field that is not stored with + * TermVector support + * @see #DEFAULT_MAX_NUM_TOKENS_PARSED + */ + public int getMaxNumTokensParsed() { + return maxNumTokensParsed; + } + + /** + * @param i The maximum number of tokens to parse in each example doc field that is not stored + * with TermVector support + */ + public void setMaxNumTokensParsed(int i) { + maxNumTokensParsed = i; + } + + /** + * Return a query that will return docs like the passed lucene document ID. + * + * @param docNum the documentID of the lucene doc to generate the 'More Like This" query for. + * @return a query that will return docs like the passed lucene document ID. + */ + public Query like(int docNum) throws IOException { + if (fieldNames == null) { + // gather list of valid fields from lucene + Collection fields; + if (irArray == null) { + fields = FieldInfos.getIndexedFields(ir); + } else { + fields = new ArrayList<>(); + for (IndexReader ir : irArray) { + fields.addAll(FieldInfos.getIndexedFields(ir)); + } + } + fieldNames = fields.toArray(String[]::new); + } + + return createQuery(retrieveTerms(docNum)); + } + + /** + * @param filteredDocument Document with field values extracted for selected fields. + * @return More Like This query for the passed document. + */ + public Query like(Map> filteredDocument) throws IOException { + if (fieldNames == null) { + // gather list of valid fields from lucene + Collection fields; + if (irArray == null) { + fields = FieldInfos.getIndexedFields(ir); + } else { + fields = new ArrayList<>(); + for (IndexReader ir : irArray) { + fields.addAll(FieldInfos.getIndexedFields(ir)); + } + } + fieldNames = fields.toArray(String[]::new); + } + return createQuery(retrieveTerms(filteredDocument)); + } + + /** + * Return a query that will return docs like the passed Readers. This was added in order to treat + * multi-value fields. + * + * @return a query that will return docs like the passed Readers. + */ + public Query like(String fieldName, Reader... readers) throws IOException { + Map> perFieldTermFrequencies = new HashMap<>(); + for (Reader r : readers) { + addTermFrequencies(r, perFieldTermFrequencies, fieldName); + } + return createQuery(createQueue(perFieldTermFrequencies)); + } + + /** Create the More like query from a PriorityQueue */ + private Query createQuery(PriorityQueue q) { + BooleanQuery.Builder query = new BooleanQuery.Builder(); + ScoreTerm scoreTerm; + float bestScore = -1; + + while ((scoreTerm = q.pop()) != null) { + Query tq = new TermQuery(new Term(scoreTerm.topField, scoreTerm.word)); + + if (boost) { + if (bestScore == -1) { + bestScore = (scoreTerm.score); + } + float myScore = (scoreTerm.score); + tq = new BoostQuery(tq, boostFactor * myScore / bestScore); + } + + try { + query.add(tq, BooleanClause.Occur.SHOULD); + } catch ( + @SuppressWarnings("unused") + IndexSearcher.TooManyClauses ignore) { + break; + } + } + return query.build(); + } + + /** + * Create a PriorityQueue from a word->tf map. + * + * @param perFieldTermFrequencies a per field map of words keyed on the word(String) with Int + * objects as the values. + */ + private PriorityQueue createQueue( + Map> perFieldTermFrequencies) throws IOException { + // have collected all words in doc and their freqs + final int limit = Math.min(maxQueryTerms, this.getTermsCount(perFieldTermFrequencies)); + FreqQ queue = new FreqQ(limit); // will order words by score + for (Map.Entry> entry : perFieldTermFrequencies.entrySet()) { + Map perWordTermFrequencies = entry.getValue(); + String fieldName = entry.getKey(); + + long numDocs; + if (irArray == null) { + numDocs = ir.getDocCount(fieldName); + if (numDocs == -1) { + numDocs = ir.numDocs(); + } + } else { + numDocs = 0L; + for (IndexReader ir : irArray) { + long localNumDocs = ir.getDocCount(fieldName); + if (localNumDocs == -1) { + localNumDocs = ir.numDocs(); + } + numDocs += localNumDocs; + } + } + + for (Map.Entry tfEntry : perWordTermFrequencies.entrySet()) { // for every word + String word = tfEntry.getKey(); + int tf = tfEntry.getValue().x; // term freq in the source doc + if (minTermFreq > 0 && tf < minTermFreq) { + continue; // filter out words that don't occur enough times in the source + } + + long docFreq; + var fieldTerm = new Term(fieldName, word); + if (irArray == null) { + docFreq = ir.docFreq(fieldTerm); + } else { + docFreq = 0; + for (IndexReader ir : irArray) { + docFreq += ir.docFreq(fieldTerm); + } + } + + if (minDocFreq > 0L && docFreq < minDocFreq) { + continue; // filter out words that don't occur in enough docs + } + + if (docFreq > maxDocFreq) { + continue; // filter out words that occur in too many docs + } + + if (docFreq == 0) { + continue; // index update problem? + } + + float idf = similarity.idf(docFreq, numDocs); + float score = tf * idf; + + if (queue.size() < limit) { + // there is still space in the queue + queue.add(new ScoreTerm(word, fieldName, score)); + } else { + ScoreTerm term = queue.top(); + // update the smallest in the queue in place and update the queue. + if (term.score < score) { + term.update(word, fieldName, score); + queue.updateTop(); + } + } + } + } + return queue; + } + + private int getTermsCount(Map> perFieldTermFrequencies) { + int totalTermsCount = 0; + Collection> values = perFieldTermFrequencies.values(); + for (Map perWordTermFrequencies : values) { + totalTermsCount += perWordTermFrequencies.size(); + } + return totalTermsCount; + } + + /** Describe the parameters that control how the "more like this" query is formed. */ + public String describeParams() { + StringBuilder sb = new StringBuilder(); + sb.append("\t").append("maxQueryTerms : ").append(maxQueryTerms).append("\n"); + sb.append("\t").append("minWordLen : ").append(minWordLen).append("\n"); + sb.append("\t").append("maxWordLen : ").append(maxWordLen).append("\n"); + sb.append("\t").append("fieldNames : "); + String delim = ""; + for (String fieldName : fieldNames) { + sb.append(delim).append(fieldName); + delim = ", "; + } + sb.append("\n"); + sb.append("\t").append("boost : ").append(boost).append("\n"); + sb.append("\t").append("minTermFreq : ").append(minTermFreq).append("\n"); + sb.append("\t").append("minDocFreq : ").append(minDocFreq).append("\n"); + return sb.toString(); + } + + /** + * Find words for a more-like-this query former. + * + * @param docNum the id of the lucene document from which to find terms + */ + private PriorityQueue retrieveTerms(int docNum) throws IOException { + Map> field2termFreqMap = new HashMap<>(); + if (irArray == null) { + retrieveTermsOfIndexReader(ir, docNum, field2termFreqMap); + } else { + for (IndexReader ir : irArray) { + retrieveTermsOfIndexReader(ir, docNum, field2termFreqMap); + } + } + + return createQueue(field2termFreqMap); + } + + private void retrieveTermsOfIndexReader(IndexReader ir, int docNum, Map> field2termFreqMap) + throws IOException { + for (String fieldName : fieldNames) { + final Fields vectors = ir.getTermVectors(docNum); + final Terms vector; + if (vectors != null) { + vector = vectors.terms(fieldName); + } else { + vector = null; + } + + // field does not store term vector info + if (vector == null) { + Document d = ir.document(docNum); + IndexableField[] fields = d.getFields(fieldName); + for (IndexableField field : fields) { + final String stringValue = field.stringValue(); + if (stringValue != null) { + addTermFrequencies(new StringReader(stringValue), field2termFreqMap, fieldName); + } + } + } else { + addTermFrequencies(field2termFreqMap, vector, fieldName); + } + } + } + + private PriorityQueue retrieveTerms(Map> field2fieldValues) + throws IOException { + Map> field2termFreqMap = new HashMap<>(); + for (String fieldName : fieldNames) { + Collection fieldValues = field2fieldValues.get(fieldName); + if (fieldValues == null) { + continue; + } + for (Object fieldValue : fieldValues) { + if (fieldValue != null) { + addTermFrequencies( + new StringReader(String.valueOf(fieldValue)), field2termFreqMap, fieldName); + } + } + } + return createQueue(field2termFreqMap); + } + /** + * Adds terms and frequencies found in vector into the Map termFreqMap + * + * @param field2termFreqMap a Map of terms and their frequencies per field + * @param vector List of terms and their frequencies for a doc/field + */ + private void addTermFrequencies( + Map> field2termFreqMap, Terms vector, String fieldName) + throws IOException { + Map termFreqMap = + field2termFreqMap.computeIfAbsent(fieldName, k -> new HashMap<>()); + final TermsEnum termsEnum = vector.iterator(); + final CharsRefBuilder spare = new CharsRefBuilder(); + BytesRef text; + while ((text = termsEnum.next()) != null) { + spare.copyUTF8Bytes(text); + final String term = spare.toString(); + if (isNoiseWord(term)) { + continue; + } + final int freq = (int) termsEnum.totalTermFreq(); + + // increment frequency + Int cnt = termFreqMap.get(term); + if (cnt == null) { + cnt = new Int(); + termFreqMap.put(term, cnt); + cnt.x = freq; + } else { + cnt.x += freq; + } + } + } + + /** + * Adds term frequencies found by tokenizing text from reader into the Map words + * + * @param r a source of text to be tokenized + * @param perFieldTermFrequencies a Map of terms and their frequencies per field + * @param fieldName Used by analyzer for any special per-field analysis + */ + private void addTermFrequencies( + Reader r, Map> perFieldTermFrequencies, String fieldName) + throws IOException { + if (analyzer == null) { + throw new UnsupportedOperationException( + "To use MoreLikeThis without " + "term vectors, you must provide an Analyzer"); + } + Map termFreqMap = + perFieldTermFrequencies.computeIfAbsent(fieldName, k -> new HashMap<>()); + try (TokenStream ts = analyzer.tokenStream(fieldName, r)) { + int tokenCount = 0; + // for every token + CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); + TermFrequencyAttribute tfAtt = ts.addAttribute(TermFrequencyAttribute.class); + ts.reset(); + while (ts.incrementToken()) { + String word = termAtt.toString(); + tokenCount++; + if (tokenCount > maxNumTokensParsed) { + break; + } + if (isNoiseWord(word)) { + continue; + } + + // increment frequency + Int cnt = termFreqMap.get(word); + if (cnt == null) { + termFreqMap.put(word, new Int(tfAtt.getTermFrequency())); + } else { + cnt.x += tfAtt.getTermFrequency(); + } + } + ts.end(); + } + } + + /** + * determines if the passed term is likely to be of interest in "more like" comparisons + * + * @param term The word being considered + * @return true if should be ignored, false if should be used in further analysis + */ + private boolean isNoiseWord(String term) { + int len = term.length(); + if (minWordLen > 0 && len < minWordLen) { + return true; + } + if (maxWordLen > 0 && len > maxWordLen) { + return true; + } + return stopWords != null && stopWords.contains(term); + } + + /** + * Find words for a more-like-this query former. The result is a priority queue of arrays with one + * entry for every word in the document. Each array has 6 elements. The elements are: + * + *
    + *
  1. The word (String) + *
  2. The top field that this word comes from (String) + *
  3. The score for this word (Float) + *
  4. The IDF value (Float) + *
  5. The frequency of this word in the index (Integer) + *
  6. The frequency of this word in the source document (Integer) + *
+ * + * This is a somewhat "advanced" routine, and in general only the 1st entry in the array is of + * interest. This method is exposed so that you can identify the "interesting words" in a + * document. For an easier method to call see {@link #retrieveInterestingTerms + * retrieveInterestingTerms()}. + * + * @param r the reader that has the content of the document + * @param fieldName field passed to the analyzer to use when analyzing the content + * @return the most interesting words in the document ordered by score, with the highest scoring, + * or best entry, first + * @see #retrieveInterestingTerms + */ + private PriorityQueue retrieveTerms(Reader r, String fieldName) throws IOException { + Map> field2termFreqMap = new HashMap<>(); + addTermFrequencies(r, field2termFreqMap, fieldName); + return createQueue(field2termFreqMap); + } + + /** @see #retrieveInterestingTerms(java.io.Reader, String) */ + public String[] retrieveInterestingTerms(int docNum) throws IOException { + ArrayList al = new ArrayList<>(maxQueryTerms); + PriorityQueue pq = retrieveTerms(docNum); + ScoreTerm scoreTerm; + // have to be careful, retrieveTerms returns all words but that's probably not useful to our + // caller... + int lim = maxQueryTerms; + // we just want to return the top words + while (((scoreTerm = pq.pop()) != null) && lim-- > 0) { + al.add(scoreTerm.word); // the 1st entry is the interesting word + } + String[] res = new String[al.size()]; + return al.toArray(res); + } + + /** + * Convenience routine to make it easy to return the most interesting words in a document. More + * advanced users will call {@link #retrieveTerms(Reader, String) retrieveTerms()} directly. + * + * @param r the source document + * @param fieldName field passed to analyzer to use when analyzing the content + * @return the most interesting words in the document + * @see #retrieveTerms(java.io.Reader, String) + * @see #setMaxQueryTerms + */ + public String[] retrieveInterestingTerms(Reader r, String fieldName) throws IOException { + ArrayList al = new ArrayList<>(maxQueryTerms); + PriorityQueue pq = retrieveTerms(r, fieldName); + ScoreTerm scoreTerm; + // have to be careful, retrieveTerms returns all words but that's probably not useful to our + // caller... + int lim = maxQueryTerms; + // we just want to return the top words + while (((scoreTerm = pq.pop()) != null) && lim-- > 0) { + al.add(scoreTerm.word); // the 1st entry is the interesting word + } + String[] res = new String[al.size()]; + return al.toArray(res); + } + + /** PriorityQueue that orders words by score. */ + private static class FreqQ extends PriorityQueue { + FreqQ(int maxSize) { + super(maxSize); + } + + @Override + protected boolean lessThan(ScoreTerm a, ScoreTerm b) { + return a.score < b.score; + } + } + + private static class ScoreTerm { + // only really need 1st 3 entries, other ones are for troubleshooting + String word; + String topField; + float score; + + ScoreTerm(String word, String topField, float score) { + this.word = word; + this.topField = topField; + this.score = score; + } + + void update(String word, String topField, float score) { + this.word = word; + this.topField = topField; + this.score = score; + } + } + + /** Use for frequencies and to avoid renewing Integers. */ + private static class Int { + int x; + + Int() { + this(1); + } + + Int(int initialValue) { + x = initialValue; + } + } +} \ No newline at end of file From ca37d1fb68ebd14033dbc2fb3ee4ad747f159881 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 11:35:01 +0200 Subject: [PATCH 06/23] Rewrite lucene transformers and implement MoreLikeThis sharding --- .../cavallium/dbengine/database/LLUtils.java | 72 ---------------- .../database/disk/LLIndexContext.java | 81 ----------------- ...dexContexts.java => LLIndexSearchers.java} | 46 +++++----- .../database/disk/LLLocalLuceneIndex.java | 11 +-- .../disk/LLLocalMultiLuceneIndex.java | 77 ++++++----------- .../dbengine/lucene/LuceneUtils.java | 86 +++++++++++++++++++ .../lucene/searcher/LLSearchTransformer.java | 11 ++- .../lucene/searcher/LuceneLocalSearcher.java | 10 ++- .../lucene/searcher/LuceneMultiSearcher.java | 25 ++++-- 9 files changed, 168 insertions(+), 251 deletions(-) delete mode 100644 src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java rename src/main/java/it/cavallium/dbengine/database/disk/{LLIndexContexts.java => LLIndexSearchers.java} (74%) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 1589748..a4072f7 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -399,78 +399,6 @@ public class LLUtils { .doOnDiscard(Send.class, Send::close); } - public static Mono getMoreLikeThisQuery( - List indexSearchers, - @Nullable LLSnapshot snapshot, - LocalQueryParams localQueryParams, - Analyzer analyzer, - Similarity similarity, - Flux>> mltDocumentFieldsFlux) { - Query luceneAdditionalQuery; - try { - luceneAdditionalQuery = localQueryParams.query(); - } catch (Exception e) { - return Mono.error(e); - } - return mltDocumentFieldsFlux - .collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new) - .flatMap(mltDocumentFields -> Mono.fromCallable(() -> { - mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty()); - if (mltDocumentFields.isEmpty()) { - return new LocalQueryParams(new MatchNoDocsQuery(), - localQueryParams.offset(), - localQueryParams.limit(), - localQueryParams.minCompetitiveScore(), - localQueryParams.sort(), - localQueryParams.scoreMode() - ); - } - MultiMoreLikeThis mlt; - if (indexSearchers.size() == 1) { - mlt = new MultiMoreLikeThis(indexSearchers.get(0).getIndexReader(), null); - } else { - IndexReader[] indexReaders = new IndexReader[indexSearchers.size()]; - for (int i = 0, size = indexSearchers.size(); i < size; i++) { - indexReaders[i] = indexSearchers.get(i).getIndexReader(); - } - mlt = new MultiMoreLikeThis(indexReaders, null); - } - mlt.setAnalyzer(analyzer); - mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); - mlt.setMinTermFreq(1); - mlt.setMinDocFreq(3); - mlt.setMaxDocFreqPct(20); - mlt.setBoost(localQueryParams.scoreMode().needsScores()); - mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); - if (similarity instanceof TFIDFSimilarity tfidfSimilarity) { - mlt.setSimilarity(tfidfSimilarity); - } else { - mlt.setSimilarity(new ClassicSimilarity()); - } - - // Get the reference docId and apply it to MoreLikeThis, to generate the query - @SuppressWarnings({"unchecked", "rawtypes"}) - var mltQuery = mlt.like((Map) mltDocumentFields); - Query luceneQuery; - if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { - luceneQuery = new BooleanQuery.Builder() - .add(mltQuery, Occur.MUST) - .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) - .build(); - } else { - luceneQuery = mltQuery; - } - - return new LocalQueryParams(luceneQuery, - localQueryParams.offset(), - localQueryParams.limit(), - localQueryParams.minCompetitiveScore(), - localQueryParams.sort(), - localQueryParams.scoreMode() - ); - }).subscribeOn(Schedulers.boundedElastic())); - } - public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java deleted file mode 100644 index 032c5d3..0000000 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContext.java +++ /dev/null @@ -1,81 +0,0 @@ -package it.cavallium.dbengine.database.disk; - -import io.net5.buffer.api.Drop; -import io.net5.buffer.api.Owned; -import io.net5.buffer.api.Send; -import io.net5.buffer.api.internal.ResourceSupport; -import it.cavallium.dbengine.lucene.searcher.LLSearchTransformer; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.search.IndexSearcher; - -public class LLIndexContext extends ResourceSupport { - - private LLIndexSearcher indexSearcher; - private LLSearchTransformer indexQueryTransformer; - - protected LLIndexContext(Send indexSearcher, - LLSearchTransformer indexQueryTransformer, - Drop drop) { - super(new CloseOnDrop(drop)); - this.indexSearcher = indexSearcher.receive(); - this.indexQueryTransformer = indexQueryTransformer; - } - - @Override - protected RuntimeException createResourceClosedException() { - return new IllegalStateException("Closed"); - } - - @Override - protected Owned prepareSend() { - var indexSearcher = this.indexSearcher.send(); - var indexQueryTransformer = this.indexQueryTransformer; - makeInaccessible(); - return drop -> new LLIndexContext(indexSearcher, indexQueryTransformer, drop); - } - - private void makeInaccessible() { - this.indexSearcher = null; - this.indexQueryTransformer = null; - } - - public IndexSearcher getIndexSearcher() { - if (!isOwned()) { - throw new UnsupportedOperationException("Closed"); - } - return indexSearcher.getIndexSearcher(); - } - - public IndexReader getIndexReader() { - if (!isOwned()) { - throw new UnsupportedOperationException("Closed"); - } - return indexSearcher.getIndexReader(); - } - - public LLSearchTransformer getIndexQueryTransformer() { - if (!isOwned()) { - throw new UnsupportedOperationException("Closed"); - } - return indexQueryTransformer; - } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(LLIndexContext obj) { - try { - if (obj.indexSearcher != null) obj.indexSearcher.close(); - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } -} diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java similarity index 74% rename from src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java rename to src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java index 97a795b..0b2d296 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexContexts.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java @@ -20,39 +20,39 @@ import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.Term; -public interface LLIndexContexts extends Resource { +public interface LLIndexSearchers extends Resource { - static LLIndexContexts of(List> indexSearchers) { + static LLIndexSearchers of(List> indexSearchers) { return new ShardedIndexSearchers(indexSearchers, d -> {}); } - static UnshardedIndexSearchers unsharded(Send indexSearcher) { + static UnshardedIndexSearchers unsharded(Send indexSearcher) { return new UnshardedIndexSearchers(indexSearcher, d -> {}); } - Iterable shards(); + Iterable shards(); - LLIndexContext shard(int shardIndex); + LLIndexSearcher shard(int shardIndex); IndexReader allShards(); - class UnshardedIndexSearchers extends ResourceSupport - implements LLIndexContexts { + class UnshardedIndexSearchers extends ResourceSupport + implements LLIndexSearchers { - private LLIndexContext indexSearcher; + private LLIndexSearcher indexSearcher; - public UnshardedIndexSearchers(Send indexSearcher, Drop drop) { + public UnshardedIndexSearchers(Send indexSearcher, Drop drop) { super(new CloseOnDrop(drop)); this.indexSearcher = indexSearcher.receive(); } @Override - public Iterable shards() { + public Iterable shards() { return Collections.singleton(indexSearcher); } @Override - public LLIndexContext shard(int shardIndex) { + public LLIndexSearcher shard(int shardIndex) { if (!isOwned()) { throw attachTrace(new IllegalStateException("UnshardedIndexSearchers must be owned to be used")); } @@ -67,7 +67,7 @@ public interface LLIndexContexts extends Resource { return indexSearcher.getIndexReader(); } - public LLIndexContext shard() { + public LLIndexSearcher shard() { return this.shard(0); } @@ -78,7 +78,7 @@ public interface LLIndexContexts extends Resource { @Override protected Owned prepareSend() { - Send indexSearcher = this.indexSearcher.send(); + Send indexSearcher = this.indexSearcher.send(); this.makeInaccessible(); return drop -> new UnshardedIndexSearchers(indexSearcher, drop); } @@ -107,26 +107,26 @@ public interface LLIndexContexts extends Resource { } } - class ShardedIndexSearchers extends ResourceSupport - implements LLIndexContexts { + class ShardedIndexSearchers extends ResourceSupport + implements LLIndexSearchers { - private List indexSearchers; + private List indexSearchers; - public ShardedIndexSearchers(List> indexSearchers, Drop drop) { + public ShardedIndexSearchers(List> indexSearchers, Drop drop) { super(new CloseOnDrop(drop)); this.indexSearchers = new ArrayList<>(indexSearchers.size()); - for (Send indexSearcher : indexSearchers) { + for (Send indexSearcher : indexSearchers) { this.indexSearchers.add(indexSearcher.receive()); } } @Override - public Iterable shards() { + public Iterable shards() { return Collections.unmodifiableList(indexSearchers); } @Override - public LLIndexContext shard(int shardIndex) { + public LLIndexSearcher shard(int shardIndex) { if (!isOwned()) { throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); } @@ -161,8 +161,8 @@ public interface LLIndexContexts extends Resource { @Override protected Owned prepareSend() { - List> indexSearchers = new ArrayList<>(this.indexSearchers.size()); - for (LLIndexContext indexSearcher : this.indexSearchers) { + List> indexSearchers = new ArrayList<>(this.indexSearchers.size()); + for (LLIndexSearcher indexSearcher : this.indexSearchers) { indexSearchers.add(indexSearcher.send()); } this.makeInaccessible(); @@ -185,7 +185,7 @@ public interface LLIndexContexts extends Resource { public void drop(ShardedIndexSearchers obj) { try { if (obj.indexSearchers != null) { - for (LLIndexContext indexSearcher : obj.indexSearchers) { + for (LLIndexSearcher indexSearcher : obj.indexSearchers) { indexSearcher.close(); } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java index 22281d1..ef1294b 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; import java.util.Set; import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; @@ -322,7 +321,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { QueryParams queryParams, String keyFieldName, Flux>> mltDocumentFieldsFlux) { - return LLUtils + return LuceneUtils .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) .flatMap(modifiedLocalQuery -> searcherManager .retrieveSearcher(snapshot) @@ -339,7 +338,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { QueryParams queryParams, Flux>> mltDocumentFieldsFlux, LuceneMultiSearcher shardSearcher) { - return LLUtils + return LuceneUtils .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) .flatMap(modifiedLocalQuery -> searcherManager .retrieveSearcher(snapshot) @@ -363,13 +362,9 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { .doOnDiscard(Send.class, Send::close); } - public Mono> retrieveContext(@Nullable LLSnapshot snapshot, - @Nullable LLSearchTransformer indexQueryTransformer) { + public Mono> retrieveSearcher(@Nullable LLSnapshot snapshot) { return searcherManager .retrieveSearcher(snapshot) - .map(indexSearcherToReceive -> new LLIndexContext(indexSearcherToReceive, - Objects.requireNonNullElse(indexQueryTransformer, LLSearchTransformer.NO_TRANSFORMATION), - d -> {}).send()) .doOnDiscard(Send.class, Send::close); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java index da10690..bb755e2 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java @@ -28,7 +28,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -37,13 +36,11 @@ import reactor.util.function.Tuple2; public class LLLocalMultiLuceneIndex implements LLLuceneIndex { - // Scheduler used to get callback values of LuceneStreamSearcher without creating deadlocks - protected final Scheduler luceneSearcherScheduler = LuceneUtils.newLuceneSearcherScheduler(true); - private final ConcurrentHashMap registeredSnapshots = new ConcurrentHashMap<>(); private final AtomicLong nextSnapshotNumber = new AtomicLong(1); private final LLLocalLuceneIndex[] luceneIndices; - + private final IndicizerAnalyzers indicizerAnalyzers; + private final IndicizerSimilarities indicizerSimilarities; private final LuceneMultiSearcher multiSearcher = new AdaptiveLuceneMultiSearcher(); @@ -74,6 +71,8 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { ); } this.luceneIndices = luceneIndices; + this.indicizerAnalyzers = indicizerAnalyzers; + this.indicizerSimilarities = indicizerSimilarities; } private LLLocalLuceneIndex getLuceneIndex(LLTerm id) { @@ -89,18 +88,19 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { return luceneIndices[0].getLuceneIndexName(); } - private Flux> getIndexContexts(LLSnapshot snapshot, - Function indexQueryTransformers) { + private Mono> getIndexSearchers(LLSnapshot snapshot) { return Flux .fromArray(luceneIndices) .index() // Resolve the snapshot of each shard .flatMap(tuple -> Mono .fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1())) - .flatMap(luceneSnapshot -> tuple.getT2().retrieveContext( - luceneSnapshot.orElse(null), indexQueryTransformers.apply(tuple.getT2())) + .flatMap(luceneSnapshot -> tuple.getT2().retrieveSearcher( + luceneSnapshot.orElse(null)) ) - ); + ) + .collectList() + .map(searchers -> LLIndexSearchers.of(searchers).send()); } @Override @@ -198,38 +198,18 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { String keyFieldName, Flux>> mltDocumentFields) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - Flux> serchers = this - .getIndexContexts(snapshot, luceneIndex -> LLSearchTransformer.NO_TRANSFORMATION); + var searchers = this.getIndexSearchers(snapshot); + var transformer = new MoreLikeThisTransformer(mltDocumentFields); // Collect all the shards results into a single global result return multiSearcher - .collect(serchers, localQueryParams, keyFieldName) + .collectMulti(searchers, localQueryParams, keyFieldName, transformer) // Transform the result type .map(resultToReceive -> { var result = resultToReceive.receive(); return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); }); - - return multiSearcher - // Create shard searcher - .createShardSearcher(localQueryParams) - .flatMap(shardSearcher -> Flux - // Iterate the indexed shards - .fromArray(luceneIndices).index() - // Resolve the snapshot of each shard - .flatMap(tuple -> Mono - .fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1())) - .map(luceneSnapshot -> new LuceneIndexWithSnapshot(tuple.getT2(), luceneSnapshot)) - ) - // Execute the query and collect it using the shard searcher - .flatMap(luceneIndexWithSnapshot -> luceneIndexWithSnapshot.luceneIndex() - .distributedMoreLikeThis(luceneIndexWithSnapshot.snapshot.orElse(null), queryParams, mltDocumentFields, shardSearcher)) - // Collect all the shards results into a single global result - .then(shardSearcher.collect(localQueryParams, keyFieldName, luceneSearcherScheduler)) - ) - // Fix the result type - .map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result.release())); } @Override @@ -237,11 +217,11 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { QueryParams queryParams, String keyFieldName) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - Flux> serchers = getIndexContexts(snapshot); + var searchers = getIndexSearchers(snapshot); // Collect all the shards results into a single global result return multiSearcher - .collect(serchers, localQueryParams, keyFieldName) + .collectMulti(searchers, localQueryParams, keyFieldName, LLSearchTransformer.NO_TRANSFORMATION) // Transform the result type .map(resultToReceive -> { var result = resultToReceive.receive(); @@ -310,29 +290,22 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { private class MoreLikeThisTransformer implements LLSearchTransformer { - private final LLLocalLuceneIndex luceneIndex; - private final LLSnapshot snapshot; - private final String keyFieldName; private final Flux>> mltDocumentFields; - public MoreLikeThisTransformer(LLLocalLuceneIndex luceneIndex, - @Nullable LLSnapshot snapshot, - String keyFieldName, - Flux>> mltDocumentFields) { - this.luceneIndex = luceneIndex; - this.snapshot = snapshot; - this.keyFieldName = keyFieldName; + public MoreLikeThisTransformer(Flux>> mltDocumentFields) { this.mltDocumentFields = mltDocumentFields; } @Override - public Mono transform(Mono queryParamsMono) { - return queryParamsMono - .flatMap(queryParams -> { - luceneIndex.getMoreLikeThisTransformer(snapshot, queryParams, mltDocumentFields, ); - }); - LLLocalMultiLuceneIndex.this. - return null; + public Mono transform(Mono inputMono) { + return inputMono.flatMap(input -> { + var defaultAnalyzer = LLLocalMultiLuceneIndex.this.indicizerAnalyzers.defaultAnalyzer(); + var defaultSimilarity = LLLocalMultiLuceneIndex.this.indicizerSimilarities.defaultSimilarity(); + var luceneAnalyzer = LuceneUtils.getAnalyzer(defaultAnalyzer); + var luceneSimilarity = LuceneUtils.getSimilarity(defaultSimilarity); + return LuceneUtils.getMoreLikeThisQuery(input.indexSearchers(), input.queryParams(), + luceneAnalyzer, luceneSimilarity, mltDocumentFields); + }); } } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index 773a166..d18f7ff 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -6,16 +6,20 @@ import it.cavallium.dbengine.client.IndicizerSimilarities; import it.cavallium.dbengine.client.query.QueryParser; import it.cavallium.dbengine.client.query.current.data.QueryParams; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; +import it.cavallium.dbengine.database.EnglishItalianStopFilter; import it.cavallium.dbengine.database.LLKeyScore; +import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.collections.DatabaseMapDictionary; import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep; import it.cavallium.dbengine.database.collections.ValueGetter; import it.cavallium.dbengine.database.disk.LLIndexContexts; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer; import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity; import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer; +import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import it.cavallium.dbengine.lucene.similarity.NGramSimilarity; import java.io.EOFException; @@ -24,9 +28,11 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.Comparator; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.NoSuchElementException; +import java.util.Set; import java.util.stream.Collectors; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.LowerCaseFilter; @@ -39,7 +45,13 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopDocs; @@ -49,6 +61,7 @@ import org.apache.lucene.search.similarities.BooleanSimilarity; import org.apache.lucene.search.similarities.ClassicSimilarity; import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper; import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.novasearch.lucene.search.similarities.BM25Similarity; @@ -59,9 +72,11 @@ import org.novasearch.lucene.search.similarities.RobertsonSimilarity; import org.warp.commonutils.log.Logger; import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; import reactor.util.concurrent.Queues; +import reactor.util.function.Tuple2; public class LuceneUtils { @@ -505,4 +520,75 @@ public class LuceneUtils { true ); } + + public static Mono getMoreLikeThisQuery( + List indexSearchers, + LocalQueryParams localQueryParams, + Analyzer analyzer, + Similarity similarity, + Flux>> mltDocumentFieldsFlux) { + Query luceneAdditionalQuery; + try { + luceneAdditionalQuery = localQueryParams.query(); + } catch (Exception e) { + return Mono.error(e); + } + return mltDocumentFieldsFlux + .collectMap(Tuple2::getT1, Tuple2::getT2, HashMap::new) + .flatMap(mltDocumentFields -> Mono.fromCallable(() -> { + mltDocumentFields.entrySet().removeIf(entry -> entry.getValue().isEmpty()); + if (mltDocumentFields.isEmpty()) { + return new LocalQueryParams(new MatchNoDocsQuery(), + localQueryParams.offset(), + localQueryParams.limit(), + localQueryParams.minCompetitiveScore(), + localQueryParams.sort(), + localQueryParams.scoreMode() + ); + } + MultiMoreLikeThis mlt; + if (indexSearchers.size() == 1) { + mlt = new MultiMoreLikeThis(indexSearchers.get(0).getIndexReader(), null); + } else { + IndexReader[] indexReaders = new IndexReader[indexSearchers.size()]; + for (int i = 0, size = indexSearchers.size(); i < size; i++) { + indexReaders[i] = indexSearchers.get(i).getIndexReader(); + } + mlt = new MultiMoreLikeThis(indexReaders, null); + } + mlt.setAnalyzer(analyzer); + mlt.setFieldNames(mltDocumentFields.keySet().toArray(String[]::new)); + mlt.setMinTermFreq(1); + mlt.setMinDocFreq(3); + mlt.setMaxDocFreqPct(20); + mlt.setBoost(localQueryParams.scoreMode().needsScores()); + mlt.setStopWords(EnglishItalianStopFilter.getStopWordsString()); + if (similarity instanceof TFIDFSimilarity tfidfSimilarity) { + mlt.setSimilarity(tfidfSimilarity); + } else { + mlt.setSimilarity(new ClassicSimilarity()); + } + + // Get the reference docId and apply it to MoreLikeThis, to generate the query + @SuppressWarnings({"unchecked", "rawtypes"}) + var mltQuery = mlt.like((Map) mltDocumentFields); + Query luceneQuery; + if (!(luceneAdditionalQuery instanceof MatchAllDocsQuery)) { + luceneQuery = new BooleanQuery.Builder() + .add(mltQuery, Occur.MUST) + .add(new ConstantScoreQuery(luceneAdditionalQuery), Occur.MUST) + .build(); + } else { + luceneQuery = mltQuery; + } + + return new LocalQueryParams(luceneQuery, + localQueryParams.offset(), + localQueryParams.limit(), + localQueryParams.minCompetitiveScore(), + localQueryParams.sort(), + localQueryParams.scoreMode() + ); + }).subscribeOn(Schedulers.boundedElastic())); + } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java index 0383297..b098a90 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LLSearchTransformer.java @@ -1,10 +1,17 @@ package it.cavallium.dbengine.lucene.searcher; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import java.util.List; +import org.apache.lucene.index.IndexReader; import reactor.core.publisher.Mono; public interface LLSearchTransformer { - LLSearchTransformer NO_TRANSFORMATION = queryParamsMono -> queryParamsMono; + LLSearchTransformer NO_TRANSFORMATION = queryParamsMono -> queryParamsMono + .map(TransformerInput::queryParams); - Mono transform(Mono queryParamsMono); + record TransformerInput(List indexSearchers, + LocalQueryParams queryParams) {} + + Mono transform(Mono inputMono); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java index 767b241..2d67dd8 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java @@ -9,10 +9,12 @@ public interface LuceneLocalSearcher { /** * @param indexSearcherMono Lucene index searcher - * @param queryParams the query parameters - * @param keyFieldName the name of the key field + * @param queryParams the query parameters + * @param keyFieldName the name of the key field + * @param transformer the search query transformer */ - Mono> collect(Mono> indexSearcherMono, + Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName); + String keyFieldName, + LLSearchTransformer transformer); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java index cf75cac..3e45e6d 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java @@ -3,28 +3,35 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; public interface LuceneMultiSearcher extends LuceneLocalSearcher { /** - * @param queryParams the query parameters - * @param keyFieldName the name of the key field + * @param indexSearchersMono Lucene index searcher + * @param queryParams the query parameters + * @param keyFieldName the name of the key field + * @param transformer the search query transformer */ - Mono> collect(Flux> indexSearchersFlux, + Mono> collectMulti(Mono> indexSearchersMono, LocalQueryParams queryParams, - String keyFieldName); + String keyFieldName, + LLSearchTransformer transformer); /** * @param indexSearcherMono Lucene index searcher - * @param queryParams the query parameters - * @param keyFieldName the name of the key field + * @param queryParams the query parameters + * @param keyFieldName the name of the key field + * @param transformer the search query transformer */ @Override - default Mono> collect(Mono> indexSearcherMono, + default Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName) { - return this.collect(indexSearcherMono.flux(), queryParams, keyFieldName); + String keyFieldName, + LLSearchTransformer transformer) { + var searchers = indexSearcherMono.map(a -> LLIndexSearchers.unsharded(a).send()); + return this.collectMulti(searchers, queryParams, keyFieldName, transformer); } } From 4a883ca8ea3e54826c9aa93ae9e62c130281a898 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 11:52:21 +0200 Subject: [PATCH 07/23] Done implementing Local and LocalMulti lucene indices --- .../database/disk/LLLocalLuceneIndex.java | 79 ++++++++++--------- .../disk/LLLocalMultiLuceneIndex.java | 33 ++++---- 2 files changed, 55 insertions(+), 57 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java index ef1294b..df12df5 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalLuceneIndex.java @@ -1,6 +1,7 @@ package it.cavallium.dbengine.database.disk; import static it.cavallium.dbengine.database.LLUtils.MARKER_LUCENE; +import static it.cavallium.dbengine.lucene.searcher.LLSearchTransformer.NO_TRANSFORMATION; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.DirectIOOptions; @@ -29,6 +30,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; +import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper; import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; @@ -69,7 +71,8 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { private final IndexWriter indexWriter; private final SnapshotsManager snapshotsManager; private final IndexSearcherManager searcherManager; - private final Similarity similarity; + private final PerFieldAnalyzerWrapper luceneAnalyzer; + private final Similarity luceneSimilarity; private final Directory directory; private final boolean lowMemory; @@ -157,9 +160,10 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { this.luceneIndexName = name; var snapshotter = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); this.lowMemory = lowMemory; - this.similarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities); + this.luceneAnalyzer = LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers); + this.luceneSimilarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities); - var indexWriterConfig = new IndexWriterConfig(LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers)); + var indexWriterConfig = new IndexWriterConfig(luceneAnalyzer); indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND); indexWriterConfig.setIndexDeletionPolicy(snapshotter); indexWriterConfig.setCommitOnClose(true); @@ -179,15 +183,16 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { writerSchedulerMaxThreadCount = concurrentMergeScheduler.getMaxThreadCount(); mergeScheduler = concurrentMergeScheduler; } + logger.trace("WriterSchedulerMaxThreadCount: {}", writerSchedulerMaxThreadCount); indexWriterConfig.setMergeScheduler(mergeScheduler); indexWriterConfig.setRAMBufferSizeMB(luceneOptions.indexWriterBufferSize() / 1024D / 1024D); indexWriterConfig.setReaderPooling(false); - indexWriterConfig.setSimilarity(getSimilarity()); + indexWriterConfig.setSimilarity(getLuceneSimilarity()); this.indexWriter = new IndexWriter(directory, indexWriterConfig); this.snapshotsManager = new SnapshotsManager(indexWriter, snapshotter); this.searcherManager = new CachedIndexSearcherManager(indexWriter, snapshotsManager, - getSimilarity(), + getLuceneSimilarity(), luceneOptions.applyAllDeletes(), luceneOptions.writeAllDeletes(), luceneOptions.queryRefreshDebounceTime() @@ -199,8 +204,8 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { TimeUnit.MILLISECONDS); } - private Similarity getSimilarity() { - return similarity; + private Similarity getLuceneSimilarity() { + return luceneSimilarity; } @Override @@ -321,45 +326,26 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { QueryParams queryParams, String keyFieldName, Flux>> mltDocumentFieldsFlux) { - return LuceneUtils - .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) - .flatMap(modifiedLocalQuery -> searcherManager - .retrieveSearcher(snapshot) - .transform(indexSearcher -> localSearcher.collect(indexSearcher, modifiedLocalQuery, keyFieldName)) - ) - .map(resultToReceive -> { - var result = resultToReceive.receive(); - return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); - }) - .doOnDiscard(Send.class, Send::close); - } + LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); + var searcher = this.searcherManager.retrieveSearcher(snapshot); + var transformer = new MoreLikeThisTransformer(mltDocumentFieldsFlux); - public Mono getMoreLikeThisTransformer(@Nullable LLSnapshot snapshot, - QueryParams queryParams, - Flux>> mltDocumentFieldsFlux, - LuceneMultiSearcher shardSearcher) { - return LuceneUtils - .getMoreLikeThisQuery(this, snapshot, LuceneUtils.toLocalQueryParams(queryParams), mltDocumentFieldsFlux) - .flatMap(modifiedLocalQuery -> searcherManager - .retrieveSearcher(snapshot) - .flatMap(indexSearcher -> shardSearcher.searchOn(indexSearcher, modifiedLocalQuery)) - ) - .doOnDiscard(Send.class, Send::close); + return localSearcher.collect(searcher, localQueryParams, keyFieldName, transformer).map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); + }).doOnDiscard(Send.class, Send::close); } @Override public Mono> search(@Nullable LLSnapshot snapshot, QueryParams queryParams, String keyFieldName) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); - return searcherManager - .retrieveSearcher(snapshot) - .transform(indexSearcher -> localSearcher.collect(indexSearcher, localQueryParams, - LLSearchTransformer.NO_TRANSFORMATION, keyFieldName)) - .map(resultToReceive -> { - var result = resultToReceive.receive(); - return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); - }) - .doOnDiscard(Send.class, Send::close); + var searcher = searcherManager.retrieveSearcher(snapshot); + + return localSearcher.collect(searcher, localQueryParams, keyFieldName, NO_TRANSFORMATION).map(resultToReceive -> { + var result = resultToReceive.receive(); + return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); + }).doOnDiscard(Send.class, Send::close); } public Mono> retrieveSearcher(@Nullable LLSnapshot snapshot) { @@ -443,4 +429,19 @@ public class LLLocalLuceneIndex implements LLLuceneIndex { public boolean isLowMemoryMode() { return lowMemory; } + + private class MoreLikeThisTransformer implements LLSearchTransformer { + + private final Flux>> mltDocumentFieldsFlux; + + public MoreLikeThisTransformer(Flux>> mltDocumentFieldsFlux) { + this.mltDocumentFieldsFlux = mltDocumentFieldsFlux; + } + + @Override + public Mono transform(Mono inputMono) { + return inputMono.flatMap(input -> LuceneUtils.getMoreLikeThisQuery(input.indexSearchers(), input.queryParams(), + luceneAnalyzer, luceneSimilarity, mltDocumentFieldsFlux)); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java index bb755e2..08ea765 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java @@ -28,10 +28,11 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper; +import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Scheduler; import reactor.util.function.Tuple2; public class LLLocalMultiLuceneIndex implements LLLuceneIndex { @@ -39,8 +40,8 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { private final ConcurrentHashMap registeredSnapshots = new ConcurrentHashMap<>(); private final AtomicLong nextSnapshotNumber = new AtomicLong(1); private final LLLocalLuceneIndex[] luceneIndices; - private final IndicizerAnalyzers indicizerAnalyzers; - private final IndicizerSimilarities indicizerSimilarities; + private final PerFieldAnalyzerWrapper luceneAnalyzer; + private final PerFieldSimilarityWrapper luceneSimilarity; private final LuceneMultiSearcher multiSearcher = new AdaptiveLuceneMultiSearcher(); @@ -71,8 +72,8 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { ); } this.luceneIndices = luceneIndices; - this.indicizerAnalyzers = indicizerAnalyzers; - this.indicizerSimilarities = indicizerSimilarities; + this.luceneAnalyzer = LuceneUtils.toPerFieldAnalyzerWrapper(indicizerAnalyzers); + this.luceneSimilarity = LuceneUtils.toPerFieldSimilarityWrapper(indicizerSimilarities); } private LLLocalLuceneIndex getLuceneIndex(LLTerm id) { @@ -199,7 +200,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { Flux>> mltDocumentFields) { LocalQueryParams localQueryParams = LuceneUtils.toLocalQueryParams(queryParams); var searchers = this.getIndexSearchers(snapshot); - var transformer = new MoreLikeThisTransformer(mltDocumentFields); + var transformer = new MultiMoreLikeThisTransformer(mltDocumentFields); // Collect all the shards results into a single global result return multiSearcher @@ -209,7 +210,8 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { var result = resultToReceive.receive(); return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); - }); + }) + .doOnDiscard(Send.class, Send::close); } @Override @@ -227,7 +229,8 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { var result = resultToReceive.receive(); return new LLSearchResultShard(result.results(), result.totalHitsCount(), d -> result.close()).send(); - }); + }) + .doOnDiscard(Send.class, Send::close); } @Override @@ -288,24 +291,18 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { return luceneIndices[0].isLowMemoryMode(); } - private class MoreLikeThisTransformer implements LLSearchTransformer { + private class MultiMoreLikeThisTransformer implements LLSearchTransformer { private final Flux>> mltDocumentFields; - public MoreLikeThisTransformer(Flux>> mltDocumentFields) { + public MultiMoreLikeThisTransformer(Flux>> mltDocumentFields) { this.mltDocumentFields = mltDocumentFields; } @Override public Mono transform(Mono inputMono) { - return inputMono.flatMap(input -> { - var defaultAnalyzer = LLLocalMultiLuceneIndex.this.indicizerAnalyzers.defaultAnalyzer(); - var defaultSimilarity = LLLocalMultiLuceneIndex.this.indicizerSimilarities.defaultSimilarity(); - var luceneAnalyzer = LuceneUtils.getAnalyzer(defaultAnalyzer); - var luceneSimilarity = LuceneUtils.getSimilarity(defaultSimilarity); - return LuceneUtils.getMoreLikeThisQuery(input.indexSearchers(), input.queryParams(), - luceneAnalyzer, luceneSimilarity, mltDocumentFields); - }); + return inputMono.flatMap(input -> LuceneUtils.getMoreLikeThisQuery(input.indexSearchers(), input.queryParams(), + luceneAnalyzer, luceneSimilarity, mltDocumentFields)); } } } From 3c5edbc06e924f6069bcb3b7caa4e1c55bae2177 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 12:51:27 +0200 Subject: [PATCH 08/23] Finished initial refactoring --- .../cavallium/dbengine/database/LLUtils.java | 30 +++++++-- .../dbengine/lucene/LuceneUtils.java | 6 +- .../searcher/AdaptiveLuceneLocalSearcher.java | 11 ++-- .../searcher/AdaptiveLuceneMultiSearcher.java | 13 ++-- .../searcher/CountLuceneLocalSearcher.java | 6 +- .../lucene/searcher/LuceneLocalSearcher.java | 1 - .../lucene/searcher/LuceneMultiSearcher.java | 1 - .../ScoredSimpleLuceneShardSearcher.java | 23 ++++--- .../searcher/SimpleLuceneLocalSearcher.java | 27 +++++--- ...leUnsortedUnscoredLuceneMultiSearcher.java | 62 ++++++++++--------- 10 files changed, 105 insertions(+), 75 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index a4072f7..5194c1e 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -389,12 +389,30 @@ public class LLUtils { Function> resourceClosure, boolean cleanupOnSuccess) { return Mono.usingWhen(resourceSupplier, resourceClosure, r -> { - if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); - } else { - return Mono.empty(); - } - }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) + .doOnDiscard(Resource.class, Resource::close) + .doOnDiscard(Send.class, Send::close); + } + + /** + * cleanup resource + * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful + */ + public static > Mono usingSendResource(Mono> resourceSupplier, + Function> resourceClosure, + boolean cleanupOnSuccess) { + return Mono.usingWhen(resourceSupplier.map(Send::receive), resourceClosure, r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) .doOnDiscard(Resource.class, Resource::close) .doOnDiscard(Send.class, Send::close); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index d18f7ff..3f19b63 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -12,8 +12,8 @@ import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.collections.DatabaseMapDictionary; import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep; import it.cavallium.dbengine.database.collections.ValueGetter; -import it.cavallium.dbengine.database.disk.LLIndexContexts; import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.lucene.analyzer.NCharGramAnalyzer; import it.cavallium.dbengine.lucene.analyzer.NCharGramEdgeAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer; @@ -371,7 +371,7 @@ public class LuceneUtils { } public static Flux convertHits(Flux hitsFlux, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, String keyFieldName, boolean preserveOrder) { if (preserveOrder) { @@ -396,7 +396,7 @@ public class LuceneUtils { @Nullable private static LLKeyScore mapHitBlocking(ScoreDoc hit, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, String keyFieldName) { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called mapHitBlocking in a nonblocking thread"); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java index 50a92d5..b4560d3 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneLocalSearcher.java @@ -1,8 +1,8 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import reactor.core.publisher.Mono; public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher { @@ -12,13 +12,14 @@ public class AdaptiveLuceneLocalSearcher implements LuceneLocalSearcher { private static final LuceneLocalSearcher countSearcher = new CountLuceneLocalSearcher(); @Override - public Mono> collect(Mono> indexSearcher, + public Mono> collect(Mono> indexSearcher, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + LLSearchTransformer transformer) { if (queryParams.limit() == 0) { - return countSearcher.collect(indexSearcher, queryParams, keyFieldName); + return countSearcher.collect(indexSearcher, queryParams, keyFieldName, transformer); } else { - return localSearcher.collect(indexSearcher, queryParams, keyFieldName); + return localSearcher.collect(indexSearcher, queryParams, keyFieldName, transformer); } } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java index f26170f..8558a66 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/AdaptiveLuceneMultiSearcher.java @@ -1,8 +1,8 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -18,15 +18,16 @@ public class AdaptiveLuceneMultiSearcher implements LuceneMultiSearcher { = new SimpleUnsortedUnscoredLuceneMultiSearcher(new SimpleLuceneLocalSearcher()); @Override - public Mono> collect(Flux> indexSearchersFlux, + public Mono> collectMulti(Mono> indexSearchersMono, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + LLSearchTransformer transformer) { if (queryParams.limit() == 0) { - return countLuceneMultiSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); + return countLuceneMultiSearcher.collectMulti(indexSearchersMono, queryParams, keyFieldName, transformer); } else if (queryParams.isSorted() || queryParams.isScored()) { - return scoredSimpleLuceneShardSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); + return scoredSimpleLuceneShardSearcher.collectMulti(indexSearchersMono, queryParams, keyFieldName, transformer); } else { - return unscoredPagedLuceneMultiSearcher.collect(indexSearchersFlux, queryParams, keyFieldName); + return unscoredPagedLuceneMultiSearcher.collectMulti(indexSearchersMono, queryParams, keyFieldName, transformer); } } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java index 2d4ba85..1e38688 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/CountLuceneLocalSearcher.java @@ -3,7 +3,6 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -12,9 +11,10 @@ import reactor.core.scheduler.Schedulers; public class CountLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono> collect(Mono> indexSearcherMono, + public Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + LLSearchTransformer transformer) { return Mono .usingWhen( indexSearcherMono, diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java index 2d67dd8..dfc5bb5 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneLocalSearcher.java @@ -1,7 +1,6 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import reactor.core.publisher.Mono; diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java index 3e45e6d..642ca4e 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneMultiSearcher.java @@ -1,7 +1,6 @@ package it.cavallium.dbengine.lucene.searcher; import io.net5.buffer.api.Send; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearchers; import reactor.core.publisher.Flux; diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java index 97fe3f9..472aea6 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java @@ -6,8 +6,8 @@ import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SE import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexContext; -import it.cavallium.dbengine.database.disk.LLIndexContexts; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import it.cavallium.dbengine.lucene.LuceneUtils; import java.util.Arrays; import java.util.Objects; @@ -23,15 +23,14 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { } @Override - public Mono> collect(Flux> indexSearchersFlux, + public Mono> collectMulti(Mono> indexSearchersMono, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + LLSearchTransformer transformer) { Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); PaginationInfo paginationInfo = getPaginationInfo(queryParams); - var indexSearchersMono = indexSearchersFlux.collectList().map(LLIndexContexts::of); - - return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this + return LLUtils.usingSendResource(indexSearchersMono, indexSearchers -> this // Search first page results .searchFirstPage(indexSearchers, queryParams, paginationInfo) // Compute the results of the first page @@ -66,7 +65,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { /** * Search effectively the raw results of the first page */ - private Mono searchFirstPage(LLIndexContexts indexSearchers, + private Mono searchFirstPage(LLIndexSearchers indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); @@ -81,7 +80,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { * Compute the results of the first page, extracting useful data */ private Mono computeFirstPageResults(Mono firstPageDataMono, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, String keyFieldName, LocalQueryParams queryParams) { return firstPageDataMono.map(firstPageData -> { @@ -98,7 +97,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { } private Mono> computeOtherResults(Mono firstResultMono, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, LocalQueryParams queryParams, String keyFieldName) { return firstResultMono.map(firstResult -> { @@ -116,7 +115,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { /** * Search effectively the merged raw results of the next pages */ - private Flux searchOtherPages(LLIndexContexts indexSearchers, + private Flux searchOtherPages(LLIndexSearchers indexSearchers, LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { return Flux .defer(() -> { @@ -139,7 +138,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { * skip the first n results in the first page */ private Mono searchPage(LocalQueryParams queryParams, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, boolean allowPagination, int resultsOffset, CurrentPageInfo s) { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index 7c1526a..f2c00d7 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -7,10 +7,10 @@ import static it.cavallium.dbengine.lucene.searcher.PaginationInfo.MAX_SINGLE_SE import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexContext; -import it.cavallium.dbengine.database.disk.LLIndexContexts; +import it.cavallium.dbengine.database.disk.LLIndexSearcher; +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.database.disk.LLIndexContexts.UnshardedIndexSearchers; import java.io.IOException; import java.util.Arrays; import java.util.Objects; @@ -25,14 +25,15 @@ import reactor.core.scheduler.Schedulers; public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { @Override - public Mono> collect(Mono> indexSearcherMono, + public Mono> collect(Mono> indexSearcherMono, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + LLSearchTransformer transformer) { Objects.requireNonNull(queryParams.scoreMode(), "ScoreMode must not be null"); PaginationInfo paginationInfo = getPaginationInfo(queryParams); - var indexSearchersMono = indexSearcherMono.map(LLIndexContexts::unsharded); + var indexSearchersMono = indexSearcherMono.map(LLIndexSearchers::unsharded); return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this // Search first page results @@ -61,7 +62,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { /** * Search effectively the raw results of the first page */ - private Mono searchFirstPage(UnshardedIndexSearchers indexSearchers, + private Mono searchFirstPage(LLIndexSearchers indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); @@ -76,7 +77,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { * Compute the results of the first page, extracting useful data */ private Mono computeFirstPageResults(Mono firstPageDataMono, - LLIndexContexts indexSearchers, + LLIndexSearchers indexSearchers, String keyFieldName, LocalQueryParams queryParams) { return firstPageDataMono.map(firstPageData -> { @@ -132,7 +133,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { * skip the first n results in the first page */ private CurrentPageInfo searchPageSync(LocalQueryParams queryParams, - UnshardedIndexSearchers indexSearchers, + LLIndexSearchers indexSearchers, boolean allowPagination, int resultsOffset, CurrentPageInfo s, @@ -141,13 +142,19 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { if (resultsOffset < 0) { throw new IndexOutOfBoundsException(resultsOffset); } + UnshardedIndexSearchers unshardedIndexSearchers; + if (indexSearchers instanceof UnshardedIndexSearchers unshardedIndexSearchers1) { + unshardedIndexSearchers = unshardedIndexSearchers1; + } else { + throw new IllegalArgumentException(); + } if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { TopDocs pageTopDocs; try { TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), allowPagination, queryParams.isScored()); - indexSearchers.shard().getIndexSearcher().search(queryParams.query(), collector); + unshardedIndexSearchers.shard().getIndexSearcher().search(queryParams.query(), collector); if (resultsOffset > 0) { pageTopDocs = collector.topDocs(resultsOffset, s.currentPageLimit()); } else { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index 045c1d2..7904e11 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -6,8 +6,8 @@ import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; -import it.cavallium.dbengine.database.disk.LLIndexContext; import it.cavallium.dbengine.database.disk.LLIndexSearcher; +import it.cavallium.dbengine.database.disk.LLIndexSearchers; import java.util.ArrayList; import java.util.Comparator; import java.util.List; @@ -23,10 +23,11 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } @Override - public Mono> collect(Flux> indexSearchersFlux, + public Mono> collectMulti(Mono> indexSearchersMono, LocalQueryParams queryParams, - String keyFieldName) { - return Mono + String keyFieldName, + LLSearchTransformer transformer) { + var indexSearchersResource = Mono .fromRunnable(() -> { LLUtils.ensureBlocking(); if (!queryParams.isSorted()) { @@ -38,31 +39,36 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } }) - .thenMany(indexSearchersFlux) - .flatMap(resSend -> localSearcher.collect(Mono.just(resSend).share(), queryParams, keyFieldName)) - .collectList() - .map(results -> { - List resultsToDrop = new ArrayList<>(results.size()); - List> resultsFluxes = new ArrayList<>(results.size()); - boolean exactTotalHitsCount = true; - long totalHitsCountValue = 0; - for (Send resultToReceive : results) { - LuceneSearchResult result = resultToReceive.receive(); - resultsToDrop.add(result); - resultsFluxes.add(result.results()); - exactTotalHitsCount &= result.totalHitsCount().exact(); - totalHitsCountValue += result.totalHitsCount().value(); - } + .then(indexSearchersMono.map(Send::receive)); - var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); - Flux mergedFluxes = Flux.merge(resultsFluxes); + return LLUtils.usingResource(indexSearchersResource, + indexSearchers -> Flux.fromIterable(indexSearchers.shards()) + .flatMap(searcher -> localSearcher + .collect(Mono.just(searcher.send()), queryParams, keyFieldName, transformer)) + .collectList() + .map(results -> { + List resultsToDrop = new ArrayList<>(results.size()); + List> resultsFluxes = new ArrayList<>(results.size()); + boolean exactTotalHitsCount = true; + long totalHitsCountValue = 0; + for (Send resultToReceive : results) { + LuceneSearchResult result = resultToReceive.receive(); + resultsToDrop.add(result); + resultsFluxes.add(result.results()); + exactTotalHitsCount &= result.totalHitsCount().exact(); + totalHitsCountValue += result.totalHitsCount().value(); + } - return new LuceneSearchResult(totalHitsCount, mergedFluxes, d -> { - for (LuceneSearchResult luceneSearchResult : resultsToDrop) { - luceneSearchResult.close(); - } - }).send(); - }) - .doOnDiscard(Send.class, Send::close); + var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); + Flux mergedFluxes = Flux.merge(resultsFluxes); + + return new LuceneSearchResult(totalHitsCount, mergedFluxes, d -> { + for (LuceneSearchResult luceneSearchResult : resultsToDrop) { + luceneSearchResult.close(); + } + }).send(); + }), + true + ); } } From b8adbf452ef8e2d5ab99aed224a105853c948a94 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 16:22:39 +0200 Subject: [PATCH 09/23] Add resources flux --- .../cavallium/dbengine/database/LLUtils.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 5194c1e..83fe3f9 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -417,6 +417,24 @@ public class LLUtils { .doOnDiscard(Send.class, Send::close); } + /** + * cleanup resource + * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful + */ + public static > Flux usingSendResources(Mono> resourceSupplier, + Function> resourceClosure, + boolean cleanupOnSuccess) { + return Flux.usingWhen(resourceSupplier.map(Send::receive), resourceClosure, r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(r::close); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) + .doOnDiscard(Resource.class, Resource::close) + .doOnDiscard(Send.class, Send::close); + } + public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull From bd97704a0bea3abca590f43d990bc29738ac7ecb Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Mon, 20 Sep 2021 18:20:59 +0200 Subject: [PATCH 10/23] Better page limits --- .../database/disk/LLIndexSearchers.java | 2 +- .../dbengine/lucene/LuceneUtils.java | 7 ++++ .../lucene/searcher/CurrentPageInfo.java | 11 ------ .../searcher/ExponentialPageLimits.java | 34 +++++++++++++++++++ .../lucene/searcher/LocalQueryParams.java | 2 +- .../dbengine/lucene/searcher/PageLimits.java | 11 ++++++ .../lucene/searcher/PaginationInfo.java | 2 +- .../ScoredSimpleLuceneShardSearcher.java | 23 +++++++------ .../searcher/SimpleLuceneLocalSearcher.java | 15 ++++---- .../lucene/searcher/SinglePageLimits.java | 23 +++++++++++++ 10 files changed, 99 insertions(+), 31 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/SinglePageLimits.java diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java index 0b2d296..6ac8213 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java @@ -68,7 +68,7 @@ public interface LLIndexSearchers extends Resource { } public LLIndexSearcher shard() { - return this.shard(0); + return this.shard(-1); } @Override diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index 3f19b63..0139dda 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -20,7 +20,9 @@ import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer; import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity; import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer; import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis; +import it.cavallium.dbengine.lucene.searcher.ExponentialPageLimits; import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; +import it.cavallium.dbengine.lucene.searcher.PageLimits; import it.cavallium.dbengine.lucene.similarity.NGramSimilarity; import java.io.EOFException; import java.io.IOException; @@ -113,6 +115,8 @@ public class LuceneUtils { private static final Similarity luceneLDPNoLengthSimilarityInstance = new LdpSimilarity(0, 0.5f); private static final Similarity luceneBooleanSimilarityInstance = new BooleanSimilarity(); private static final Similarity luceneRobertsonSimilarityInstance = new RobertsonSimilarity(); + // TODO: remove this default page limits and make the limits configurable into QueryParams + private static final PageLimits DEFAULT_PAGE_LIMITS = new ExponentialPageLimits(); @SuppressWarnings("DuplicatedCode") public static Analyzer getAnalyzer(TextFieldsAnalyzer analyzer) { @@ -364,6 +368,7 @@ public class LuceneUtils { return new LocalQueryParams(QueryParser.toQuery(queryParams.query()), safeLongToInt(queryParams.offset()), safeLongToInt(queryParams.limit()), + DEFAULT_PAGE_LIMITS, queryParams.minCompetitiveScore().getNullable(), QueryParser.toSort(queryParams.sort()), QueryParser.toScoreMode(queryParams.scoreMode()) @@ -541,6 +546,7 @@ public class LuceneUtils { return new LocalQueryParams(new MatchNoDocsQuery(), localQueryParams.offset(), localQueryParams.limit(), + DEFAULT_PAGE_LIMITS, localQueryParams.minCompetitiveScore(), localQueryParams.sort(), localQueryParams.scoreMode() @@ -585,6 +591,7 @@ public class LuceneUtils { return new LocalQueryParams(luceneQuery, localQueryParams.offset(), localQueryParams.limit(), + DEFAULT_PAGE_LIMITS, localQueryParams.minCompetitiveScore(), localQueryParams.sort(), localQueryParams.scoreMode() diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/CurrentPageInfo.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/CurrentPageInfo.java index ed5614f..6437f65 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/CurrentPageInfo.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/CurrentPageInfo.java @@ -8,17 +8,6 @@ import org.jetbrains.annotations.Nullable; record CurrentPageInfo(@Nullable ScoreDoc last, long remainingLimit, int pageIndex) { - private static final int MAX_ITEMS_PER_PAGE = 500; - public static final Comparator TIE_BREAKER = Comparator.comparingInt((d) -> d.shardIndex); public static final CurrentPageInfo EMPTY_STATUS = new CurrentPageInfo(null, 0, 0); - - int currentPageLimit() { - if (pageIndex >= 10) { // safety - return MAX_ITEMS_PER_PAGE; - } - var min = Math.min(MAX_ITEMS_PER_PAGE, LuceneUtils.safeLongToInt(pageIndex * (0b1L << pageIndex))); - assert min > 0; - return min; - } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java new file mode 100644 index 0000000..7b106d7 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java @@ -0,0 +1,34 @@ +package it.cavallium.dbengine.lucene.searcher; + +import it.cavallium.dbengine.lucene.LuceneUtils; + +public class ExponentialPageLimits implements PageLimits { + + private final int firstPageLimit; + private final int maxItemsPerPage; + + public ExponentialPageLimits() { + this(DEFAULT_MIN_ITEMS_PER_PAGE); + } + + public ExponentialPageLimits(int firstPageLimit) { + this(firstPageLimit, DEFAULT_MAX_ITEMS_PER_PAGE); + } + + public ExponentialPageLimits(int firstPageLimit, int maxItemsPerPage) { + this.firstPageLimit = firstPageLimit; + this.maxItemsPerPage = maxItemsPerPage; + } + + @Override + public int getPageLimit(int pageIndex) { + if (pageIndex >= 10) { // safety + return maxItemsPerPage; + } + var limitedPageIndex = Math.max(1, pageIndex); + var min = Math.max(firstPageLimit, Math.min(maxItemsPerPage, + LuceneUtils.safeLongToInt(limitedPageIndex * (0b1L << limitedPageIndex)))); + assert min > 0; + return min; + } +} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalQueryParams.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalQueryParams.java index f2f34b4..8967054 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalQueryParams.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LocalQueryParams.java @@ -6,7 +6,7 @@ import org.apache.lucene.search.Sort; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -public record LocalQueryParams(@NotNull Query query, int offset, int limit, +public record LocalQueryParams(@NotNull Query query, int offset, int limit, @NotNull PageLimits pageLimits, @Nullable Float minCompetitiveScore, @Nullable Sort sort, @NotNull ScoreMode scoreMode) { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java new file mode 100644 index 0000000..c3cd6ff --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java @@ -0,0 +1,11 @@ +package it.cavallium.dbengine.lucene.searcher; + +import it.cavallium.dbengine.lucene.LuceneUtils; + +public interface PageLimits { + + int DEFAULT_MIN_ITEMS_PER_PAGE = 10; + int DEFAULT_MAX_ITEMS_PER_PAGE = 500; + + int getPageLimit(int pageIndex); +} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/PaginationInfo.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/PaginationInfo.java index 5b75bce..ad207c0 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/PaginationInfo.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/PaginationInfo.java @@ -3,7 +3,7 @@ package it.cavallium.dbengine.lucene.searcher; import java.util.Comparator; import org.apache.lucene.search.ScoreDoc; -public record PaginationInfo(long totalLimit, long firstPageOffset, long firstPageLimit, boolean forceSinglePage) { +public record PaginationInfo(long totalLimit, long firstPageOffset, PageLimits pageLimits, boolean forceSinglePage) { public static final int MAX_SINGLE_SEARCH_LIMIT = 256; public static final int FIRST_PAGE_LIMIT = 10; diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java index 472aea6..c6c6159 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java @@ -56,9 +56,9 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { */ private PaginationInfo getPaginationInfo(LocalQueryParams queryParams) { if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.pageLimits(), true); } else { - return new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.pageLimits(), false); } } @@ -68,12 +68,13 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { private Mono searchFirstPage(LLIndexSearchers indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { - var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); + var limit = paginationInfo.totalLimit(); + var pageLimits = paginationInfo.pageLimits(); var pagination = !paginationInfo.forceSinglePage(); var resultsOffset = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()); return Mono .fromSupplier(() -> new CurrentPageInfo(null, limit, 0)) - .flatMap(s -> this.searchPage(queryParams, indexSearchers, pagination, resultsOffset, s)); + .flatMap(s -> this.searchPage(queryParams, indexSearchers, pagination, pageLimits, resultsOffset, s)); } /** @@ -120,8 +121,9 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { return Flux .defer(() -> { AtomicReference currentPageInfoRef = new AtomicReference<>(secondPageInfo); - return Flux - .defer(() -> searchPage(queryParams, indexSearchers, true, 0, currentPageInfoRef.get())) + return this + .searchPage(queryParams, indexSearchers, true, queryParams.pageLimits(), + 0, currentPageInfoRef.get()) .doOnNext(s -> currentPageInfoRef.set(s.nextPageInfo())) .repeatWhen(s -> s.takeWhile(n -> n > 0)); }) @@ -140,6 +142,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { private Mono searchPage(LocalQueryParams queryParams, LLIndexSearchers indexSearchers, boolean allowPagination, + PageLimits pageLimits, int resultsOffset, CurrentPageInfo s) { return Mono @@ -150,10 +153,10 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { } if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { var sort = getSort(queryParams); - var limit = s.currentPageLimit(); + var pageLimit = pageLimits.getPageLimit(s.pageIndex()); var totalHitsThreshold = LuceneUtils.totalHitsThreshold(); - return new ScoringShardsCollectorManager(sort, limit, null, - totalHitsThreshold, resultsOffset, s.currentPageLimit()); + return new ScoringShardsCollectorManager(sort, pageLimit, null, + totalHitsThreshold, resultsOffset, pageLimit); } else { return null; } @@ -171,7 +174,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); long nextRemainingLimit; if (allowPagination) { - nextRemainingLimit = s.remainingLimit() - s.currentPageLimit(); + nextRemainingLimit = s.remainingLimit() - pageLimits.getPageLimit(s.pageIndex()); } else { nextRemainingLimit = 0L; } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index f2c00d7..683351b 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -53,9 +53,9 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { */ private PaginationInfo getPaginationInfo(LocalQueryParams queryParams) { if (queryParams.limit() <= MAX_SINGLE_SEARCH_LIMIT) { - return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.limit(), true); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.pageLimits(), true); } else { - return new PaginationInfo(queryParams.limit(), queryParams.offset(), FIRST_PAGE_LIMIT, false); + return new PaginationInfo(queryParams.limit(), queryParams.offset(), queryParams.pageLimits(), false); } } @@ -65,7 +65,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { private Mono searchFirstPage(LLIndexSearchers indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { - var limit = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset() + paginationInfo.firstPageLimit()); + var limit = paginationInfo.totalLimit(); var pagination = !paginationInfo.forceSinglePage(); var resultsOffset = LuceneUtils.safeLongToInt(paginationInfo.firstPageOffset()); return Mono @@ -148,15 +148,16 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { } else { throw new IllegalArgumentException(); } + var currentPageLimit = queryParams.pageLimits().getPageLimit(s.pageIndex()); if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { TopDocs pageTopDocs; try { TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), - s.currentPageLimit(), s.last(), LuceneUtils.totalHitsThreshold(), allowPagination, - queryParams.isScored()); + currentPageLimit, s.last(), LuceneUtils.totalHitsThreshold(), + allowPagination, queryParams.isScored()); unshardedIndexSearchers.shard().getIndexSearcher().search(queryParams.query(), collector); if (resultsOffset > 0) { - pageTopDocs = collector.topDocs(resultsOffset, s.currentPageLimit()); + pageTopDocs = collector.topDocs(resultsOffset, currentPageLimit); } else { pageTopDocs = collector.topDocs(); } @@ -167,7 +168,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { var pageLastDoc = LuceneUtils.getLastScoreDoc(pageTopDocs.scoreDocs); long nextRemainingLimit; if (allowPagination) { - nextRemainingLimit = s.remainingLimit() - s.currentPageLimit(); + nextRemainingLimit = s.remainingLimit() - currentPageLimit; } else { nextRemainingLimit = 0L; } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SinglePageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SinglePageLimits.java new file mode 100644 index 0000000..7d019ee --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SinglePageLimits.java @@ -0,0 +1,23 @@ +package it.cavallium.dbengine.lucene.searcher; + +public class SinglePageLimits implements PageLimits { + + private final int firstPageLimit; + + public SinglePageLimits() { + this(DEFAULT_MIN_ITEMS_PER_PAGE); + } + + public SinglePageLimits(int firstPageLimit) { + this.firstPageLimit = firstPageLimit; + } + + @Override + public int getPageLimit(int pageIndex) { + if (pageIndex == 0) { + return firstPageLimit; + } else { + return 0; + } + } +} From 0fee105f0b65f9db2d1f694a133c5b3227be3309 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Tue, 21 Sep 2021 02:01:02 +0200 Subject: [PATCH 11/23] Bugfixes --- .../dbengine/client/SearchResult.java | 8 +++- .../dbengine/client/SearchResultKeys.java | 6 ++- .../database/LLSearchResultShard.java | 6 ++- .../disk/CachedIndexSearcherManager.java | 9 ++++- .../database/disk/LLIndexSearcher.java | 15 +++++-- .../searcher/ExponentialPageLimits.java | 31 +++++++++----- .../lucene/searcher/LinearPageLimits.java | 40 +++++++++++++++++++ .../dbengine/lucene/searcher/PageLimits.java | 2 +- ...leUnsortedUnscoredLuceneMultiSearcher.java | 4 +- 9 files changed, 100 insertions(+), 21 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/lucene/searcher/LinearPageLimits.java diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResult.java b/src/main/java/it/cavallium/dbengine/client/SearchResult.java index 40e6ba7..5026ac4 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResult.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResult.java @@ -18,7 +18,7 @@ public final class SearchResult extends ResourceSupport public SearchResult(Flux> results, TotalHitsCount totalHitsCount, Drop> drop) { - super(new SearchResult.CloseOnDrop<>(drop)); + super(new CloseOnDrop<>(drop)); this.results = results; this.totalHitsCount = totalHitsCount; } @@ -68,7 +68,11 @@ public final class SearchResult extends ResourceSupport @Override public void drop(SearchResult obj) { - delegate.drop(obj); + try { + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } } } } diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java index d784a14..c22cbdf 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java @@ -80,7 +80,11 @@ public final class SearchResultKeys extends ResourceSupport obj) { - delegate.drop(obj); + try { + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } } } diff --git a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java index f3db927..82ac4f7 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java +++ b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java @@ -84,7 +84,11 @@ public final class LLSearchResultShard extends ResourceSupport onInvalidateCache, ResourceSupport::close) .map(searcher -> searcher.copy(this::dropCachedIndexSearcher).send()) diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java index e534357..5ffeca6 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -14,16 +14,19 @@ import org.slf4j.LoggerFactory; public class LLIndexSearcher extends ResourceSupport { private static final Logger logger = LoggerFactory.getLogger(LLIndexSearcher.class); + private final boolean ownsIndexSearcher; private IndexSearcher indexSearcher; private SearcherManager associatedSearcherManager; public LLIndexSearcher(IndexSearcher indexSearcher, @Nullable SearcherManager associatedSearcherManager, + boolean ownsIndexSearcher, Drop drop) { super(new LLIndexSearcher.CloseOnDrop(drop)); this.indexSearcher = indexSearcher; this.associatedSearcherManager = associatedSearcherManager; + this.ownsIndexSearcher = ownsIndexSearcher; } public IndexReader getIndexReader() { @@ -45,10 +48,14 @@ public class LLIndexSearcher extends ResourceSupport new LLIndexSearcher(indexSearcher, associatedSearcherManager, drop); + return drop -> new LLIndexSearcher(indexSearcher, associatedSearcherManager, ownsIndexSearcher, drop); } private void makeInaccessible() { @@ -80,7 +87,7 @@ public class LLIndexSearcher extends ResourceSupport 0) { obj.associatedSearcherManager.release(obj.indexSearcher); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java index 7b106d7..ba0d6ef 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ExponentialPageLimits.java @@ -2,33 +2,46 @@ package it.cavallium.dbengine.lucene.searcher; import it.cavallium.dbengine.lucene.LuceneUtils; +/** + *
y = 2 ^ (x + pageIndexOffset) + firstPageLimit
+ */ public class ExponentialPageLimits implements PageLimits { + private static final int DEFAULT_PAGE_INDEX_OFFSET = 0; + + private final int pageIndexOffset; private final int firstPageLimit; private final int maxItemsPerPage; public ExponentialPageLimits() { - this(DEFAULT_MIN_ITEMS_PER_PAGE); + this(DEFAULT_PAGE_INDEX_OFFSET); } - public ExponentialPageLimits(int firstPageLimit) { - this(firstPageLimit, DEFAULT_MAX_ITEMS_PER_PAGE); + public ExponentialPageLimits(int pageIndexOffset) { + this(pageIndexOffset, DEFAULT_MIN_ITEMS_PER_PAGE); } - public ExponentialPageLimits(int firstPageLimit, int maxItemsPerPage) { + public ExponentialPageLimits(int pageIndexOffset, int firstPageLimit) { + this(pageIndexOffset, firstPageLimit, DEFAULT_MAX_ITEMS_PER_PAGE); + } + + public ExponentialPageLimits(int pageIndexOffset, int firstPageLimit, int maxItemsPerPage) { + this.pageIndexOffset = pageIndexOffset; this.firstPageLimit = firstPageLimit; this.maxItemsPerPage = maxItemsPerPage; } @Override public int getPageLimit(int pageIndex) { - if (pageIndex >= 10) { // safety + var offsetedIndex = pageIndex + pageIndexOffset; + var power = 0b1L << offsetedIndex; + + if (offsetedIndex >= 30) { // safety return maxItemsPerPage; } - var limitedPageIndex = Math.max(1, pageIndex); - var min = Math.max(firstPageLimit, Math.min(maxItemsPerPage, - LuceneUtils.safeLongToInt(limitedPageIndex * (0b1L << limitedPageIndex)))); + + var min = Math.max(firstPageLimit, Math.min(maxItemsPerPage, firstPageLimit + power)); assert min > 0; - return min; + return LuceneUtils.safeLongToInt(min); } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LinearPageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LinearPageLimits.java new file mode 100644 index 0000000..28a2e2d --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LinearPageLimits.java @@ -0,0 +1,40 @@ +package it.cavallium.dbengine.lucene.searcher; + +import it.cavallium.dbengine.lucene.LuceneUtils; + +/** + *
y = (x * factor) + firstPageLimit
+ */ +public class LinearPageLimits implements PageLimits { + + private static final double DEFAULT_FACTOR = 0.5d; + + private final double factor; + private final double firstPageLimit; + private final double maxItemsPerPage; + + public LinearPageLimits() { + this(DEFAULT_FACTOR, DEFAULT_MIN_ITEMS_PER_PAGE); + } + + public LinearPageLimits(double factor) { + this(factor, DEFAULT_MIN_ITEMS_PER_PAGE); + } + + public LinearPageLimits(double factor, int firstPageLimit) { + this(factor, firstPageLimit, DEFAULT_MAX_ITEMS_PER_PAGE); + } + + public LinearPageLimits(double factor, int firstPageLimit, int maxItemsPerPage) { + this.factor = factor; + this.firstPageLimit = firstPageLimit; + this.maxItemsPerPage = maxItemsPerPage; + } + + @Override + public int getPageLimit(int pageIndex) { + double min = Math.min(maxItemsPerPage, firstPageLimit + (pageIndex * factor)); + assert min > 0d; + return (int) min; + } +} diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java index c3cd6ff..2051d86 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/PageLimits.java @@ -5,7 +5,7 @@ import it.cavallium.dbengine.lucene.LuceneUtils; public interface PageLimits { int DEFAULT_MIN_ITEMS_PER_PAGE = 10; - int DEFAULT_MAX_ITEMS_PER_PAGE = 500; + int DEFAULT_MAX_ITEMS_PER_PAGE = 250; int getPageLimit(int pageIndex); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index 7904e11..6638c5a 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -30,11 +30,11 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea var indexSearchersResource = Mono .fromRunnable(() -> { LLUtils.ensureBlocking(); - if (!queryParams.isSorted()) { + if (queryParams.isSorted()) { throw new UnsupportedOperationException("Sorted queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } - if (!queryParams.isScored()) { + if (queryParams.isScored()) { throw new UnsupportedOperationException("Scored queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } From 8b73a05177856c79ccc29991bc0b0623a9d00fd5 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Wed, 22 Sep 2021 11:03:39 +0200 Subject: [PATCH 12/23] Bugfixes and upgraded to java 17 --- pom.xml | 8 +- .../cavallium/dbengine/database/LLUtils.java | 77 ++++++++++++------- .../disk/CachedIndexSearcherManager.java | 70 ++++++----------- .../database/disk/IndexSearcherManager.java | 4 - .../database/disk/LLIndexSearcher.java | 41 +--------- .../database/disk/LLIndexSearchers.java | 53 ++++++++----- .../database/disk/LLLocalDictionary.java | 32 ++++---- .../disk/LLLocalKeyValueDatabase.java | 4 +- .../disk/LLLocalMultiLuceneIndex.java | 4 +- .../database/disk/MemorySegmentUtils.java | 63 +++++++++++---- .../dbengine/lucene/LuceneUtils.java | 10 ++- .../lucene/searcher/LuceneSearchResult.java | 6 +- .../ScoredSimpleLuceneShardSearcher.java | 58 ++++++++------ .../ScoringShardsCollectorManager.java | 17 +++- .../searcher/SimpleLuceneLocalSearcher.java | 36 ++++----- ...leUnsortedUnscoredLuceneMultiSearcher.java | 28 +++++-- .../java/org/rocksdb/CappedWriteBatch.java | 6 +- .../it/cavallium/dbengine/DbTestUtils.java | 2 +- 18 files changed, 290 insertions(+), 229 deletions(-) diff --git a/pom.xml b/pom.xml index 010d651..13d2df6 100644 --- a/pom.xml +++ b/pom.xml @@ -484,7 +484,7 @@ maven-compiler-plugin 3.8.1 - 16 + 17 io.soabase.record-builder @@ -499,8 +499,8 @@ --enable-preview --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED - 16 - 16 + 17 + 17 @@ -533,7 +533,7 @@ - --enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED + --enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED --enable-native-access ci diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 83fe3f9..947de21 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -52,6 +52,7 @@ import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; @@ -339,7 +340,7 @@ public class LLUtils { PlatformDependent.freeDirectBuffer(directBuffer); directBuffer = null; } - directBuffer = LLUtils.obtainDirect(buffer); + directBuffer = LLUtils.obtainDirect(buffer, true); buffer.ensureWritable(size); } } @@ -373,12 +374,12 @@ public class LLUtils { boolean cleanupOnSuccess) { return Mono.usingWhen(resourceSupplier, resourceClosure, r -> { if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); + return Mono.fromRunnable(() -> r.close()); } else { return Mono.empty(); } - }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) - .doOnDiscard(Send.class, Send::close); + }, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close())) + .doOnDiscard(Send.class, send -> send.close()); } /** @@ -390,13 +391,13 @@ public class LLUtils { boolean cleanupOnSuccess) { return Mono.usingWhen(resourceSupplier, resourceClosure, r -> { if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); + return Mono.fromRunnable(() -> r.close()); } else { return Mono.empty(); } - }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) - .doOnDiscard(Resource.class, Resource::close) - .doOnDiscard(Send.class, Send::close); + }, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close())) + .doOnDiscard(Resource.class, resource -> resource.close()) + .doOnDiscard(Send.class, send -> send.close()); } /** @@ -408,13 +409,13 @@ public class LLUtils { boolean cleanupOnSuccess) { return Mono.usingWhen(resourceSupplier.map(Send::receive), resourceClosure, r -> { if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); + return Mono.fromRunnable(() -> r.close()); } else { return Mono.empty(); } - }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) - .doOnDiscard(Resource.class, Resource::close) - .doOnDiscard(Send.class, Send::close); + }, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close())) + .doOnDiscard(Resource.class, resource -> resource.close()) + .doOnDiscard(Send.class, send -> send.close()); } /** @@ -426,13 +427,22 @@ public class LLUtils { boolean cleanupOnSuccess) { return Flux.usingWhen(resourceSupplier.map(Send::receive), resourceClosure, r -> { if (cleanupOnSuccess) { - return Mono.fromRunnable(r::close); + return Mono.fromRunnable(() -> r.close()); } else { return Mono.empty(); } - }, (r, ex) -> Mono.fromRunnable(r::close), r -> Mono.fromRunnable(r::close)) - .doOnDiscard(Resource.class, Resource::close) - .doOnDiscard(Send.class, Send::close); + }, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close())) + .doOnDiscard(Resource.class, resource -> resource.close()) + .doOnDiscard(Send.class, send -> send.close()); + } + + public static boolean isSet(ScoreDoc[] scoreDocs) { + for (ScoreDoc scoreDoc : scoreDocs) { + if (scoreDoc == null) { + return false; + } + } + return true; } public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @@ -440,16 +450,16 @@ public class LLUtils { @NotNull public static DirectBuffer newDirect(BufferAllocator allocator, int size) { try (var buf = allocator.allocate(size)) { - var direct = obtainDirect(buf); + var direct = obtainDirect(buf, true); return new DirectBuffer(buf.send(), direct); } } @NotNull - public static DirectBuffer convertToDirect(BufferAllocator allocator, Send content) { + public static DirectBuffer convertToReadableDirect(BufferAllocator allocator, Send content) { try (var buf = content.receive()) { - if (buf.countComponents() != 0) { - var direct = obtainDirect(buf); + if (buf.countComponents() == 1) { + var direct = obtainDirect(buf, false); return new DirectBuffer(buf.send(), direct); } else { var direct = newDirect(allocator, buf.readableBytes()); @@ -462,7 +472,7 @@ public class LLUtils { } @NotNull - public static ByteBuffer obtainDirect(Buffer buffer) { + public static ByteBuffer obtainDirect(Buffer buffer, boolean writable) { if (!PlatformDependent.hasUnsafe()) { throw new UnsupportedOperationException("Please enable unsafe support or disable netty direct buffers", PlatformDependent.getUnsafeUnavailabilityCause() @@ -470,15 +480,28 @@ public class LLUtils { } if (!MemorySegmentUtils.isSupported()) { throw new UnsupportedOperationException("Foreign Memory Access API support is disabled." - + " Please set \"--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit\""); + + " Please set \"" + MemorySegmentUtils.getSuggestedArgs() + "\"", + MemorySegmentUtils.getUnsupportedCause() + ); } assert buffer.isAccessible(); AtomicLong nativeAddress = new AtomicLong(0); - if (buffer.countComponents() == 1 && buffer.countReadableComponents() == 1) { - buffer.forEachReadable(0, (i, c) -> { - nativeAddress.setPlain(c.readableNativeAddress()); - return false; - }); + if (buffer.countComponents() == 1) { + if (writable) { + if (buffer.countWritableComponents() == 1) { + buffer.forEachWritable(0, (i, c) -> { + nativeAddress.setPlain(c.writableNativeAddress()); + return false; + }); + } + } else { + if (buffer.countReadableComponents() == 1) { + buffer.forEachReadable(0, (i, c) -> { + nativeAddress.setPlain(c.readableNativeAddress()); + return false; + }); + } + } } if (nativeAddress.getPlain() == 0) { if (buffer.capacity() == 0) { diff --git a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java b/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java index 84447c9..343e8c8 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/CachedIndexSearcherManager.java @@ -7,10 +7,12 @@ import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.database.LLSnapshot; import java.io.IOException; +import java.io.UncheckedIOException; import java.time.Duration; import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.search.IndexSearcher; @@ -27,6 +29,7 @@ import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; import reactor.core.publisher.Sinks.Empty; import reactor.core.scheduler.Schedulers; +import reactor.util.function.Tuples; public class CachedIndexSearcherManager implements IndexSearcherManager { @@ -128,35 +131,30 @@ public class CachedIndexSearcherManager implements IndexSearcherManager { } private Mono> generateCachedSearcher(@Nullable LLSnapshot snapshot) { - var onClose = this.closeRequested.asMono(); - var onQueryRefresh = Mono.delay(queryRefreshDebounceTime).then(); - var onInvalidateCache = Mono.firstWithSignal(onClose, onQueryRefresh); + // todo: check if defer is really needed + return Mono.defer(() -> { + var onClose = this.closeRequested.asMono(); + var onQueryRefresh = Mono.delay(queryRefreshDebounceTime).then(); + var onInvalidateCache = Mono.firstWithSignal(onClose, onQueryRefresh).doOnNext(s -> System.err.println("Invalidation triggered")); - return Mono.fromCallable(() -> { - activeSearchers.register(); - IndexSearcher indexSearcher; - SearcherManager associatedSearcherManager; - boolean ownsIndexSearcher; - if (snapshot == null) { - indexSearcher = searcherManager.acquire(); + return Mono.fromCallable(() -> { + activeSearchers.register(); + IndexSearcher indexSearcher; + if (snapshot == null) { + indexSearcher = searcherManager.acquire(); + } else { + indexSearcher = snapshotsManager.resolveSnapshot(snapshot).getIndexSearcher(); + } indexSearcher.setSimilarity(similarity); - associatedSearcherManager = searcherManager; - ownsIndexSearcher = true; - } else { - indexSearcher = snapshotsManager.resolveSnapshot(snapshot).getIndexSearcher(); - associatedSearcherManager = null; - ownsIndexSearcher = false; - } - return new LLIndexSearcher(indexSearcher, - associatedSearcherManager, - ownsIndexSearcher, - this::dropCachedIndexSearcher - ); - }) - .cacheInvalidateWhen(indexSearcher -> onInvalidateCache, ResourceSupport::close) - .map(searcher -> searcher.copy(this::dropCachedIndexSearcher).send()) - .takeUntilOther(onClose) - .doOnDiscard(ResourceSupport.class, ResourceSupport::close); + assert indexSearcher.getIndexReader().getRefCount() > 0; + return indexSearcher; + }) + // todo: re-enable caching if needed + //.cacheInvalidateWhen(tuple -> onInvalidateCache) + .map(indexSearcher -> new LLIndexSearcher(indexSearcher, this::dropCachedIndexSearcher).send()) + .takeUntilOther(onClose) + .doOnDiscard(Send.class, Send::close); + }); } private void dropCachedIndexSearcher(LLIndexSearcher cachedIndexSearcher) { @@ -188,24 +186,6 @@ public class CachedIndexSearcherManager implements IndexSearcherManager { } } - @Override - public Flux searchMany(@Nullable LLSnapshot snapshot, Function> searcherFunction) { - return Flux.usingWhen( - this.retrieveSearcher(snapshot).map(Send::receive), - indexSearcher -> searcherFunction.apply(indexSearcher.getIndexSearcher()), - cachedIndexSearcher -> Mono.fromRunnable(cachedIndexSearcher::close) - ); - } - - @Override - public Mono search(@Nullable LLSnapshot snapshot, Function> searcherFunction) { - return Mono.usingWhen( - this.retrieveSearcher(snapshot).map(Send::receive), - indexSearcher -> searcherFunction.apply(indexSearcher.getIndexSearcher()), - cachedIndexSearcher -> Mono.fromRunnable(cachedIndexSearcher::close) - ); - } - @Override public Mono> retrieveSearcher(@Nullable LLSnapshot snapshot) { if (snapshot == null) { diff --git a/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java b/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java index c86f222..bc3e133 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/IndexSearcherManager.java @@ -15,10 +15,6 @@ public interface IndexSearcherManager { void maybeRefresh() throws IOException; - Flux searchMany(@Nullable LLSnapshot snapshot, Function> searcherFunction); - - Mono search(@Nullable LLSnapshot snapshot, Function> searcherFunction); - Mono> retrieveSearcher(@Nullable LLSnapshot snapshot); Mono close(); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java index 5ffeca6..093eccd 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -13,51 +13,27 @@ import org.slf4j.LoggerFactory; public class LLIndexSearcher extends ResourceSupport { - private static final Logger logger = LoggerFactory.getLogger(LLIndexSearcher.class); - private final boolean ownsIndexSearcher; - private IndexSearcher indexSearcher; - private SearcherManager associatedSearcherManager; - public LLIndexSearcher(IndexSearcher indexSearcher, - @Nullable SearcherManager associatedSearcherManager, - boolean ownsIndexSearcher, - Drop drop) { + public LLIndexSearcher(IndexSearcher indexSearcher, Drop drop) { super(new LLIndexSearcher.CloseOnDrop(drop)); this.indexSearcher = indexSearcher; - this.associatedSearcherManager = associatedSearcherManager; - this.ownsIndexSearcher = ownsIndexSearcher; } public IndexReader getIndexReader() { if (!isOwned()) { - throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); + throw attachTrace(new IllegalStateException("LLIndexSearcher must be owned to be used")); } return indexSearcher.getIndexReader(); } public IndexSearcher getIndexSearcher() { if (!isOwned()) { - throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); + throw attachTrace(new IllegalStateException("LLIndexSearcher must be owned to be used")); } return indexSearcher; } - public LLIndexSearcher copy(Drop drop) { - if (!isOwned()) { - throw attachTrace(new IllegalStateException("CachedIndexSearcher must be owned to be used")); - } - var copyIndexSearcher = this.indexSearcher; - boolean ownsIndexSearcher; - if (this.ownsIndexSearcher && associatedSearcherManager != null) { - copyIndexSearcher.getIndexReader().incRef(); - ownsIndexSearcher = true; - } else { - ownsIndexSearcher = false; - } - return new LLIndexSearcher(copyIndexSearcher, associatedSearcherManager, ownsIndexSearcher, drop); - } - @Override protected RuntimeException createResourceClosedException() { return new IllegalStateException("Closed"); @@ -66,14 +42,12 @@ public class LLIndexSearcher extends ResourceSupport prepareSend() { var indexSearcher = this.indexSearcher; - var associatedSearcherManager = this.associatedSearcherManager; makeInaccessible(); - return drop -> new LLIndexSearcher(indexSearcher, associatedSearcherManager, ownsIndexSearcher, drop); + return drop -> new LLIndexSearcher(indexSearcher, drop); } private void makeInaccessible() { this.indexSearcher = null; - this.associatedSearcherManager = null; } private static class CloseOnDrop implements Drop { @@ -87,14 +61,7 @@ public class LLIndexSearcher extends ResourceSupport 0) { - obj.associatedSearcherManager.release(obj.indexSearcher); - } - } delegate.drop(obj); - } catch (IOException e) { - logger.error("Failed to drop CachedIndexSearcher", e); } finally { obj.makeInaccessible(); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java index 6ac8213..684385b 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java @@ -19,6 +19,7 @@ import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; public interface LLIndexSearchers extends Resource { @@ -30,9 +31,9 @@ public interface LLIndexSearchers extends Resource { return new UnshardedIndexSearchers(indexSearcher, d -> {}); } - Iterable shards(); + List shards(); - LLIndexSearcher shard(int shardIndex); + IndexSearcher shard(int shardIndex); IndexReader allShards(); @@ -47,19 +48,19 @@ public interface LLIndexSearchers extends Resource { } @Override - public Iterable shards() { - return Collections.singleton(indexSearcher); + public List shards() { + return List.of(indexSearcher.getIndexSearcher()); } @Override - public LLIndexSearcher shard(int shardIndex) { + public IndexSearcher shard(int shardIndex) { if (!isOwned()) { throw attachTrace(new IllegalStateException("UnshardedIndexSearchers must be owned to be used")); } if (shardIndex != -1) { throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid, this is a unsharded index"); } - return indexSearcher; + return indexSearcher.getIndexSearcher(); } @Override @@ -67,7 +68,7 @@ public interface LLIndexSearchers extends Resource { return indexSearcher.getIndexReader(); } - public LLIndexSearcher shard() { + public IndexSearcher shard() { return this.shard(-1); } @@ -111,43 +112,53 @@ public interface LLIndexSearchers extends Resource { implements LLIndexSearchers { private List indexSearchers; + private List indexSearchersVals; public ShardedIndexSearchers(List> indexSearchers, Drop drop) { super(new CloseOnDrop(drop)); this.indexSearchers = new ArrayList<>(indexSearchers.size()); - for (Send indexSearcher : indexSearchers) { - this.indexSearchers.add(indexSearcher.receive()); + this.indexSearchersVals = new ArrayList<>(indexSearchers.size()); + for (Send llIndexSearcher : indexSearchers) { + var indexSearcher = llIndexSearcher.receive(); + this.indexSearchers.add(indexSearcher); + this.indexSearchersVals.add(indexSearcher.getIndexSearcher()); } } @Override - public Iterable shards() { - return Collections.unmodifiableList(indexSearchers); + public List shards() { + if (!isOwned()) { + throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); + } + return Collections.unmodifiableList(indexSearchersVals); } @Override - public LLIndexSearcher shard(int shardIndex) { + public IndexSearcher shard(int shardIndex) { if (!isOwned()) { throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); } if (shardIndex < 0) { throw new IndexOutOfBoundsException("Shard index " + shardIndex + " is invalid"); } - return indexSearchers.get(shardIndex); + return indexSearchersVals.get(shardIndex); } @Override public IndexReader allShards() { - var irs = new IndexReader[indexSearchers.size()]; - for (int i = 0, s = indexSearchers.size(); i < s; i++) { - irs[i] = indexSearchers.get(i).getIndexReader(); + if (!isOwned()) { + throw attachTrace(new IllegalStateException("ShardedIndexSearchers must be owned to be used")); + } + var irs = new IndexReader[indexSearchersVals.size()]; + for (int i = 0, s = indexSearchersVals.size(); i < s; i++) { + irs[i] = indexSearchersVals.get(i).getIndexReader(); } Object2IntOpenHashMap indexes = new Object2IntOpenHashMap<>(); for (int i = 0; i < irs.length; i++) { indexes.put(irs[i], i); } try { - return new MultiReader(irs, Comparator.comparingInt(indexes::getInt), true); + return new MultiReader(irs, Comparator.comparingInt(indexes::getInt), false); } catch (IOException ex) { // This shouldn't happen throw new UncheckedIOException(ex); @@ -171,10 +182,12 @@ public interface LLIndexSearchers extends Resource { private void makeInaccessible() { this.indexSearchers = null; + this.indexSearchersVals = null; } private static class CloseOnDrop implements Drop { + private volatile boolean dropped = false; private final Drop delegate; public CloseOnDrop(Drop drop) { @@ -184,11 +197,15 @@ public interface LLIndexSearchers extends Resource { @Override public void drop(ShardedIndexSearchers obj) { try { + assert !dropped; if (obj.indexSearchers != null) { for (LLIndexSearcher indexSearcher : obj.indexSearchers) { - indexSearcher.close(); + if (indexSearcher.isAccessible()) { + indexSearcher.close(); + } } } + dropped = true; delegate.drop(obj); } finally { obj.makeInaccessible(); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java index 65db216..51351cd 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java @@ -300,7 +300,7 @@ public class LLLocalDictionary implements LLDictionary { // Unfortunately it's not feasible until RocksDB implements keyMayExist with buffers // Create the key nio buffer to pass to RocksDB - var keyNioBuffer = LLUtils.convertToDirect(alloc, key.send()); + var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); // Create a direct result buffer because RocksDB works only with direct buffers try (Buffer resultBuf = alloc.allocate(INITIAL_DIRECT_READ_BYTE_BUF_SIZE_BYTES)) { int valueSize; @@ -308,7 +308,7 @@ public class LLLocalDictionary implements LLDictionary { ByteBuffer resultNioBuf; do { // Create the result nio buffer to pass to RocksDB - resultNioBuf = LLUtils.obtainDirect(resultBuf); + resultNioBuf = LLUtils.obtainDirect(resultBuf, true); assert keyNioBuffer.byteBuffer().isDirect(); assert resultNioBuf.isDirect(); valueSize = db.get(cfh, @@ -415,10 +415,10 @@ public class LLLocalDictionary implements LLDictionary { throw new UnsupportedOperationException("Called dbPut in a nonblocking thread"); } if (databaseOptions.allowNettyDirect()) { - var keyNioBuffer = LLUtils.convertToDirect(alloc, key.send()); + var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); try (var ignored1 = keyNioBuffer.buffer().receive()) { assert keyNioBuffer.byteBuffer().isDirect(); - var valueNioBuffer = LLUtils.convertToDirect(alloc, value.send()); + var valueNioBuffer = LLUtils.convertToReadableDirect(alloc, value.send()); try (var ignored2 = valueNioBuffer.buffer().receive()) { assert valueNioBuffer.byteBuffer().isDirect(); db.put(cfh, validWriteOptions, keyNioBuffer.byteBuffer(), valueNioBuffer.byteBuffer()); @@ -479,7 +479,7 @@ public class LLLocalDictionary implements LLDictionary { if (range.hasMin()) { try (var rangeMin = range.getMin().receive()) { if (databaseOptions.allowNettyDirect()) { - var directBuf = LLUtils.convertToDirect(alloc, rangeMin.send()); + var directBuf = LLUtils.convertToReadableDirect(alloc, rangeMin.send()); cloned1 = directBuf.buffer().receive(); direct1 = directBuf.byteBuffer(); readOpts.setIterateLowerBound(slice1 = new DirectSlice(directBuf.byteBuffer())); @@ -491,7 +491,7 @@ public class LLLocalDictionary implements LLDictionary { if (range.hasMax()) { try (var rangeMax = range.getMax().receive()) { if (databaseOptions.allowNettyDirect()) { - var directBuf = LLUtils.convertToDirect(alloc, rangeMax.send()); + var directBuf = LLUtils.convertToReadableDirect(alloc, rangeMax.send()); cloned2 = directBuf.buffer().receive(); direct2 = directBuf.byteBuffer(); readOpts.setIterateUpperBound(slice2 = new DirectSlice(directBuf.byteBuffer())); @@ -504,7 +504,7 @@ public class LLLocalDictionary implements LLDictionary { if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { try (var rangeMin = range.getMin().receive()) { if (databaseOptions.allowNettyDirect()) { - var directBuf = LLUtils.convertToDirect(alloc, rangeMin.send()); + var directBuf = LLUtils.convertToReadableDirect(alloc, rangeMin.send()); cloned3 = directBuf.buffer().receive(); direct3 = directBuf.byteBuffer(); rocksIterator.seek(directBuf.byteBuffer()); @@ -910,7 +910,7 @@ public class LLLocalDictionary implements LLDictionary { } var validWriteOptions = Objects.requireNonNullElse(writeOptions, EMPTY_WRITE_OPTIONS); if (databaseOptions.allowNettyDirect()) { - var keyNioBuffer = LLUtils.convertToDirect(alloc, key.send()); + var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); try { db.delete(cfh, validWriteOptions, keyNioBuffer.byteBuffer()); } finally { @@ -1176,9 +1176,9 @@ public class LLLocalDictionary implements LLDictionary { batch.close(); } else { for (LLEntry entry : entriesWindow) { - var k = LLUtils.convertToDirect(alloc, entry.getKey()); + var k = LLUtils.convertToReadableDirect(alloc, entry.getKey()); try { - var v = LLUtils.convertToDirect(alloc, entry.getValue()); + var v = LLUtils.convertToReadableDirect(alloc, entry.getValue()); try { db.put(cfh, EMPTY_WRITE_OPTIONS, k.byteBuffer(), v.byteBuffer()); } finally { @@ -1309,9 +1309,9 @@ public class LLLocalDictionary implements LLDictionary { } else { int i = 0; for (Tuple2 entry : entriesWindow) { - var k = LLUtils.convertToDirect(alloc, entry.getT1().send()); + var k = LLUtils.convertToReadableDirect(alloc, entry.getT1().send()); try { - var v = LLUtils.convertToDirect(alloc, updatedValuesToWrite.get(i)); + var v = LLUtils.convertToReadableDirect(alloc, updatedValuesToWrite.get(i)); try { db.put(cfh, EMPTY_WRITE_OPTIONS, k.byteBuffer(), v.byteBuffer()); } finally { @@ -1679,9 +1679,9 @@ public class LLLocalDictionary implements LLDictionary { if (!USE_WRITE_BATCHES_IN_SET_RANGE) { for (LLEntry entry : entriesList) { assert entry.isAccessible(); - var k = LLUtils.convertToDirect(alloc, entry.getKey()); + var k = LLUtils.convertToReadableDirect(alloc, entry.getKey()); try { - var v = LLUtils.convertToDirect(alloc, entry.getValue()); + var v = LLUtils.convertToReadableDirect(alloc, entry.getValue()); try { db.put(cfh, EMPTY_WRITE_OPTIONS, k.byteBuffer(), v.byteBuffer()); } finally { @@ -1874,7 +1874,7 @@ public class LLLocalDictionary implements LLDictionary { Send bufferToReceive) { try (var buffer = bufferToReceive.receive()) { if (allowNettyDirect) { - var direct = LLUtils.convertToDirect(alloc, buffer.send()); + var direct = LLUtils.convertToReadableDirect(alloc, buffer.send()); assert direct.byteBuffer().isDirect(); rocksIterator.seek(direct.byteBuffer()); return () -> { @@ -1895,7 +1895,7 @@ public class LLLocalDictionary implements LLDictionary { requireNonNull(buffer); AbstractSlice slice; if (allowNettyDirect && LLLocalDictionary.USE_DIRECT_BUFFER_BOUNDS) { - var direct = LLUtils.convertToDirect(alloc, buffer.send()); + var direct = LLUtils.convertToReadableDirect(alloc, buffer.send()); buffer = direct.buffer().receive(); assert direct.byteBuffer().isDirect(); slice = new DirectSlice(direct.byteBuffer(), buffer.readableBytes()); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyValueDatabase.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyValueDatabase.java index e5e0f92..950ec41 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyValueDatabase.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyValueDatabase.java @@ -100,7 +100,9 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase { } if (!MemorySegmentUtils.isSupported()) { throw new UnsupportedOperationException("Foreign Memory Access API support is disabled." - + " Please set \"--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit\""); + + " Please set \"" + MemorySegmentUtils.getSuggestedArgs() + "\"", + MemorySegmentUtils.getUnsupportedCause() + ); } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java index 08ea765..7b3814a 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalMultiLuceneIndex.java @@ -96,9 +96,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex { // Resolve the snapshot of each shard .flatMap(tuple -> Mono .fromCallable(() -> resolveSnapshotOptional(snapshot, (int) (long) tuple.getT1())) - .flatMap(luceneSnapshot -> tuple.getT2().retrieveSearcher( - luceneSnapshot.orElse(null)) - ) + .flatMap(luceneSnapshot -> tuple.getT2().retrieveSearcher(luceneSnapshot.orElse(null))) ) .collectList() .map(searchers -> LLIndexSearchers.of(searchers).send()); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java b/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java index 94f6236..608404e 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java @@ -17,32 +17,29 @@ public class MemorySegmentUtils { private static final Object NATIVE; static { - Lookup lookup = MethodHandles.publicLookup(); + Lookup lookup = MethodHandles.lookup(); Object nativeVal = null; - MethodHandle ofNativeRestricted; - try { - ofNativeRestricted = lookup.findStatic(Class.forName("jdk.incubator.foreign.MemorySegment"), - "ofNativeRestricted", - MethodType.methodType(Class.forName("jdk.incubator.foreign.MemorySegment")) - ); + var ofNativeRestricted = getJava16NativeRestricted(lookup); + if (ofNativeRestricted == null) { + cause = null; + ofNativeRestricted = getJava17NativeRestricted(lookup); + } + if (ofNativeRestricted != null) { try { nativeVal = ofNativeRestricted.invoke(); } catch (Throwable e) { cause = e; } - } catch (NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) { - ofNativeRestricted = null; - cause = e; } OF_NATIVE_RESTRICTED = ofNativeRestricted; MethodHandle asSlice; try { - asSlice = lookup.findVirtual(Class.forName("jdk.incubator.foreign.MemorySegment"), + asSlice = lookup.findVirtual(lookup.findClass("jdk.incubator.foreign.MemorySegment"), "asSlice", - MethodType.methodType(Class.forName("jdk.incubator.foreign.MemorySegment"), long.class, long.class) + MethodType.methodType(lookup.findClass("jdk.incubator.foreign.MemorySegment"), long.class, long.class) ); } catch (NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) { asSlice = null; @@ -52,7 +49,7 @@ public class MemorySegmentUtils { MethodHandle asByteBuffer; try { - asByteBuffer = lookup.findVirtual(Class.forName("jdk.incubator.foreign.MemorySegment"), + asByteBuffer = lookup.findVirtual(lookup.findClass("jdk.incubator.foreign.MemorySegment"), "asByteBuffer", MethodType.methodType(ByteBuffer.class)); } catch (NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) { asByteBuffer = null; @@ -63,6 +60,36 @@ public class MemorySegmentUtils { NATIVE = nativeVal; } + @SuppressWarnings("JavaLangInvokeHandleSignature") + private static MethodHandle getJava16NativeRestricted(Lookup lookup) { + MethodHandle ofNativeRestricted; + try { + ofNativeRestricted = lookup.findStatic(lookup.findClass("jdk.incubator.foreign.MemorySegment"), + "ofNativeRestricted", + MethodType.methodType(lookup.findClass("jdk.incubator.foreign.MemorySegment")) + ); + } catch (NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) { + ofNativeRestricted = null; + cause = e; + } + return ofNativeRestricted; + } + + @SuppressWarnings("JavaLangInvokeHandleSignature") + private static MethodHandle getJava17NativeRestricted(Lookup lookup) { + MethodHandle ofNativeRestricted; + try { + ofNativeRestricted = lookup.findStatic(lookup.findClass("jdk.incubator.foreign.MemorySegment"), + "globalNativeSegment", + MethodType.methodType(lookup.findClass("jdk.incubator.foreign.MemorySegment")) + ); + } catch (NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) { + ofNativeRestricted = null; + cause = e; + } + return ofNativeRestricted; + } + public static ByteBuffer directBuffer(long address, long size) { if (address <= 0) { throw new IllegalArgumentException("Address is " + address); @@ -76,13 +103,15 @@ public class MemorySegmentUtils { return PlatformDependent.directBuffer(address, (int) size); } throw new UnsupportedOperationException("Foreign Memory Access API is disabled!" - + " Please set \"--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit\""); + + " Please set \"" + MemorySegmentUtils.getSuggestedArgs() + "\"", + getUnsupportedCause() + ); } var memorySegment = AS_SLICE.invoke(NATIVE, address, size); return (ByteBuffer) AS_BYTE_BUFFER.invoke(memorySegment); } catch (Throwable e) { throw new UnsupportedOperationException("Foreign Memory Access API is disabled!" - + " Please set \"--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit\"", e); + + " Please set \"" + MemorySegmentUtils.getSuggestedArgs() + "\"", e); } } @@ -93,4 +122,8 @@ public class MemorySegmentUtils { public static Throwable getUnsupportedCause() { return cause; } + + public static String getSuggestedArgs() { + return "--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --enable-native-access"; + } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index 0139dda..2d4dd88 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -51,6 +51,7 @@ import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; @@ -376,7 +377,7 @@ public class LuceneUtils { } public static Flux convertHits(Flux hitsFlux, - LLIndexSearchers indexSearchers, + List indexSearchers, String keyFieldName, boolean preserveOrder) { if (preserveOrder) { @@ -401,7 +402,7 @@ public class LuceneUtils { @Nullable private static LLKeyScore mapHitBlocking(ScoreDoc hit, - LLIndexSearchers indexSearchers, + List indexSearchers, String keyFieldName) { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called mapHitBlocking in a nonblocking thread"); @@ -409,7 +410,10 @@ public class LuceneUtils { int shardDocId = hit.doc; int shardIndex = hit.shardIndex; float score = hit.score; - var indexSearcher = indexSearchers.shard(shardIndex); + if (shardIndex == -1 && indexSearchers.size() == 1) { + shardIndex = 0; + } + var indexSearcher = indexSearchers.get(shardIndex); try { String collectedDoc = keyOfTopDoc(shardDocId, indexSearcher.getIndexReader(), keyFieldName); return new LLKeyScore(shardDocId, score, collectedDoc); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java index 2746c82..3e2037a 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java @@ -89,7 +89,11 @@ public final class LuceneSearchResult extends ResourceSupport this // Search first page results - .searchFirstPage(indexSearchers, queryParams, paginationInfo) + .searchFirstPage(indexSearchers.shards(), queryParams, paginationInfo) // Compute the results of the first page .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers, keyFieldName, queryParams)) // Compute other results - .transform(firstResult -> this.computeOtherResults(firstResult, indexSearchers, queryParams, keyFieldName)) + .map(firstResult -> this.computeOtherResults(firstResult, indexSearchers.shards(), queryParams, keyFieldName, indexSearchers::close)) // Ensure that one LuceneSearchResult is always returned .single(), false); @@ -65,7 +68,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { /** * Search effectively the raw results of the first page */ - private Mono searchFirstPage(LLIndexSearchers indexSearchers, + private Mono searchFirstPage(Iterable indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { var limit = paginationInfo.totalLimit(); @@ -86,9 +89,11 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { LocalQueryParams queryParams) { return firstPageDataMono.map(firstPageData -> { var totalHitsCount = LuceneUtils.convertTotalHitsCount(firstPageData.topDocs().totalHits); + var scoreDocs = firstPageData.topDocs().scoreDocs; + assert LLUtils.isSet(scoreDocs); - Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageData.topDocs().scoreDocs), - indexSearchers, keyFieldName, true) + Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(scoreDocs), + indexSearchers.shards(), keyFieldName, true) .take(queryParams.limit(), true); CurrentPageInfo nextPageInfo = firstPageData.nextPageInfo(); @@ -97,33 +102,35 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { }); } - private Mono> computeOtherResults(Mono firstResultMono, - LLIndexSearchers indexSearchers, + private Send computeOtherResults(FirstPageResults firstResult, + List indexSearchers, LocalQueryParams queryParams, - String keyFieldName) { - return firstResultMono.map(firstResult -> { - var totalHitsCount = firstResult.totalHitsCount(); - var firstPageHitsFlux = firstResult.firstPageHitsFlux(); - var secondPageInfo = firstResult.nextPageInfo(); + String keyFieldName, + Runnable drop) { + var totalHitsCount = firstResult.totalHitsCount(); + var firstPageHitsFlux = firstResult.firstPageHitsFlux(); + var secondPageInfo = firstResult.nextPageInfo(); - Flux nextHitsFlux = searchOtherPages(indexSearchers, queryParams, keyFieldName, secondPageInfo); + Flux nextHitsFlux = searchOtherPages(indexSearchers, queryParams, keyFieldName, secondPageInfo); - Flux combinedFlux = firstPageHitsFlux.concatWith(nextHitsFlux); - return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> indexSearchers.close()).send(); - }); + Flux combinedFlux = firstPageHitsFlux.concatWith(nextHitsFlux); + return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> drop.run()).send(); } /** * Search effectively the merged raw results of the next pages */ - private Flux searchOtherPages(LLIndexSearchers indexSearchers, + private Flux searchOtherPages(List indexSearchers, LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { return Flux .defer(() -> { AtomicReference currentPageInfoRef = new AtomicReference<>(secondPageInfo); - return this - .searchPage(queryParams, indexSearchers, true, queryParams.pageLimits(), - 0, currentPageInfoRef.get()) + return Mono + .fromSupplier(currentPageInfoRef::get) + .doOnNext(s -> System.err.println("Current page info: " + s)) + .flatMap(currentPageInfo -> this.searchPage(queryParams, indexSearchers, true, + queryParams.pageLimits(), 0, currentPageInfo)) + .doOnNext(s -> System.err.println("Next page info: " + s.nextPageInfo())) .doOnNext(s -> currentPageInfoRef.set(s.nextPageInfo())) .repeatWhen(s -> s.takeWhile(n -> n > 0)); }) @@ -140,7 +147,7 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { * skip the first n results in the first page */ private Mono searchPage(LocalQueryParams queryParams, - LLIndexSearchers indexSearchers, + Iterable indexSearchers, boolean allowPagination, PageLimits pageLimits, int resultsOffset, @@ -154,18 +161,19 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { var sort = getSort(queryParams); var pageLimit = pageLimits.getPageLimit(s.pageIndex()); + var after = (FieldDoc) s.last(); var totalHitsThreshold = LuceneUtils.totalHitsThreshold(); - return new ScoringShardsCollectorManager(sort, pageLimit, null, - totalHitsThreshold, resultsOffset, pageLimit); + return new ScoringShardsCollectorManager(sort, pageLimit, after, totalHitsThreshold, + resultsOffset); } else { return null; } }) .flatMap(sharedManager -> Flux - .fromIterable(indexSearchers.shards()) + .fromIterable(indexSearchers) .flatMap(shard -> Mono.fromCallable(() -> { var collector = sharedManager.newCollector(); - shard.getIndexSearcher().search(queryParams.query(), collector); + shard.search(queryParams.query(), collector); return collector; })) .collectList() diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoringShardsCollectorManager.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoringShardsCollectorManager.java index 85908d1..92011ad 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoringShardsCollectorManager.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoringShardsCollectorManager.java @@ -2,6 +2,7 @@ package it.cavallium.dbengine.lucene.searcher; import static it.cavallium.dbengine.lucene.searcher.CurrentPageInfo.TIE_BREAKER; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.lucene.LuceneUtils; import java.io.IOException; import java.util.Collection; @@ -34,6 +35,14 @@ public class ScoringShardsCollectorManager implements CollectorManager 2147483630) { + this.topN = 2147483630 - startN; + } else if (topN != null && topN > 2147483630) { + this.topN = 2147483630; + } else { + this.topN = topN; + } this.sharedCollectorManager = TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold); } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java index 683351b..b5cb26a 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleLuceneLocalSearcher.java @@ -13,7 +13,9 @@ import it.cavallium.dbengine.database.disk.LLIndexSearchers.UnshardedIndexSearch import it.cavallium.dbengine.lucene.LuceneUtils; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Objects; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; @@ -37,12 +39,13 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { return LLUtils.usingResource(indexSearchersMono, indexSearchers -> this // Search first page results - .searchFirstPage(indexSearchers, queryParams, paginationInfo) + .searchFirstPage(indexSearchers.shards(), queryParams, paginationInfo) // Compute the results of the first page - .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers, + .transform(firstPageTopDocsMono -> this.computeFirstPageResults(firstPageTopDocsMono, indexSearchers.shards(), keyFieldName, queryParams)) // Compute other results - .transform(firstResult -> this.computeOtherResults(firstResult, indexSearchers, queryParams, keyFieldName)) + .transform(firstResult -> this.computeOtherResults(firstResult, indexSearchers.shards(), queryParams, + keyFieldName, indexSearchers::close)) // Ensure that one LuceneSearchResult is always returned .single(), false); @@ -62,7 +65,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { /** * Search effectively the raw results of the first page */ - private Mono searchFirstPage(LLIndexSearchers indexSearchers, + private Mono searchFirstPage(List indexSearchers, LocalQueryParams queryParams, PaginationInfo paginationInfo) { var limit = paginationInfo.totalLimit(); @@ -77,13 +80,15 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { * Compute the results of the first page, extracting useful data */ private Mono computeFirstPageResults(Mono firstPageDataMono, - LLIndexSearchers indexSearchers, + List indexSearchers, String keyFieldName, LocalQueryParams queryParams) { return firstPageDataMono.map(firstPageData -> { var totalHitsCount = LuceneUtils.convertTotalHitsCount(firstPageData.topDocs().totalHits); + var scoreDocs = firstPageData.topDocs().scoreDocs; + assert LLUtils.isSet(scoreDocs); - Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(firstPageData.topDocs().scoreDocs), + Flux firstPageHitsFlux = LuceneUtils.convertHits(Flux.fromArray(scoreDocs), indexSearchers, keyFieldName, true) .take(queryParams.limit(), true); @@ -94,9 +99,10 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { } private Mono> computeOtherResults(Mono firstResultMono, - UnshardedIndexSearchers indexSearchers, + List indexSearchers, LocalQueryParams queryParams, - String keyFieldName) { + String keyFieldName, + Runnable drop) { return firstResultMono.map(firstResult -> { var totalHitsCount = firstResult.totalHitsCount(); var firstPageHitsFlux = firstResult.firstPageHitsFlux(); @@ -105,14 +111,14 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { Flux nextHitsFlux = searchOtherPages(indexSearchers, queryParams, keyFieldName, secondPageInfo); Flux combinedFlux = firstPageHitsFlux.concatWith(nextHitsFlux); - return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> indexSearchers.close()).send(); + return new LuceneSearchResult(totalHitsCount, combinedFlux, d -> drop.run()).send(); }); } /** * Search effectively the merged raw results of the next pages */ - private Flux searchOtherPages(UnshardedIndexSearchers indexSearchers, + private Flux searchOtherPages(List indexSearchers, LocalQueryParams queryParams, String keyFieldName, CurrentPageInfo secondPageInfo) { return Flux .generate( @@ -133,7 +139,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { * skip the first n results in the first page */ private CurrentPageInfo searchPageSync(LocalQueryParams queryParams, - LLIndexSearchers indexSearchers, + List indexSearchers, boolean allowPagination, int resultsOffset, CurrentPageInfo s, @@ -142,12 +148,6 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { if (resultsOffset < 0) { throw new IndexOutOfBoundsException(resultsOffset); } - UnshardedIndexSearchers unshardedIndexSearchers; - if (indexSearchers instanceof UnshardedIndexSearchers unshardedIndexSearchers1) { - unshardedIndexSearchers = unshardedIndexSearchers1; - } else { - throw new IllegalArgumentException(); - } var currentPageLimit = queryParams.pageLimits().getPageLimit(s.pageIndex()); if ((s.pageIndex() == 0 || s.last() != null) && s.remainingLimit() > 0) { TopDocs pageTopDocs; @@ -155,7 +155,7 @@ public class SimpleLuceneLocalSearcher implements LuceneLocalSearcher { TopDocsCollector collector = TopDocsSearcher.getTopDocsCollector(queryParams.sort(), currentPageLimit, s.last(), LuceneUtils.totalHitsThreshold(), allowPagination, queryParams.isScored()); - unshardedIndexSearchers.shard().getIndexSearcher().search(queryParams.query(), collector); + indexSearchers.get(0).search(queryParams.query(), collector); if (resultsOffset > 0) { pageTopDocs = collector.topDocs(resultsOffset, currentPageLimit); } else { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index 6638c5a..a1c7503 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -1,15 +1,12 @@ package it.cavallium.dbengine.lucene.searcher; -import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; -import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearchers; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -40,11 +37,14 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } }) .then(indexSearchersMono.map(Send::receive)); + var localQueryParams = getLocalQueryParams(queryParams); return LLUtils.usingResource(indexSearchersResource, indexSearchers -> Flux.fromIterable(indexSearchers.shards()) - .flatMap(searcher -> localSearcher - .collect(Mono.just(searcher.send()), queryParams, keyFieldName, transformer)) + .flatMap(searcher -> { + var llSearcher = Mono.fromCallable(() -> new LLIndexSearcher(searcher, d -> {}).send()); + return localSearcher.collect(llSearcher, localQueryParams, keyFieldName, transformer); + }) .collectList() .map(results -> { List resultsToDrop = new ArrayList<>(results.size()); @@ -60,7 +60,10 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } var totalHitsCount = new TotalHitsCount(totalHitsCountValue, exactTotalHitsCount); - Flux mergedFluxes = Flux.merge(resultsFluxes); + Flux mergedFluxes = Flux + .merge(resultsFluxes) + .skip(queryParams.offset()) + .take(queryParams.limit(), true); return new LuceneSearchResult(totalHitsCount, mergedFluxes, d -> { for (LuceneSearchResult luceneSearchResult : resultsToDrop) { @@ -68,7 +71,18 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea } }).send(); }), - true + false + ); + } + + private LocalQueryParams getLocalQueryParams(LocalQueryParams queryParams) { + return new LocalQueryParams(queryParams.query(), + 0, + queryParams.limit(), + queryParams.pageLimits(), + queryParams.minCompetitiveScore(), + queryParams.sort(), + queryParams.scoreMode() ); } } diff --git a/src/main/java/org/rocksdb/CappedWriteBatch.java b/src/main/java/org/rocksdb/CappedWriteBatch.java index f5e11d6..1d242f5 100644 --- a/src/main/java/org/rocksdb/CappedWriteBatch.java +++ b/src/main/java/org/rocksdb/CappedWriteBatch.java @@ -105,13 +105,13 @@ public class CappedWriteBatch extends WriteBatch { var value = valueToReceive.receive(); if (USE_FAST_DIRECT_BUFFERS && isDirect(key) && isDirect(value)) { buffersToRelease.add(value); - var keyNioBuffer = LLUtils.convertToDirect(alloc, key.send()); + var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); key = keyNioBuffer.buffer().receive(); buffersToRelease.add(key); byteBuffersToRelease.add(keyNioBuffer.byteBuffer()); assert keyNioBuffer.byteBuffer().isDirect(); - var valueNioBuffer = LLUtils.convertToDirect(alloc, value.send()); + var valueNioBuffer = LLUtils.convertToReadableDirect(alloc, value.send()); value = valueNioBuffer.buffer().receive(); buffersToRelease.add(value); byteBuffersToRelease.add(valueNioBuffer.byteBuffer()); @@ -172,7 +172,7 @@ public class CappedWriteBatch extends WriteBatch { public synchronized void delete(ColumnFamilyHandle columnFamilyHandle, Send keyToReceive) throws RocksDBException { var key = keyToReceive.receive(); if (USE_FAST_DIRECT_BUFFERS) { - var keyNioBuffer = LLUtils.convertToDirect(alloc, key.send()); + var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); key = keyNioBuffer.buffer().receive(); buffersToRelease.add(key); byteBuffersToRelease.add(keyNioBuffer.byteBuffer()); diff --git a/src/test/java/it/cavallium/dbengine/DbTestUtils.java b/src/test/java/it/cavallium/dbengine/DbTestUtils.java index dfb27c7..79007fe 100644 --- a/src/test/java/it/cavallium/dbengine/DbTestUtils.java +++ b/src/test/java/it/cavallium/dbengine/DbTestUtils.java @@ -98,7 +98,7 @@ public class DbTestUtils { if (!MemorySegmentUtils.isSupported()) { System.err.println("Warning! Foreign Memory Access API is not available!" + " Netty direct buffers will not be used in tests!" - + " Please set \"--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit\""); + + " Please set \"" + MemorySegmentUtils.getSuggestedArgs() + "\""); if (MemorySegmentUtils.getUnsupportedCause() != null) { System.err.println("\tCause: " + MemorySegmentUtils.getUnsupportedCause().getClass().getName() + ":" + MemorySegmentUtils.getUnsupportedCause().getLocalizedMessage()); From e034f3b77865580e993748b271cb6bd5e5da131a Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Wed, 22 Sep 2021 18:33:28 +0200 Subject: [PATCH 13/23] Support memory segments --- pom.xml | 9 + .../dbengine/client/MappedSerializer.java | 5 +- .../client/MappedSerializerFixedLength.java | 5 +- .../cavallium/dbengine/database/LLUtils.java | 83 ++------ .../database/RepeatedElementList.java | 7 +- .../database/collections/DatabaseEmpty.java | 7 +- .../collections/DatabaseMapDictionary.java | 48 +++-- .../DatabaseMapDictionaryDeep.java | 191 ++++++++---------- .../DatabaseMapDictionaryHashed.java | 2 +- .../collections/DatabaseSetDictionary.java | 4 +- .../DatabaseSetDictionaryHashed.java | 2 +- .../collections/ValueWithHashSerializer.java | 21 +- .../collections/ValuesSetSerializer.java | 14 +- .../database/disk/LLLocalDictionary.java | 15 +- .../LLLocalEntryReactiveRocksIterator.java | 2 +- .../LLLocalGroupedReactiveRocksIterator.java | 17 +- ...LLLocalKeyPrefixReactiveRocksIterator.java | 2 +- .../disk/LLLocalKeyReactiveRocksIterator.java | 5 +- .../disk/LLLocalReactiveRocksIterator.java | 20 +- .../serialization/BufferDataInput.java | 53 ++++- .../serialization/CodecSerializer.java | 3 +- .../database/serialization/Serializer.java | 17 +- .../SerializerFixedBinaryLength.java | 26 ++- .../dbengine/lucene/LuceneUtils.java | 3 +- .../it/cavallium/dbengine/DbTestUtils.java | 5 +- .../database/collections/TestRanges.java | 8 +- 26 files changed, 297 insertions(+), 277 deletions(-) diff --git a/pom.xml b/pom.xml index 13d2df6..ec68272 100644 --- a/pom.xml +++ b/pom.xml @@ -99,6 +99,10 @@ io.net5 netty-buffer + + io.net5.incubator + netty-incubator-buffer-memseg + javax.xml.bind jaxb-api @@ -266,6 +270,11 @@ netty-buffer 5.0.0.Final-SNAPSHOT + + io.net5.incubator + netty-incubator-buffer-memseg + 0.0.1.Final-SNAPSHOT + javax.xml.bind jaxb-api diff --git a/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java b/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java index 353ee52..f29c5ff 100644 --- a/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java +++ b/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java @@ -5,6 +5,7 @@ import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.Serializer; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class MappedSerializer implements Serializer { @@ -18,7 +19,7 @@ public class MappedSerializer implements Serializer { } @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException { + public @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException { try (serialized) { var deserialized = serializer.deserialize(serialized); return new DeserializationResult<>(keyMapper.map(deserialized.deserializedData()), deserialized.bytesRead()); @@ -26,7 +27,7 @@ public class MappedSerializer implements Serializer { } @Override - public @NotNull Send serialize(@NotNull B deserialized) throws SerializationException { + public @Nullable Send serialize(@NotNull B deserialized) throws SerializationException { return serializer.serialize(keyMapper.unmap(deserialized)); } } diff --git a/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java b/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java index 1db9558..2d970f8 100644 --- a/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java +++ b/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java @@ -5,6 +5,7 @@ import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class MappedSerializerFixedLength implements SerializerFixedBinaryLength { @@ -18,7 +19,7 @@ public class MappedSerializerFixedLength implements SerializerFixedBinaryL } @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException { + public @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException { try (serialized) { var deserialized = fixedLengthSerializer.deserialize(serialized); return new DeserializationResult<>(keyMapper.map(deserialized.deserializedData()), deserialized.bytesRead()); @@ -26,7 +27,7 @@ public class MappedSerializerFixedLength implements SerializerFixedBinaryL } @Override - public @NotNull Send serialize(@NotNull B deserialized) throws SerializationException { + public @Nullable Send serialize(@NotNull B deserialized) throws SerializationException { return fixedLengthSerializer.serialize(keyMapper.unmap(deserialized)); } diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 947de21..1cd7699 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -313,7 +313,7 @@ public class LLUtils { */ @SuppressWarnings("ConstantConditions") @Nullable - public static Send readNullableDirectNioBuffer(BufferAllocator alloc, ToIntFunction reader) { + public static Buffer readNullableDirectNioBuffer(BufferAllocator alloc, ToIntFunction reader) { ByteBuffer directBuffer; Buffer buffer; { @@ -330,7 +330,7 @@ public class LLUtils { if (size != RocksDB.NOT_FOUND) { if (size == directBuffer.limit()) { buffer.readerOffset(0).writerOffset(size); - return buffer.send(); + return buffer; } else { assert size > directBuffer.limit(); assert directBuffer.limit() > 0; @@ -522,85 +522,32 @@ public class LLUtils { } @NotNull - public static Send readDirectNioBuffer(BufferAllocator alloc, ToIntFunction reader) { - var nullableSend = readNullableDirectNioBuffer(alloc, reader); - try (var buffer = nullableSend != null ? nullableSend.receive() : null) { - if (buffer == null) { - throw new IllegalStateException("A non-nullable buffer read operation tried to return a \"not found\" element"); - } - return buffer.send(); + public static Buffer readDirectNioBuffer(BufferAllocator alloc, ToIntFunction reader) { + var nullable = readNullableDirectNioBuffer(alloc, reader); + if (nullable == null) { + throw new IllegalStateException("A non-nullable buffer read operation tried to return a \"not found\" element"); } + return nullable; } - public static Send compositeBuffer(BufferAllocator alloc, Send buffer) { - try (var composite = buffer.receive()) { - return composite.send(); - } + public static Buffer compositeBuffer(BufferAllocator alloc, Send buffer) { + return buffer.receive(); } - public static Send compositeBuffer(BufferAllocator alloc, Send buffer1, Send buffer2) { - try (var buf1 = buffer1.receive()) { - try (var buf2 = buffer2.receive()) { - try (var composite = CompositeBuffer.compose(alloc, buf1.split().send(), buf2.split().send())) { - return composite.send(); - } - } - } + public static Buffer compositeBuffer(BufferAllocator alloc, Send buffer1, Send buffer2) { + return CompositeBuffer.compose(alloc, buffer1, buffer2); } - public static Send compositeBuffer(BufferAllocator alloc, + public static Buffer compositeBuffer(BufferAllocator alloc, Send buffer1, Send buffer2, Send buffer3) { - try (var buf1 = buffer1.receive()) { - try (var buf2 = buffer2.receive()) { - try (var buf3 = buffer3.receive()) { - try (var composite = CompositeBuffer.compose(alloc, - buf1.split().send(), - buf2.split().send(), - buf3.split().send() - )) { - return composite.send(); - } - } - } - } + return CompositeBuffer.compose(alloc, buffer1, buffer2, buffer3); } @SafeVarargs - public static Send compositeBuffer(BufferAllocator alloc, Send... buffers) { - try { - return switch (buffers.length) { - case 0 -> alloc.allocate(0).send(); - case 1 -> compositeBuffer(alloc, buffers[0]); - case 2 -> compositeBuffer(alloc, buffers[0], buffers[1]); - case 3 -> compositeBuffer(alloc, buffers[0], buffers[1], buffers[2]); - default -> { - Buffer[] bufs = new Buffer[buffers.length]; - for (int i = 0; i < buffers.length; i++) { - bufs[i] = buffers[i].receive(); - } - try { - //noinspection unchecked - Send[] sentBufs = new Send[buffers.length]; - for (int i = 0; i < buffers.length; i++) { - sentBufs[i] = bufs[i].split().send(); - } - try (var composite = CompositeBuffer.compose(alloc, sentBufs)) { - yield composite.send(); - } - } finally { - for (Buffer buf : bufs) { - buf.close(); - } - } - } - }; - } finally { - for (Send buffer : buffers) { - buffer.close(); - } - } + public static Buffer compositeBuffer(BufferAllocator alloc, Send... buffers) { + return CompositeBuffer.compose(alloc, buffers); } public static Mono resolveDelta(Mono> prev, UpdateReturnMode updateReturnMode) { diff --git a/src/main/java/it/cavallium/dbengine/database/RepeatedElementList.java b/src/main/java/it/cavallium/dbengine/database/RepeatedElementList.java index 503f4ac..f1339dd 100644 --- a/src/main/java/it/cavallium/dbengine/database/RepeatedElementList.java +++ b/src/main/java/it/cavallium/dbengine/database/RepeatedElementList.java @@ -46,7 +46,7 @@ public class RepeatedElementList implements List { @NotNull @Override - public Object[] toArray() { + public Object @NotNull [] toArray() { var arr = new Object[size]; Arrays.fill(arr, element); return arr; @@ -54,7 +54,7 @@ public class RepeatedElementList implements List { @NotNull @Override - public T1[] toArray(@NotNull T1[] a) { + public T1 @NotNull [] toArray(@NotNull T1 @NotNull [] a) { var arr = Arrays.copyOf(a, size); Arrays.fill(arr, element); return arr; @@ -152,8 +152,9 @@ public class RepeatedElementList implements List { @NotNull @Override public ListIterator listIterator(int index) { - return new ListIterator() { + return new ListIterator<>() { int position = index - 1; + @Override public boolean hasNext() { return position + 1 < size; diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java index d33e58d..0f11eb0 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java @@ -7,6 +7,7 @@ import it.cavallium.dbengine.database.LLDictionary; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.Serializer.DeserializationResult; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class DatabaseEmpty { @@ -17,15 +18,15 @@ public class DatabaseEmpty { public static Serializer nothingSerializer(BufferAllocator bufferAllocator) { return new Serializer<>() { @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serialized) { + public @NotNull DeserializationResult deserialize(@Nullable Send serialized) { try (serialized) { return NOTHING_RESULT; } } @Override - public @NotNull Send serialize(@NotNull Nothing deserialized) { - return bufferAllocator.allocate(0).send(); + public @Nullable Send serialize(@NotNull Nothing deserialized) { + return null; } }; } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java index a60bf43..386326a 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java @@ -39,7 +39,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep valueSerializer; protected DatabaseMapDictionary(LLDictionary dictionary, - Send prefixKey, + @Nullable Send prefixKey, SerializerFixedBinaryLength keySuffixSerializer, Serializer valueSerializer) { // Do not retain or release or use the prefixKey here @@ -50,7 +50,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep DatabaseMapDictionary simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, Serializer valueSerializer) { - return new DatabaseMapDictionary<>(dictionary, dictionary.getAllocator().allocate(0).send(), keySerializer, valueSerializer); + return new DatabaseMapDictionary<>(dictionary, null, keySerializer, valueSerializer); } public static DatabaseMapDictionary tail(LLDictionary dictionary, @@ -60,10 +60,13 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep(dictionary, prefixKey, keySuffixSerializer, valueSerializer); } - private Send toKey(Send suffixKeyToSend) { - try (var suffixKey = suffixKeyToSend.receive()) { - assert suffixKeyConsistency(suffixKey.readableBytes()); + private Buffer toKey(Send suffixKeyToSend) { + var suffixKey = suffixKeyToSend.receive(); + assert suffixKeyConsistency(suffixKey.readableBytes()); + if (keyPrefix != null) { return LLUtils.compositeBuffer(dictionary.getAllocator(), keyPrefix.copy().send(), suffixKey.send()); + } else { + return suffixKey; } } @@ -100,7 +103,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep { try { - sink.next(LLEntry.of(this.toKey(serializeSuffix(entry.getKey())), + sink.next(LLEntry.of(this.toKey(serializeSuffix(entry.getKey()).send()).send(), valueSerializer.serialize(entry.getValue())).send()); } catch (SerializationException e) { sink.error(e); @@ -130,19 +133,22 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep> at(@Nullable CompositeSnapshot snapshot, T keySuffix) { return Mono.fromCallable(() -> - new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix)), valueSerializer)); + new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix).send()).send(), valueSerializer)); } @Override public Mono getValue(@Nullable CompositeSnapshot snapshot, T keySuffix, boolean existsAlmostCertainly) { return dictionary - .get(resolveSnapshot(snapshot), Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))), existsAlmostCertainly) + .get(resolveSnapshot(snapshot), + Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()), + existsAlmostCertainly + ) .handle((value, sink) -> deserializeValue(value, sink)); } @Override public Mono putValue(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); return dictionary .put(keyMono, valueMono, LLDictionaryResultType.VOID) @@ -160,7 +166,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep updater) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); return dictionary .update(keyMono, getSerializedUpdater(updater), updateReturnMode, existsAlmostCertainly) .handle((value, sink) -> deserializeValue(value, sink)); @@ -170,7 +176,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep> updateValueAndGetDelta(T keySuffix, boolean existsAlmostCertainly, SerializationFunction<@Nullable U, @Nullable U> updater) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); return dictionary .updateAndGetDelta(keyMono, getSerializedUpdater(updater), existsAlmostCertainly) .transform(mono -> LLUtils.mapLLDelta(mono, @@ -218,7 +224,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep putValueAndGetPrevious(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); return dictionary .put(keyMono, @@ -229,7 +235,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep putValueAndGetChanged(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); return dictionary .put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE) @@ -240,7 +246,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep remove(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); return dictionary .remove(keyMono, LLDictionaryResultType.VOID) .doOnNext(Send::close) @@ -249,7 +255,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep removeAndGetPrevious(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); return dictionary .remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE) .handle((value, sink) -> deserializeValue(value, sink)); @@ -257,7 +263,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep removeAndGetStatus(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); return dictionary .remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) .map(LLUtils::responseToBoolean); @@ -268,7 +274,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep>>handle((keySuffix, sink) -> { try { - sink.next(Tuples.of(keySuffix, toKey(serializeSuffix(keySuffix)))); + sink.next(Tuples.of(keySuffix, toKey(serializeSuffix(keySuffix).send()).send())); } catch (SerializationException ex) { sink.error(ex); } @@ -297,7 +303,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep serializeEntry(T key, U value) throws SerializationException { - try (var serializedKey = toKey(serializeSuffix(key)).receive()) { + try (var serializedKey = toKey(serializeSuffix(key).send())) { try (var serializedValue = valueSerializer.serialize(value).receive()) { return LLEntry.of(serializedKey.send(), serializedValue.send()).send(); } @@ -337,7 +343,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep, X>>handle((entry, sink) -> { try { - sink.next(Tuples.of(serializeSuffix(entry.getT1()), entry.getT2())); + sink.next(Tuples.of(serializeSuffix(entry.getT1()).send(), entry.getT2())); } catch (SerializationException ex) { sink.error(ex); } @@ -373,7 +379,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep(dictionary, - toKey(keySuffixWithExt.send()), + toKey(keySuffixWithExt.send()).send(), valueSerializer ) )); @@ -412,7 +418,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep { try { - sink.next(LLEntry.of(toKey(serializeSuffix(entry.getKey())), + sink.next(LLEntry.of(toKey(serializeSuffix(entry.getKey()).send()).send(), valueSerializer.serialize(entry.getValue())).send()); } catch (SerializationException e) { sink.error(e); diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java index 075b5dc..7dc589a 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java @@ -2,6 +2,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; +import io.net5.buffer.api.CompositeBuffer; import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; @@ -36,53 +37,39 @@ public class DatabaseMapDictionaryDeep> implem protected final Mono> rangeMono; private volatile boolean released; - private static Send incrementPrefix(BufferAllocator alloc, Send originalKeySend, int prefixLength) { - try (var originalKey = originalKeySend.receive()) { - assert originalKey.readableBytes() >= prefixLength; - var originalKeyLength = originalKey.readableBytes(); - try (Buffer copiedBuf = alloc.allocate(originalKey.readableBytes())) { - boolean overflowed = true; - final int ff = 0xFF; - int writtenBytes = 0; - copiedBuf.writerOffset(prefixLength); - for (int i = prefixLength - 1; i >= 0; i--) { - int iByte = originalKey.getUnsignedByte(i); - if (iByte != ff) { - copiedBuf.setUnsignedByte(i, iByte + 1); - writtenBytes++; - overflowed = false; - break; - } else { - copiedBuf.setUnsignedByte(i, 0x00); - writtenBytes++; - overflowed = true; - } - } - assert prefixLength - writtenBytes >= 0; - if (prefixLength - writtenBytes > 0) { - originalKey.copyInto(0, copiedBuf, 0, (prefixLength - writtenBytes)); - } - - copiedBuf.writerOffset(originalKeyLength); - - if (originalKeyLength - prefixLength > 0) { - originalKey.copyInto(prefixLength, copiedBuf, prefixLength, originalKeyLength - prefixLength); - } - - if (overflowed) { - copiedBuf.ensureWritable(originalKeyLength + 1); - copiedBuf.writerOffset(originalKeyLength + 1); - for (int i = 0; i < originalKeyLength; i++) { - copiedBuf.setUnsignedByte(i, 0xFF); - } - copiedBuf.setUnsignedByte(originalKeyLength, (byte) 0x00); - } - return copiedBuf.send(); + private static void incrementPrefix(Buffer prefix, int prefixLength) { + assert prefix.readableBytes() >= prefixLength; + assert prefix.readerOffset() == 0; + final var originalKeyLength = prefix.readableBytes(); + boolean overflowed = true; + final int ff = 0xFF; + int writtenBytes = 0; + for (int i = prefixLength - 1; i >= 0; i--) { + int iByte = prefix.getUnsignedByte(i); + if (iByte != ff) { + prefix.setUnsignedByte(i, iByte + 1); + writtenBytes++; + overflowed = false; + break; + } else { + prefix.setUnsignedByte(i, 0x00); + writtenBytes++; } } + assert prefixLength - writtenBytes >= 0; + + if (overflowed) { + assert prefix.writerOffset() == originalKeyLength; + prefix.ensureWritable(1, 1, true); + prefix.writerOffset(originalKeyLength + 1); + for (int i = 0; i < originalKeyLength; i++) { + prefix.setUnsignedByte(i, 0xFF); + } + prefix.setUnsignedByte(originalKeyLength, (byte) 0x00); + } } - static Send firstRangeKey(BufferAllocator alloc, + static Buffer firstRangeKey(BufferAllocator alloc, Send prefixKey, int prefixLength, int suffixLength, @@ -90,40 +77,44 @@ public class DatabaseMapDictionaryDeep> implem return zeroFillKeySuffixAndExt(alloc, prefixKey, prefixLength, suffixLength, extLength); } - static Send nextRangeKey(BufferAllocator alloc, + static Buffer nextRangeKey(BufferAllocator alloc, Send prefixKey, int prefixLength, int suffixLength, int extLength) { try (prefixKey) { - try (Send nonIncremented = zeroFillKeySuffixAndExt(alloc, prefixKey, prefixLength, suffixLength, - extLength)) { - return incrementPrefix(alloc, nonIncremented, prefixLength); - } + Buffer nonIncremented = zeroFillKeySuffixAndExt(alloc, prefixKey, prefixLength, suffixLength, extLength); + incrementPrefix(nonIncremented, prefixLength); + return nonIncremented; } } - protected static Send zeroFillKeySuffixAndExt(BufferAllocator alloc, - Send prefixKeySend, + protected static Buffer zeroFillKeySuffixAndExt(BufferAllocator alloc, + @Nullable Send prefixKeySend, int prefixLength, int suffixLength, int extLength) { - try (var prefixKey = prefixKeySend.receive()) { - assert prefixKey.readableBytes() == prefixLength; - assert suffixLength > 0; - assert extLength >= 0; - try (Buffer zeroSuffixAndExt = alloc.allocate(suffixLength + extLength)) { + try (var result = prefixKeySend == null ? null : prefixKeySend.receive()) { + if (result == null) { + assert prefixLength == 0; + var buf = alloc.allocate(prefixLength + suffixLength + extLength); + buf.writerOffset(prefixLength + suffixLength + extLength); + buf.fill((byte) 0); + return buf; + } else { + assert result.readableBytes() == prefixLength; + assert suffixLength > 0; + assert extLength >= 0; + result.ensureWritable(suffixLength + extLength, suffixLength + extLength, true); for (int i = 0; i < suffixLength + extLength; i++) { - zeroSuffixAndExt.writeByte((byte) 0x0); - } - try (Buffer result = LLUtils.compositeBuffer(alloc, prefixKey.send(), zeroSuffixAndExt.send()).receive()) { - return result.send(); + result.writeByte((byte) 0x0); } + return result; } } } - static Send firstRangeKey( + static Buffer firstRangeKey( BufferAllocator alloc, Send prefixKey, Send suffixKey, @@ -133,25 +124,25 @@ public class DatabaseMapDictionaryDeep> implem return zeroFillKeyExt(alloc, prefixKey, suffixKey, prefixLength, suffixLength, extLength); } - static Send nextRangeKey( + static Buffer nextRangeKey( BufferAllocator alloc, Send prefixKey, Send suffixKey, int prefixLength, int suffixLength, int extLength) { - try (Send nonIncremented = zeroFillKeyExt(alloc, + Buffer nonIncremented = zeroFillKeyExt(alloc, prefixKey, suffixKey, prefixLength, suffixLength, extLength - )) { - return incrementPrefix(alloc, nonIncremented, prefixLength + suffixLength); - } + ); + incrementPrefix(nonIncremented, prefixLength + suffixLength); + return nonIncremented; } - protected static Send zeroFillKeyExt( + protected static Buffer zeroFillKeyExt( BufferAllocator alloc, Send prefixKeySend, Send suffixKeySend, @@ -165,17 +156,14 @@ public class DatabaseMapDictionaryDeep> implem assert suffixLength > 0; assert extLength >= 0; - try (var ext = alloc.allocate(extLength)) { - for (int i = 0; i < extLength; i++) { - ext.writeByte((byte) 0); - } - - try (Buffer result = LLUtils.compositeBuffer(alloc, prefixKey.send(), suffixKey.send(), ext.send()) - .receive()) { - assert result.readableBytes() == prefixLength + suffixLength + extLength; - return result.send(); - } + Buffer result = LLUtils.compositeBuffer(alloc, prefixKey.send(), suffixKey.send()); + result.ensureWritable(extLength, extLength, true); + for (int i = 0; i < extLength; i++) { + result.writeByte((byte) 0); } + + assert result.readableBytes() == prefixLength + suffixLength + extLength; + return result; } } } @@ -187,8 +175,7 @@ public class DatabaseMapDictionaryDeep> implem public static DatabaseMapDictionaryDeep> simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, SubStageGetterSingle subStageGetter) { - return new DatabaseMapDictionaryDeep<>(dictionary, dictionary.getAllocator().allocate(0).send(), - keySerializer, subStageGetter, 0); + return new DatabaseMapDictionaryDeep<>(dictionary, null, keySerializer, subStageGetter, 0); } public static > DatabaseMapDictionaryDeep deepTail(LLDictionary dictionary, @@ -196,7 +183,7 @@ public class DatabaseMapDictionaryDeep> implem int keyExtLength, SubStageGetter subStageGetter) { return new DatabaseMapDictionaryDeep<>(dictionary, - dictionary.getAllocator().allocate(0).send(), + null, keySerializer, subStageGetter, keyExtLength @@ -212,39 +199,34 @@ public class DatabaseMapDictionaryDeep> implem } protected DatabaseMapDictionaryDeep(LLDictionary dictionary, - Send prefixKeyToReceive, + @Nullable Send prefixKeyToReceive, SerializerFixedBinaryLength keySuffixSerializer, SubStageGetter subStageGetter, int keyExtLength) { - try (var prefixKey = prefixKeyToReceive.receive()) { + try (var prefixKey = prefixKeyToReceive == null ? null : prefixKeyToReceive.receive()) { this.dictionary = dictionary; this.alloc = dictionary.getAllocator(); this.subStageGetter = subStageGetter; this.keySuffixSerializer = keySuffixSerializer; - this.keyPrefix = prefixKey.copy(); - assert keyPrefix.isAccessible(); - this.keyPrefixLength = keyPrefix.readableBytes(); + assert prefixKey == null || prefixKey.isAccessible(); + this.keyPrefixLength = prefixKey == null ? 0 : prefixKey.readableBytes(); this.keySuffixLength = keySuffixSerializer.getSerializedBinaryLength(); this.keyExtLength = keyExtLength; - try (Buffer firstKey = firstRangeKey(alloc, - prefixKey.copy().send(), - keyPrefixLength, - keySuffixLength, - keyExtLength - ).receive().compact()) { - try (Buffer nextRangeKey = nextRangeKey(alloc, - prefixKey.copy().send(), - keyPrefixLength, - keySuffixLength, - keyExtLength - ).receive().compact()) { - assert keyPrefix.isAccessible(); + Buffer firstKey = firstRangeKey(alloc, prefixKey == null ? null : prefixKey.copy().send(), keyPrefixLength, + keySuffixLength, keyExtLength); + try (firstKey) { + var nextRangeKey = nextRangeKey(alloc, prefixKey == null ? null : prefixKey.copy().send(), + keyPrefixLength, keySuffixLength, keyExtLength); + try (nextRangeKey) { + assert prefixKey == null || prefixKey.isAccessible(); assert keyPrefixLength == 0 || !LLUtils.equals(firstKey, nextRangeKey); this.range = keyPrefixLength == 0 ? LLRange.all() : LLRange.of(firstKey.send(), nextRangeKey.send()); this.rangeMono = LLUtils.lazyRetainRange(this.range); assert subStageKeysConsistency(keyPrefixLength + keySuffixLength + keyExtLength); } } + + this.keyPrefix = prefixKey == null ? null : prefixKey.send().receive(); } } @@ -278,7 +260,7 @@ public class DatabaseMapDictionaryDeep> implem protected Send toKeyWithoutExt(Send suffixKeyToReceive) { try (var suffixKey = suffixKeyToReceive.receive()) { assert suffixKey.readableBytes() == keySuffixLength; - try (Buffer result = LLUtils.compositeBuffer(alloc, keyPrefix.copy().send(), suffixKey.send()).receive()) { + try (Buffer result = LLUtils.compositeBuffer(alloc, keyPrefix.copy().send(), suffixKey.send())) { assert result.readableBytes() == keyPrefixLength + keySuffixLength; return result.send(); } @@ -306,7 +288,7 @@ public class DatabaseMapDictionaryDeep> implem @Override public Mono at(@Nullable CompositeSnapshot snapshot, T keySuffix) { return this.subStageGetter - .subStage(dictionary, snapshot, Mono.fromCallable(() -> toKeyWithoutExt(serializeSuffix(keySuffix)))) + .subStage(dictionary, snapshot, Mono.fromCallable(() -> toKeyWithoutExt(serializeSuffix(keySuffix).send()))) .transform(LLUtils::handleDiscard) .doOnDiscard(DatabaseStage.class, DatabaseStage::release); } @@ -411,12 +393,11 @@ public class DatabaseMapDictionaryDeep> implem } //todo: temporary wrapper. convert the whole class to buffers - protected Send serializeSuffix(T keySuffix) throws SerializationException { - try (Buffer suffixData = keySuffixSerializer.serialize(keySuffix).receive()) { - assert suffixKeyConsistency(suffixData.readableBytes()); - assert keyPrefix.isAccessible(); - return suffixData.send(); - } + protected Buffer serializeSuffix(T keySuffix) throws SerializationException { + Buffer suffixData = keySuffixSerializer.serialize(keySuffix).receive(); + assert suffixKeyConsistency(suffixData.readableBytes()); + assert keyPrefix.isAccessible(); + return suffixData; } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java index 9c0d89d..90f6e34 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java @@ -58,7 +58,7 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap keyHashSerializer) { return new DatabaseMapDictionaryHashed<>( dictionary, - dictionary.getAllocator().allocate(0).send(), + null, keySerializer, valueSerializer, keyHashFunction, diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java index b2506f4..f1903f8 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java @@ -23,9 +23,7 @@ public class DatabaseSetDictionary extends DatabaseMapDictionary public static DatabaseSetDictionary simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer) { - try (var buf = dictionary.getAllocator().allocate(0)) { - return new DatabaseSetDictionary<>(dictionary, buf.send(), keySerializer); - } + return new DatabaseSetDictionary<>(dictionary, null, keySerializer); } public static DatabaseSetDictionary tail(LLDictionary dictionary, diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java index 9be9db7..c9c6a38 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java @@ -36,7 +36,7 @@ public class DatabaseSetDictionaryHashed extends DatabaseMapDictionaryHas Function keyHashFunction, SerializerFixedBinaryLength keyHashSerializer) { return new DatabaseSetDictionaryHashed<>(dictionary, - dictionary.getAllocator().allocate(0).send(), + null, keySerializer, keyHashFunction, keyHashSerializer diff --git a/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java b/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java index 6a6514b..f380b73 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java @@ -8,7 +8,9 @@ import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.Serializer; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; class ValueWithHashSerializer implements Serializer> { @@ -25,8 +27,9 @@ class ValueWithHashSerializer implements Serializer> { } @Override - public @NotNull DeserializationResult> deserialize(@NotNull Send serializedToReceive) + public @NotNull DeserializationResult> deserialize(@Nullable Send serializedToReceive) throws SerializationException { + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { DeserializationResult deserializedKey = keySuffixSerializer.deserialize(serialized.copy().send()); DeserializationResult deserializedValue = valueSerializer.deserialize(serialized @@ -40,11 +43,17 @@ class ValueWithHashSerializer implements Serializer> { } @Override - public @NotNull Send serialize(@NotNull Entry deserialized) throws SerializationException { - try (Buffer keySuffix = keySuffixSerializer.serialize(deserialized.getKey()).receive()) { - try (Buffer value = valueSerializer.serialize(deserialized.getValue()).receive()) { - return LLUtils.compositeBuffer(allocator, keySuffix.send(), value.send()); - } + public @Nullable Send serialize(@NotNull Entry deserialized) throws SerializationException { + var keySuffix = keySuffixSerializer.serialize(deserialized.getKey()); + var value = valueSerializer.serialize(deserialized.getValue()); + if (value == null && keySuffix == null) { + return null; + } else if (value == null) { + return keySuffix; + } else if (keySuffix == null) { + return value; + } else { + return LLUtils.compositeBuffer(allocator, keySuffix, value).send(); } } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java b/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java index ec24f86..749ba0f 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java @@ -7,7 +7,9 @@ import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.Serializer; import it.unimi.dsi.fastutil.objects.ObjectArraySet; import java.util.ArrayList; +import java.util.Objects; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; class ValuesSetSerializer implements Serializer> { @@ -20,7 +22,8 @@ class ValuesSetSerializer implements Serializer> { } @Override - public @NotNull DeserializationResult> deserialize(@NotNull Send serializedToReceive) throws SerializationException { + public @NotNull DeserializationResult> deserialize(@Nullable Send serializedToReceive) throws SerializationException { + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { int initialReaderOffset = serialized.readerOffset(); int entriesLength = serialized.readInt(); @@ -41,9 +44,12 @@ class ValuesSetSerializer implements Serializer> { try (Buffer output = allocator.allocate(64)) { output.writeInt(deserialized.size()); for (X entry : deserialized) { - try (Buffer serialized = entrySerializer.serialize(entry).receive()) { - output.ensureWritable(serialized.readableBytes()); - output.writeBytes(serialized); + var serializedToReceive = entrySerializer.serialize(entry); + if (serializedToReceive != null) { + try (Buffer serialized = serializedToReceive.receive()) { + output.ensureWritable(serialized.readableBytes()); + output.writeBytes(serialized); + } } } return output.send(); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java index 51351cd..f4165da 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java @@ -1565,7 +1565,8 @@ public class LLLocalDictionary implements LLDictionary { return Flux.usingWhen(rangeMono, rangeSend -> Flux.using( () -> new LLLocalKeyReactiveRocksIterator(db, alloc, cfh, rangeSend, - databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot), getRangeKeysMultiDebugName), + databaseOptions.allowNettyDirect(), resolveSnapshot(snapshot) + ), llLocalKeyReactiveRocksIterator -> llLocalKeyReactiveRocksIterator.flux().subscribeOn(dbScheduler), LLLocalReactiveRocksIterator::release ).transform(LLUtils::handleDiscard), @@ -1799,7 +1800,7 @@ public class LLLocalDictionary implements LLDictionary { try { rocksIterator.status(); while (rocksIterator.isValid()) { - writeBatch.delete(cfh, LLUtils.readDirectNioBuffer(alloc, rocksIterator::key)); + writeBatch.delete(cfh, LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send()); rocksIterator.next(); rocksIterator.status(); } @@ -2123,8 +2124,8 @@ public class LLLocalDictionary implements LLDictionary { try { rocksIterator.status(); if (rocksIterator.isValid()) { - try (var key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).receive()) { - try (var value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value).receive()) { + try (var key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key)) { + try (var value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value)) { return LLEntry.of(key.send(), value.send()).send(); } } @@ -2184,7 +2185,7 @@ public class LLLocalDictionary implements LLDictionary { try { rocksIterator.status(); if (rocksIterator.isValid()) { - return LLUtils.readDirectNioBuffer(alloc, rocksIterator::key); + return LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send(); } else { return null; } @@ -2354,8 +2355,8 @@ public class LLLocalDictionary implements LLDictionary { if (!rocksIterator.isValid()) { return null; } - try (Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).receive()) { - try (Buffer value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value).receive()) { + try (Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key)) { + try (Buffer value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value)) { dbDelete(cfh, null, key.copy().send()); return LLEntry.of(key.send(), value.send()).send(); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalEntryReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalEntryReactiveRocksIterator.java index 4712fcc..60d82b7 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalEntryReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalEntryReactiveRocksIterator.java @@ -18,7 +18,7 @@ public class LLLocalEntryReactiveRocksIterator extends LLLocalReactiveRocksItera boolean allowNettyDirect, ReadOptions readOptions, String debugName) { - super(db, alloc, cfh, range, allowNettyDirect, readOptions, true, debugName); + super(db, alloc, cfh, range, allowNettyDirect, readOptions, true); } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java index 6b268c9..e2c82f5 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java @@ -7,6 +7,7 @@ import it.cavallium.dbengine.database.LLRange; import it.cavallium.dbengine.database.LLUtils; import it.unimi.dsi.fastutil.objects.ObjectArrayList; import java.util.List; +import org.jetbrains.annotations.Nullable; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; @@ -60,26 +61,28 @@ public abstract class LLLocalGroupedReactiveRocksIterator { try { rocksIterator.status(); while (rocksIterator.isValid()) { - try (Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).receive()) { + try (Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key)) { if (firstGroupKey == null) { firstGroupKey = key.copy(); } else if (!LLUtils.equals(firstGroupKey, firstGroupKey.readerOffset(), key, key.readerOffset(), prefixLength)) { break; } - Buffer value; + @Nullable Buffer value; if (readValues) { - value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value).receive(); + value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value); } else { - value = alloc.allocate(0); + value = null; } try { rocksIterator.next(); rocksIterator.status(); - T entry = getEntry(key.send(), value.send()); + T entry = getEntry(key.send(), value == null ? null : value.send()); values.add(entry); } finally { - value.close(); + if (value != null) { + value.close(); + } } } } @@ -106,7 +109,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator { }); } - public abstract T getEntry(Send key, Send value); + public abstract T getEntry(@Nullable Send key, @Nullable Send value); public void release() { range.close(); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java index 4ef635a..818882d 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java @@ -64,7 +64,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator { while (rocksIterator.isValid()) { Buffer key; if (allowNettyDirect) { - key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).receive(); + key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key); } else { key = LLUtils.fromByteArray(alloc, rocksIterator.key()); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyReactiveRocksIterator.java index dd3d32d..8e4b81f 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyReactiveRocksIterator.java @@ -15,9 +15,8 @@ public class LLLocalKeyReactiveRocksIterator extends LLLocalReactiveRocksIterato ColumnFamilyHandle cfh, Send range, boolean allowNettyDirect, - ReadOptions readOptions, - String debugName) { - super(db, alloc, cfh, range, allowNettyDirect, readOptions, false, debugName); + ReadOptions readOptions) { + super(db, alloc, cfh, range, allowNettyDirect, readOptions, false); } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java index d581a19..70428da 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java @@ -9,6 +9,7 @@ import io.net5.util.IllegalReferenceCountException; import it.cavallium.dbengine.database.LLRange; import it.cavallium.dbengine.database.LLUtils; import java.util.concurrent.atomic.AtomicBoolean; +import org.jetbrains.annotations.Nullable; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; @@ -25,7 +26,6 @@ public abstract class LLLocalReactiveRocksIterator { private final boolean allowNettyDirect; private final ReadOptions readOptions; private final boolean readValues; - private final String debugName; public LLLocalReactiveRocksIterator(RocksDB db, BufferAllocator alloc, @@ -33,8 +33,7 @@ public abstract class LLLocalReactiveRocksIterator { Send range, boolean allowNettyDirect, ReadOptions readOptions, - boolean readValues, - String debugName) { + boolean readValues) { this.db = db; this.alloc = alloc; this.cfh = cfh; @@ -42,7 +41,6 @@ public abstract class LLLocalReactiveRocksIterator { this.allowNettyDirect = allowNettyDirect; this.readOptions = readOptions; this.readValues = readValues; - this.debugName = debugName; } public Flux flux() { @@ -61,7 +59,7 @@ public abstract class LLLocalReactiveRocksIterator { if (rocksIterator.isValid()) { Buffer key; if (allowNettyDirect) { - key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).receive(); + key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key); } else { key = LLUtils.fromByteArray(alloc, rocksIterator.key()); } @@ -69,19 +67,21 @@ public abstract class LLLocalReactiveRocksIterator { Buffer value; if (readValues) { if (allowNettyDirect) { - value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value).receive(); + value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value); } else { value = LLUtils.fromByteArray(alloc, rocksIterator.value()); } } else { - value = alloc.allocate(0); + value = null; } try { rocksIterator.next(); rocksIterator.status(); - sink.next(getEntry(key.send(), value.send())); + sink.next(getEntry(key.send(), value == null ? null : value.send())); } finally { - value.close(); + if (value != null) { + value.close(); + } } } } else { @@ -100,7 +100,7 @@ public abstract class LLLocalReactiveRocksIterator { }); } - public abstract T getEntry(Send key, Send value); + public abstract T getEntry(@Nullable Send key, @Nullable Send value); public void release() { if (released.compareAndSet(false, true)) { diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/BufferDataInput.java b/src/main/java/it/cavallium/dbengine/database/serialization/BufferDataInput.java index 0f2a775..7c65b6d 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/BufferDataInput.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/BufferDataInput.java @@ -6,15 +6,17 @@ import it.cavallium.dbengine.database.SafeCloseable; import java.io.DataInput; import java.nio.charset.StandardCharsets; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class BufferDataInput implements DataInput, SafeCloseable { + @Nullable private final Buffer buf; private final int initialReaderOffset; - public BufferDataInput(Send bufferSend) { - this.buf = bufferSend.receive().makeReadOnly(); - this.initialReaderOffset = buf.readerOffset(); + public BufferDataInput(@Nullable Send bufferSend) { + this.buf = bufferSend == null ? null : bufferSend.receive().makeReadOnly(); + this.initialReaderOffset = buf == null ? 0 : buf.readerOffset(); } @Override @@ -24,75 +26,100 @@ public class BufferDataInput implements DataInput, SafeCloseable { @Override public void readFully(byte @NotNull [] b, int off, int len) { - buf.copyInto(buf.readerOffset(), b, off, len); - buf.readerOffset(buf.readerOffset() + len); + if (buf == null) { + if (len != 0) { + throw new IndexOutOfBoundsException(); + } + } else { + buf.copyInto(buf.readerOffset(), b, off, len); + buf.readerOffset(buf.readerOffset() + len); + } } @Override public int skipBytes(int n) { - n = Math.min(n, buf.readerOffset() - buf.writerOffset()); - buf.readerOffset(buf.readerOffset() + n); - return n; + if (buf == null) { + if (n != 0) { + throw new IndexOutOfBoundsException(); + } + return 0; + } else { + n = Math.min(n, buf.readerOffset() - buf.writerOffset()); + buf.readerOffset(buf.readerOffset() + n); + return n; + } } @Override public boolean readBoolean() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readUnsignedByte() != 0; } @Override public byte readByte() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readByte(); } @Override public int readUnsignedByte() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readUnsignedByte(); } @Override public short readShort() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readShort(); } @Override public int readUnsignedShort() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readUnsignedShort(); } @Override public char readChar() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readChar(); } @Override public int readInt() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readInt(); } @Override public long readLong() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readLong(); } @Override public float readFloat() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readFloat(); } @Override public double readDouble() { + if (buf == null) throw new IndexOutOfBoundsException(); return buf.readDouble(); } @Override public String readLine() { + if (buf == null) throw new IndexOutOfBoundsException(); throw new UnsupportedOperationException(); } @NotNull @Override public String readUTF() { + if (buf == null) throw new IndexOutOfBoundsException(); var len = buf.readUnsignedShort(); byte[] bytes = new byte[len]; buf.copyInto(buf.readerOffset(), bytes, 0, len); @@ -102,10 +129,16 @@ public class BufferDataInput implements DataInput, SafeCloseable { @Override public void close() { - buf.close(); + if (buf != null) { + buf.close(); + } } public int getReadBytesCount() { - return buf.readerOffset() - initialReaderOffset; + if (buf == null) { + return 0; + } else { + return buf.readerOffset() - initialReaderOffset; + } } } diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/CodecSerializer.java b/src/main/java/it/cavallium/dbengine/database/serialization/CodecSerializer.java index 62abfe9..a6b86bb 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/CodecSerializer.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/CodecSerializer.java @@ -6,6 +6,7 @@ import io.net5.buffer.api.Send; import java.io.IOError; import java.io.IOException; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.warp.commonutils.error.IndexOutOfBoundsException; public class CodecSerializer implements Serializer { @@ -37,7 +38,7 @@ public class CodecSerializer implements Serializer { } @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { try (var is = new BufferDataInput(serializedToReceive)) { int codecId; if (microCodecs) { diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java b/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java index 778a787..cd31dc2 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java @@ -5,22 +5,24 @@ import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLUtils; import java.nio.charset.StandardCharsets; +import java.util.Objects; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public interface Serializer { record DeserializationResult(T deserializedData, int bytesRead) {} - @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException; + @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException; - @NotNull Send serialize(@NotNull A deserialized) throws SerializationException; + @Nullable Send serialize(@NotNull A deserialized) throws SerializationException; Serializer> NOOP_SERIALIZER = new Serializer<>() { @Override - public @NotNull DeserializationResult> deserialize(@NotNull Send serialized) { - try (var serializedBuf = serialized.receive()) { - var readableBytes = serializedBuf.readableBytes(); - return new DeserializationResult<>(serializedBuf.send(), readableBytes); + public @NotNull DeserializationResult> deserialize(@Nullable Send serialized) { + try (var serializedBuf = serialized == null ? null : serialized.receive()) { + var readableBytes = serializedBuf == null ? 0 : serializedBuf.readableBytes(); + return new DeserializationResult<>(serializedBuf == null ? null : serializedBuf.send(), readableBytes); } } @@ -37,7 +39,8 @@ public interface Serializer { static Serializer utf8(BufferAllocator allocator) { return new Serializer<>() { @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { + Objects.requireNonNull(serializedToReceive); try (Buffer serialized = serializedToReceive.receive()) { assert serialized.isAccessible(); int length = serialized.readInt(); diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java b/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java index d93a942..d744aad 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java @@ -5,7 +5,9 @@ import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLUtils; import java.nio.charset.StandardCharsets; +import java.util.Objects; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; @SuppressWarnings("unused") public interface SerializerFixedBinaryLength extends Serializer { @@ -15,7 +17,11 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength> noop(int length) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult> deserialize(@NotNull Send serialized) { + public @NotNull DeserializationResult> deserialize(@Nullable Send serialized) { + if (length == 0 && serialized == null) { + return new DeserializationResult<>(null, 0); + } + Objects.requireNonNull(serialized); try (var buf = serialized.receive()) { if (buf.readableBytes() != getSerializedBinaryLength()) { throw new IllegalArgumentException( @@ -49,8 +55,12 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength utf8(BufferAllocator allocator, int length) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) throws SerializationException { + if (length == 0 && serializedToReceive == null) { + return new DeserializationResult<>(null, 0); + } + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { throw new SerializationException( @@ -89,7 +99,11 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength intSerializer(BufferAllocator allocator) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { + if (getSerializedBinaryLength() == 0 && serializedToReceive == null) { + return new DeserializationResult<>(null, 0); + } + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { throw new IllegalArgumentException( @@ -117,7 +131,11 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength longSerializer(BufferAllocator allocator) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { + if (getSerializedBinaryLength() == 0 && serializedToReceive == null) { + return new DeserializationResult<>(null, 0); + } + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { throw new IllegalArgumentException( diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index 2d4dd88..cff94bf 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -190,7 +190,6 @@ public class LuceneUtils { * * @return false if the result is not relevant */ - @Nullable public static boolean filterTopDoc(float score, Float minCompetitiveScore) { return minCompetitiveScore == null || score >= minCompetitiveScore; } @@ -327,7 +326,7 @@ public class LuceneUtils { assert i > 0 : "FileChannel.read with non zero-length bb.remaining() must always read at least one byte (FileChannel is in blocking mode, see spec of ReadableByteChannel)"; - pos += (long)i; + pos += i; } assert readLength == 0; diff --git a/src/test/java/it/cavallium/dbengine/DbTestUtils.java b/src/test/java/it/cavallium/dbengine/DbTestUtils.java index 79007fe..afff40e 100644 --- a/src/test/java/it/cavallium/dbengine/DbTestUtils.java +++ b/src/test/java/it/cavallium/dbengine/DbTestUtils.java @@ -25,8 +25,10 @@ import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.nio.file.Path; import java.util.Map; +import java.util.Objects; import java.util.function.Function; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.reactivestreams.Publisher; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -158,7 +160,8 @@ public class DbTestUtils { } @Override - public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { + public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { + Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { var val = serialized.readShort(); return new DeserializationResult<>(val, Short.BYTES); diff --git a/src/test/java/it/cavallium/dbengine/database/collections/TestRanges.java b/src/test/java/it/cavallium/dbengine/database/collections/TestRanges.java index a8c8190..28fb157 100644 --- a/src/test/java/it/cavallium/dbengine/database/collections/TestRanges.java +++ b/src/test/java/it/cavallium/dbengine/database/collections/TestRanges.java @@ -51,7 +51,7 @@ public class TestRanges { byte[] firstRangeKey; try (var firstRangeKeyBuf = DatabaseMapDictionaryDeep.firstRangeKey(alloc, alloc.allocate(prefixKey.length).writeBytes(prefixKey).send(), - prefixKey.length, 7, 3).receive()) { + prefixKey.length, 7, 3)) { firstRangeKey = LLUtils.toArray(firstRangeKeyBuf); } byte[] nextRangeKey; @@ -60,7 +60,7 @@ public class TestRanges { prefixKey.length, 7, 3 - ).receive()) { + )) { nextRangeKey = LLUtils.toArray(nextRangeKeyBuf); } @@ -114,7 +114,7 @@ public class TestRanges { prefixKey.length, 3, 7 - ).receive()) { + )) { firstRangeKey = LLUtils.toArray(firstRangeKeyBuf); } try (var nextRangeKeyBuf = DatabaseMapDictionaryDeep.nextRangeKey(alloc, @@ -123,7 +123,7 @@ public class TestRanges { prefixKey.length, 3, 7 - ).receive()) { + )) { byte[] nextRangeKey = LLUtils.toArray(nextRangeKeyBuf); if (Arrays.equals(prefixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}) && Arrays.equals(suffixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) { From 29086b1939afeb2d8b5c1078007d30f7590abf51 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 02:15:58 +0200 Subject: [PATCH 14/23] Add some tests for low level dictionary, log tests, pass more tests --- pom.xml | 12 +- .../dbengine/client/MappedSerializer.java | 4 +- .../client/MappedSerializerFixedLength.java | 4 +- .../cavallium/dbengine/database/LLEntry.java | 5 +- .../cavallium/dbengine/database/LLUtils.java | 103 +++--- .../database/collections/DatabaseEmpty.java | 6 +- .../collections/DatabaseMapDictionary.java | 103 +++--- .../DatabaseMapDictionaryDeep.java | 128 +++---- .../DatabaseMapDictionaryHashed.java | 3 +- .../collections/ValueWithHashSerializer.java | 13 +- .../collections/ValuesSetSerializer.java | 4 +- .../database/disk/LLLocalDictionary.java | 123 +++++-- .../database/disk/MemorySegmentUtils.java | 2 +- .../database/serialization/Serializer.java | 13 +- .../SerializerFixedBinaryLength.java | 25 +- .../dbengine/netty/NullableBuffer.java | 74 +++++ .../it/cavallium/dbengine/DbTestUtils.java | 33 ++ .../cavallium/dbengine/TestDictionaryMap.java | 103 ++++-- .../cavallium/dbengine/TestLLDictionary.java | 311 ++++++++++++++++++ .../dbengine/TestLLDictionaryLeaks.java | 63 +--- .../dbengine/TestLocalLLDictionary.java | 11 + .../dbengine/TestMemoryLLDictionary.java | 11 + src/test/resources/log4j2.xml | 20 ++ 23 files changed, 876 insertions(+), 298 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java create mode 100644 src/test/java/it/cavallium/dbengine/TestLLDictionary.java create mode 100644 src/test/java/it/cavallium/dbengine/TestLocalLLDictionary.java create mode 100644 src/test/java/it/cavallium/dbengine/TestMemoryLLDictionary.java create mode 100644 src/test/resources/log4j2.xml diff --git a/pom.xml b/pom.xml index ec68272..8aabc35 100644 --- a/pom.xml +++ b/pom.xml @@ -172,6 +172,11 @@ log4j-slf4j-impl test + + com.lmax + disruptor + 3.3.4 + org.rocksdb rocksdbjni @@ -352,6 +357,11 @@ log4j-slf4j-impl 2.14.1 + + com.lmax + disruptor + test + org.rocksdb rocksdbjni @@ -542,7 +552,7 @@ - --enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED --enable-native-access + --enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED --enable-native-access=ALL-UNNAMED ci diff --git a/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java b/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java index f29c5ff..5fcf3a4 100644 --- a/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java +++ b/src/main/java/it/cavallium/dbengine/client/MappedSerializer.java @@ -19,7 +19,7 @@ public class MappedSerializer implements Serializer { } @Override - public @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException { + public @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException { try (serialized) { var deserialized = serializer.deserialize(serialized); return new DeserializationResult<>(keyMapper.map(deserialized.deserializedData()), deserialized.bytesRead()); @@ -27,7 +27,7 @@ public class MappedSerializer implements Serializer { } @Override - public @Nullable Send serialize(@NotNull B deserialized) throws SerializationException { + public @NotNull Send serialize(@NotNull B deserialized) throws SerializationException { return serializer.serialize(keyMapper.unmap(deserialized)); } } diff --git a/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java b/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java index 2d970f8..e1caacb 100644 --- a/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java +++ b/src/main/java/it/cavallium/dbengine/client/MappedSerializerFixedLength.java @@ -19,7 +19,7 @@ public class MappedSerializerFixedLength implements SerializerFixedBinaryL } @Override - public @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException { + public @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException { try (serialized) { var deserialized = fixedLengthSerializer.deserialize(serialized); return new DeserializationResult<>(keyMapper.map(deserialized.deserializedData()), deserialized.bytesRead()); @@ -27,7 +27,7 @@ public class MappedSerializerFixedLength implements SerializerFixedBinaryL } @Override - public @Nullable Send serialize(@NotNull B deserialized) throws SerializationException { + public @NotNull Send serialize(@NotNull B deserialized) throws SerializationException { return fixedLengthSerializer.serialize(keyMapper.unmap(deserialized)); } diff --git a/src/main/java/it/cavallium/dbengine/database/LLEntry.java b/src/main/java/it/cavallium/dbengine/database/LLEntry.java index 71c0318..e93552a 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLEntry.java +++ b/src/main/java/it/cavallium/dbengine/database/LLEntry.java @@ -7,6 +7,7 @@ import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; import java.util.StringJoiner; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public class LLEntry extends ResourceSupport { @NotNull @@ -14,7 +15,7 @@ public class LLEntry extends ResourceSupport { @NotNull private final Buffer value; - private LLEntry(Send key, Send value, Drop drop) { + private LLEntry(@NotNull Send key, @NotNull Send value, Drop drop) { super(new LLEntry.CloseOnDrop(drop)); this.key = key.receive().makeReadOnly(); this.value = value.receive().makeReadOnly(); @@ -29,7 +30,7 @@ public class LLEntry extends ResourceSupport { return true; } - public static LLEntry of(Send key, Send value) { + public static LLEntry of(@NotNull Send key, @NotNull Send value) { return new LLEntry(key, value, d -> {}); } diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 1cd7699..c251e19 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -1,5 +1,7 @@ package it.cavallium.dbengine.database; +import static org.apache.commons.lang3.ArrayUtils.EMPTY_BYTE_ARRAY; + import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.net5.buffer.api.Buffer; @@ -10,30 +12,25 @@ import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; import io.net5.util.internal.PlatformDependent; import it.cavallium.dbengine.database.collections.DatabaseStage; -import it.cavallium.dbengine.database.disk.LLIndexSearcher; -import it.cavallium.dbengine.database.disk.LLLocalLuceneIndex; import it.cavallium.dbengine.database.disk.MemorySegmentUtils; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializationFunction; import it.cavallium.dbengine.lucene.RandomSortField; -import it.cavallium.dbengine.lucene.analyzer.WordAnalyzer; -import it.cavallium.dbengine.lucene.searcher.LocalQueryParams; import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.nio.charset.CharsetEncoder; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.function.ToIntFunction; -import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FloatPoint; @@ -42,24 +39,13 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; -import it.cavallium.dbengine.lucene.mlt.MultiMoreLikeThis; -import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConstantScoreQuery; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.MatchNoDocsQuery; -import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortedNumericSortField; -import org.apache.lucene.search.similarities.ClassicSimilarity; -import org.apache.lucene.search.similarities.Similarity; -import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.rocksdb.RocksDB; @@ -195,9 +181,9 @@ public class LLUtils { return new it.cavallium.dbengine.database.LLKeyScore(hit.docId(), hit.score(), hit.key()); } - public static String toStringSafe(Buffer key) { + public static String toStringSafe(@Nullable Buffer key) { try { - if (key.isAccessible()) { + if (key == null || key.isAccessible()) { return toString(key); } else { return "(released)"; @@ -207,7 +193,7 @@ public class LLUtils { } } - public static String toString(Buffer key) { + public static String toString(@Nullable Buffer key) { if (key == null) { return "null"; } else { @@ -217,20 +203,35 @@ public class LLUtils { if (iMax <= -1) { return "[]"; } else { - StringBuilder b = new StringBuilder(); - b.append('['); + StringBuilder arraySB = new StringBuilder(); + StringBuilder asciiSB = new StringBuilder(); + boolean isAscii = true; + arraySB.append('['); int i = 0; while (true) { - b.append(key.getByte(startIndex + i)); + var byteVal = key.getUnsignedByte(startIndex + i); + arraySB.append(byteVal); + if (isAscii) { + if (byteVal >= 32 && byteVal < 127) { + asciiSB.append((char) byteVal); + } else { + isAscii = false; + asciiSB = null; + } + } if (i == iLimit) { - b.append("…"); + arraySB.append("…"); } if (i == iMax || i == iLimit) { - return b.append(']').toString(); + if (isAscii) { + return asciiSB.insert(0, "\"").append("\"").toString(); + } else { + return arraySB.append(']').toString(); + } } - b.append(", "); + arraySB.append(", "); ++i; } } @@ -279,7 +280,10 @@ public class LLUtils { return true; } - public static byte[] toArray(Buffer key) { + public static byte[] toArray(@Nullable Buffer key) { + if (key == null) { + return EMPTY_BYTE_ARRAY; + } byte[] array = new byte[key.readableBytes()]; key.copyInto(key.readerOffset(), array, 0, key.readableBytes()); return array; @@ -355,7 +359,6 @@ public class LLUtils { PlatformDependent.freeDirectBuffer(directBuffer); directBuffer = null; } - buffer.close(); } } @@ -445,6 +448,22 @@ public class LLUtils { return true; } + public static Send empty(BufferAllocator allocator) { + try (var empty = CompositeBuffer.compose(allocator)) { + assert empty.readableBytes() == 0; + assert empty.capacity() == 0; + return empty.send(); + } + } + + public static Send copy(BufferAllocator allocator, Buffer buf) { + if (CompositeBuffer.isComposite(buf) && buf.capacity() == 0) { + return empty(allocator); + } else { + return buf.copy().send(); + } + } + public static record DirectBuffer(@NotNull Send buffer, @NotNull ByteBuffer byteBuffer) {} @NotNull @@ -485,18 +504,23 @@ public class LLUtils { ); } assert buffer.isAccessible(); + buffer.compact(); + assert buffer.readerOffset() == 0; AtomicLong nativeAddress = new AtomicLong(0); if (buffer.countComponents() == 1) { if (writable) { if (buffer.countWritableComponents() == 1) { buffer.forEachWritable(0, (i, c) -> { + assert c.writableNativeAddress() != 0; nativeAddress.setPlain(c.writableNativeAddress()); return false; }); } } else { - if (buffer.countReadableComponents() == 1) { + var readableComponents = buffer.countReadableComponents(); + if (readableComponents == 1) { buffer.forEachReadable(0, (i, c) -> { + assert c.readableNativeAddress() != 0; nativeAddress.setPlain(c.readableNativeAddress()); return false; }); @@ -512,7 +536,7 @@ public class LLUtils { } throw new IllegalStateException("Buffer is not direct"); } - return MemorySegmentUtils.directBuffer(nativeAddress.getPlain(), buffer.capacity()); + return MemorySegmentUtils.directBuffer(nativeAddress.getPlain(), writable ? buffer.capacity() : buffer.writerOffset()); } public static Buffer fromByteArray(BufferAllocator alloc, byte[] array) { @@ -534,22 +558,21 @@ public class LLUtils { return buffer.receive(); } - public static Buffer compositeBuffer(BufferAllocator alloc, Send buffer1, Send buffer2) { + @NotNull + public static Buffer compositeBuffer(BufferAllocator alloc, + @NotNull Send buffer1, + @NotNull Send buffer2) { return CompositeBuffer.compose(alloc, buffer1, buffer2); } + @NotNull public static Buffer compositeBuffer(BufferAllocator alloc, - Send buffer1, - Send buffer2, - Send buffer3) { + @NotNull Send buffer1, + @NotNull Send buffer2, + @NotNull Send buffer3) { return CompositeBuffer.compose(alloc, buffer1, buffer2, buffer3); } - @SafeVarargs - public static Buffer compositeBuffer(BufferAllocator alloc, Send... buffers) { - return CompositeBuffer.compose(alloc, buffers); - } - public static Mono resolveDelta(Mono> prev, UpdateReturnMode updateReturnMode) { return prev.handle((delta, sink) -> { switch (updateReturnMode) { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java index 0f11eb0..6769af6 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java @@ -2,8 +2,10 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; +import io.net5.buffer.api.CompositeBuffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.Serializer.DeserializationResult; import org.jetbrains.annotations.NotNull; @@ -25,8 +27,8 @@ public class DatabaseEmpty { } @Override - public @Nullable Send serialize(@NotNull Nothing deserialized) { - return null; + public @NotNull Send serialize(@NotNull Nothing deserialized) { + return LLUtils.empty(bufferAllocator); } }; } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java index 386326a..fd931c7 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -39,7 +40,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep valueSerializer; protected DatabaseMapDictionary(LLDictionary dictionary, - @Nullable Send prefixKey, + @NotNull Send prefixKey, SerializerFixedBinaryLength keySuffixSerializer, Serializer valueSerializer) { // Do not retain or release or use the prefixKey here @@ -50,7 +51,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep DatabaseMapDictionary simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, Serializer valueSerializer) { - return new DatabaseMapDictionary<>(dictionary, null, keySerializer, valueSerializer); + return new DatabaseMapDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, valueSerializer); } public static DatabaseMapDictionary tail(LLDictionary dictionary, @@ -60,13 +61,21 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep(dictionary, prefixKey, keySuffixSerializer, valueSerializer); } - private Buffer toKey(Send suffixKeyToSend) { - var suffixKey = suffixKeyToSend.receive(); - assert suffixKeyConsistency(suffixKey.readableBytes()); - if (keyPrefix != null) { - return LLUtils.compositeBuffer(dictionary.getAllocator(), keyPrefix.copy().send(), suffixKey.send()); - } else { - return suffixKey; + private Send toKey(Send suffixKeyToSend) { + try (var suffixKey = suffixKeyToSend.receive()) { + assert suffixKeyConsistency(suffixKey.readableBytes()); + if (keyPrefix.readableBytes() > 0) { + try (var result = LLUtils.compositeBuffer(dictionary.getAllocator(), + LLUtils.copy(dictionary.getAllocator(), keyPrefix), + suffixKey.send() + )) { + assert result.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + return result.send(); + } + } else { + assert suffixKey.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + return suffixKey.send(); + } } } @@ -84,7 +93,12 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep>handle((entrySend, sink) -> { try (var entry = entrySend.receive()) { - var key = deserializeSuffix(stripPrefix(entry.getKey())); + T key; + try (var serializedKey = entry.getKey().receive()) { + removePrefix(serializedKey); + suffixKeyConsistency(serializedKey.readableBytes()); + key = deserializeSuffix(serializedKey.send()); + } var value = valueSerializer.deserialize(entry.getValue()).deserializedData(); sink.next(Map.entry(key, value)); } catch (SerializationException ex) { @@ -103,7 +117,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep { try { - sink.next(LLEntry.of(this.toKey(serializeSuffix(entry.getKey()).send()).send(), + sink.next(LLEntry.of(this.toKey(serializeSuffix(entry.getKey())), valueSerializer.serialize(entry.getValue())).send()); } catch (SerializationException e) { sink.error(e); @@ -133,14 +147,14 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep> at(@Nullable CompositeSnapshot snapshot, T keySuffix) { return Mono.fromCallable(() -> - new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix).send()).send(), valueSerializer)); + new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix)), valueSerializer)); } @Override public Mono getValue(@Nullable CompositeSnapshot snapshot, T keySuffix, boolean existsAlmostCertainly) { return dictionary .get(resolveSnapshot(snapshot), - Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()), + Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))), existsAlmostCertainly ) .handle((value, sink) -> deserializeValue(value, sink)); @@ -148,8 +162,8 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep putValue(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); - var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))).single(); + var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)).single(); return dictionary .put(keyMono, valueMono, LLDictionaryResultType.VOID) .doOnNext(Send::close) @@ -166,7 +180,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep updater) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); return dictionary .update(keyMono, getSerializedUpdater(updater), updateReturnMode, existsAlmostCertainly) .handle((value, sink) -> deserializeValue(value, sink)); @@ -176,7 +190,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep> updateValueAndGetDelta(T keySuffix, boolean existsAlmostCertainly, SerializationFunction<@Nullable U, @Nullable U> updater) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); return dictionary .updateAndGetDelta(keyMono, getSerializedUpdater(updater), existsAlmostCertainly) .transform(mono -> LLUtils.mapLLDelta(mono, @@ -224,7 +238,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep putValueAndGetPrevious(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); return dictionary .put(keyMono, @@ -235,7 +249,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep putValueAndGetChanged(T keySuffix, U value) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); var valueMono = Mono.fromCallable(() -> valueSerializer.serialize(value)); return dictionary .put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE) @@ -246,7 +260,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep remove(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); return dictionary .remove(keyMono, LLDictionaryResultType.VOID) .doOnNext(Send::close) @@ -255,7 +269,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep removeAndGetPrevious(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); return dictionary .remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE) .handle((value, sink) -> deserializeValue(value, sink)); @@ -263,7 +277,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep removeAndGetStatus(T keySuffix) { - var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix).send()).send()); + var keyMono = Mono.fromCallable(() -> toKey(serializeSuffix(keySuffix))); return dictionary .remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) .map(LLUtils::responseToBoolean); @@ -274,7 +288,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep>>handle((keySuffix, sink) -> { try { - sink.next(Tuples.of(keySuffix, toKey(serializeSuffix(keySuffix).send()).send())); + sink.next(Tuples.of(keySuffix, toKey(serializeSuffix(keySuffix)))); } catch (SerializationException ex) { sink.error(ex); } @@ -303,9 +317,10 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep serializeEntry(T key, U value) throws SerializationException { - try (var serializedKey = toKey(serializeSuffix(key).send())) { - try (var serializedValue = valueSerializer.serialize(value).receive()) { - return LLEntry.of(serializedKey.send(), serializedValue.send()).send(); + try (var serializedKey = toKey(serializeSuffix(key))) { + var serializedValueToReceive = valueSerializer.serialize(value); + try (var serializedValue = serializedValueToReceive.receive()) { + return LLEntry.of(serializedKey, serializedValue.send()).send(); } } } @@ -343,7 +358,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep, X>>handle((entry, sink) -> { try { - sink.next(Tuples.of(serializeSuffix(entry.getT1()).send(), entry.getT2())); + sink.next(Tuples.of(serializeSuffix(entry.getT1()), entry.getT2())); } catch (SerializationException ex) { sink.error(ex); } @@ -374,16 +389,18 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep>> getAllStages(@Nullable CompositeSnapshot snapshot) { return dictionary .getRangeKeys(resolveSnapshot(snapshot), rangeMono) - .handle((key, sink) -> { - try (key) { - try (var keySuffixWithExt = stripPrefix(key).receive()) { - sink.next(Map.entry(deserializeSuffix(keySuffixWithExt.copy().send()), - new DatabaseSingle<>(dictionary, - toKey(keySuffixWithExt.send()).send(), - valueSerializer - ) - )); - } + .handle((keyBufToReceive, sink) -> { + try (var keyBuf = keyBufToReceive.receive()) { + assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + // Remove prefix. Keep only the suffix and the ext + removePrefix(keyBuf); + suffixKeyConsistency(keyBuf.readableBytes()); + sink.next(Map.entry(deserializeSuffix(keyBuf.copy().send()), + new DatabaseSingle<>(dictionary, + toKey(keyBuf.send()), + valueSerializer + ) + )); } catch (SerializationException ex) { sink.error(ex); } @@ -396,8 +413,14 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep>handle((serializedEntryToReceive, sink) -> { try (var serializedEntry = serializedEntryToReceive.receive()) { - sink.next(Map.entry(deserializeSuffix(stripPrefix(serializedEntry.getKey())), - valueSerializer.deserialize(serializedEntry.getValue()).deserializedData())); + try (var keyBuf = serializedEntry.getKey().receive()) { + assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + // Remove prefix. Keep only the suffix and the ext + removePrefix(keyBuf); + suffixKeyConsistency(keyBuf.readableBytes()); + sink.next(Map.entry(deserializeSuffix(keyBuf.send()), + valueSerializer.deserialize(serializedEntry.getValue()).deserializedData())); + } } catch (SerializationException e) { sink.error(e); } @@ -418,7 +441,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep { try { - sink.next(LLEntry.of(toKey(serializeSuffix(entry.getKey()).send()).send(), + sink.next(LLEntry.of(toKey(serializeSuffix(entry.getKey())), valueSerializer.serialize(entry.getValue())).send()); } catch (SerializationException e) { sink.error(e); diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java index 7dc589a..2dff876 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java @@ -2,7 +2,6 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; -import io.net5.buffer.api.CompositeBuffer; import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; import io.net5.util.IllegalReferenceCountException; @@ -18,6 +17,8 @@ import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -29,6 +30,7 @@ public class DatabaseMapDictionaryDeep> implem private final BufferAllocator alloc; protected final SubStageGetter subStageGetter; protected final SerializerFixedBinaryLength keySuffixSerializer; + @NotNull protected final Buffer keyPrefix; protected final int keyPrefixLength; protected final int keySuffixLength; @@ -90,27 +92,26 @@ public class DatabaseMapDictionaryDeep> implem } protected static Buffer zeroFillKeySuffixAndExt(BufferAllocator alloc, - @Nullable Send prefixKeySend, + @NotNull Send prefixKeySend, int prefixLength, int suffixLength, int extLength) { - try (var result = prefixKeySend == null ? null : prefixKeySend.receive()) { - if (result == null) { - assert prefixLength == 0; - var buf = alloc.allocate(prefixLength + suffixLength + extLength); - buf.writerOffset(prefixLength + suffixLength + extLength); - buf.fill((byte) 0); - return buf; - } else { - assert result.readableBytes() == prefixLength; - assert suffixLength > 0; - assert extLength >= 0; - result.ensureWritable(suffixLength + extLength, suffixLength + extLength, true); - for (int i = 0; i < suffixLength + extLength; i++) { - result.writeByte((byte) 0x0); - } - return result; + var result = prefixKeySend.receive(); + if (result == null) { + assert prefixLength == 0; + var buf = alloc.allocate(prefixLength + suffixLength + extLength); + buf.writerOffset(prefixLength + suffixLength + extLength); + buf.fill((byte) 0); + return buf; + } else { + assert result.readableBytes() == prefixLength; + assert suffixLength > 0; + assert extLength >= 0; + result.ensureWritable(suffixLength + extLength, suffixLength + extLength, true); + for (int i = 0; i < suffixLength + extLength; i++) { + result.writeByte((byte) 0x0); } + return result; } } @@ -175,7 +176,7 @@ public class DatabaseMapDictionaryDeep> implem public static DatabaseMapDictionaryDeep> simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, SubStageGetterSingle subStageGetter) { - return new DatabaseMapDictionaryDeep<>(dictionary, null, keySerializer, subStageGetter, 0); + return new DatabaseMapDictionaryDeep<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, subStageGetter, 0); } public static > DatabaseMapDictionaryDeep deepTail(LLDictionary dictionary, @@ -183,7 +184,7 @@ public class DatabaseMapDictionaryDeep> implem int keyExtLength, SubStageGetter subStageGetter) { return new DatabaseMapDictionaryDeep<>(dictionary, - null, + LLUtils.empty(dictionary.getAllocator()), keySerializer, subStageGetter, keyExtLength @@ -199,26 +200,26 @@ public class DatabaseMapDictionaryDeep> implem } protected DatabaseMapDictionaryDeep(LLDictionary dictionary, - @Nullable Send prefixKeyToReceive, + @NotNull Send prefixKeyToReceive, SerializerFixedBinaryLength keySuffixSerializer, SubStageGetter subStageGetter, int keyExtLength) { - try (var prefixKey = prefixKeyToReceive == null ? null : prefixKeyToReceive.receive()) { + try (var prefixKey = prefixKeyToReceive.receive()) { this.dictionary = dictionary; this.alloc = dictionary.getAllocator(); this.subStageGetter = subStageGetter; this.keySuffixSerializer = keySuffixSerializer; - assert prefixKey == null || prefixKey.isAccessible(); - this.keyPrefixLength = prefixKey == null ? 0 : prefixKey.readableBytes(); + assert prefixKey.isAccessible(); + this.keyPrefixLength = prefixKey.readableBytes(); this.keySuffixLength = keySuffixSerializer.getSerializedBinaryLength(); this.keyExtLength = keyExtLength; - Buffer firstKey = firstRangeKey(alloc, prefixKey == null ? null : prefixKey.copy().send(), keyPrefixLength, + Buffer firstKey = firstRangeKey(alloc, LLUtils.copy(alloc, prefixKey), keyPrefixLength, keySuffixLength, keyExtLength); try (firstKey) { - var nextRangeKey = nextRangeKey(alloc, prefixKey == null ? null : prefixKey.copy().send(), + var nextRangeKey = nextRangeKey(alloc, LLUtils.copy(alloc, prefixKey), keyPrefixLength, keySuffixLength, keyExtLength); try (nextRangeKey) { - assert prefixKey == null || prefixKey.isAccessible(); + assert prefixKey.isAccessible(); assert keyPrefixLength == 0 || !LLUtils.equals(firstKey, nextRangeKey); this.range = keyPrefixLength == 0 ? LLRange.all() : LLRange.of(firstKey.send(), nextRangeKey.send()); this.rangeMono = LLUtils.lazyRetainRange(this.range); @@ -226,7 +227,7 @@ public class DatabaseMapDictionaryDeep> implem } } - this.keyPrefix = prefixKey == null ? null : prefixKey.send().receive(); + this.keyPrefix = prefixKey.send().receive(); } } @@ -246,21 +247,28 @@ public class DatabaseMapDictionaryDeep> implem } /** - * Keep only suffix and ext + * Removes the prefix from the key */ - protected Send stripPrefix(Send keyToReceive) { - try (var key = keyToReceive.receive()) { - return key.copy(this.keyPrefixLength, key.readableBytes() - this.keyPrefixLength).send(); - } + protected void removePrefix(Buffer key) { + assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + key.readerOffset(key.readerOffset() + this.keyPrefixLength).compact(); + assert key.readableBytes() == keySuffixLength + keyExtLength; } /** - * Add prefix to suffix + * Removes the ext from the key */ + protected void removeExt(Buffer key) { + assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + key.writerOffset(keyPrefixLength + keySuffixLength).compact(); + assert key.readableBytes() == keyPrefixLength + keySuffixLength; + } + protected Send toKeyWithoutExt(Send suffixKeyToReceive) { try (var suffixKey = suffixKeyToReceive.receive()) { assert suffixKey.readableBytes() == keySuffixLength; - try (Buffer result = LLUtils.compositeBuffer(alloc, keyPrefix.copy().send(), suffixKey.send())) { + try (var result = Objects.requireNonNull(LLUtils.compositeBuffer(alloc, + LLUtils.copy(alloc, keyPrefix), suffixKey.send()))) { assert result.readableBytes() == keyPrefixLength + keySuffixLength; return result.send(); } @@ -287,8 +295,9 @@ public class DatabaseMapDictionaryDeep> implem @Override public Mono at(@Nullable CompositeSnapshot snapshot, T keySuffix) { + var suffixKeyWithoutExt = Mono.fromCallable(() -> toKeyWithoutExt(serializeSuffix(keySuffix))); return this.subStageGetter - .subStage(dictionary, snapshot, Mono.fromCallable(() -> toKeyWithoutExt(serializeSuffix(keySuffix).send()))) + .subStage(dictionary, snapshot, suffixKeyWithoutExt) .transform(LLUtils::handleDiscard) .doOnDiscard(DatabaseStage.class, DatabaseStage::release); } @@ -310,11 +319,10 @@ public class DatabaseMapDictionaryDeep> implem .flatMapSequential(groupKeyWithoutExtSend_ -> Mono.using( groupKeyWithoutExtSend_::receive, groupKeyWithoutExtSend -> this.subStageGetter - .subStage(dictionary, snapshot, getGroupKeyWithoutExt(groupKeyWithoutExtSend.copy().send())) + .subStage(dictionary, snapshot, Mono.fromCallable(() -> groupKeyWithoutExtSend.copy().send())) .>handle((us, sink) -> { try { - sink.next(Map.entry(this.deserializeSuffix(getGroupSuffix(groupKeyWithoutExtSend.send())), - us)); + sink.next(Map.entry(this.deserializeSuffix(getGroupSuffix(groupKeyWithoutExtSend.send())), us)); } catch (SerializationException ex) { sink.error(ex); } @@ -324,22 +332,22 @@ public class DatabaseMapDictionaryDeep> implem .transform(LLUtils::handleDiscard); } - private Send getGroupSuffix(Send groupKeyWithoutExtSend) { - try (var groupKeyWithoutExt = groupKeyWithoutExtSend.receive()) { - try (var groupSuffix = this.stripPrefix(groupKeyWithoutExt.copy().send()).receive()) { - assert subStageKeysConsistency(groupKeyWithoutExt.readableBytes() + keyExtLength); - return groupSuffix.send(); - } + private Send getGroupSuffix(Send groupKeyWithoutExt) { + try (var buffer = groupKeyWithoutExt.receive()) { + assert subStageKeysConsistency(buffer.readableBytes() + keyExtLength); + this.removePrefix(buffer); + assert subStageKeysConsistency(keyPrefixLength + buffer.readableBytes() + keyExtLength); + return buffer.send(); } } - private Mono> getGroupKeyWithoutExt(Send groupKeyWithoutExtSend) { - return Mono.fromCallable(() -> { - try (var groupKeyWithoutExt = groupKeyWithoutExtSend.receive()) { - assert subStageKeysConsistency(groupKeyWithoutExt.readableBytes() + keyExtLength); - return groupKeyWithoutExt.send(); - } - }); + private Send getGroupWithoutExt(Send groupKeyWithExtSend) { + try (var buffer = groupKeyWithExtSend.receive()) { + assert subStageKeysConsistency(buffer.readableBytes()); + this.removeExt(buffer); + assert subStageKeysConsistency(buffer.readableBytes() + keyExtLength); + return buffer.send(); + } } private boolean subStageKeysConsistency(int totalKeyLength) { @@ -383,7 +391,7 @@ public class DatabaseMapDictionaryDeep> implem } //todo: temporary wrapper. convert the whole class to buffers - protected T deserializeSuffix(Send keySuffixToReceive) throws SerializationException { + protected T deserializeSuffix(@NotNull Send keySuffixToReceive) throws SerializationException { try (var keySuffix = keySuffixToReceive.receive()) { assert suffixKeyConsistency(keySuffix.readableBytes()); var result = keySuffixSerializer.deserialize(keySuffix.send()); @@ -393,11 +401,15 @@ public class DatabaseMapDictionaryDeep> implem } //todo: temporary wrapper. convert the whole class to buffers - protected Buffer serializeSuffix(T keySuffix) throws SerializationException { - Buffer suffixData = keySuffixSerializer.serialize(keySuffix).receive(); - assert suffixKeyConsistency(suffixData.readableBytes()); - assert keyPrefix.isAccessible(); - return suffixData; + @NotNull + protected Send serializeSuffix(T keySuffix) throws SerializationException { + try (var suffixDataToReceive = keySuffixSerializer.serialize(keySuffix)) { + try (Buffer suffixData = suffixDataToReceive.receive()) { + assert suffixKeyConsistency(suffixData.readableBytes()); + assert keyPrefix.isAccessible(); + return suffixData.send(); + } + } } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java index 90f6e34..653cc35 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java @@ -6,6 +6,7 @@ import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.UpdateMode; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; @@ -58,7 +59,7 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap keyHashSerializer) { return new DatabaseMapDictionaryHashed<>( dictionary, - null, + LLUtils.empty(dictionary.getAllocator()), keySerializer, valueSerializer, keyHashFunction, diff --git a/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java b/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java index f380b73..549852e 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/ValueWithHashSerializer.java @@ -2,6 +2,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; +import io.net5.buffer.api.CompositeBuffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.serialization.SerializationException; @@ -43,17 +44,9 @@ class ValueWithHashSerializer implements Serializer> { } @Override - public @Nullable Send serialize(@NotNull Entry deserialized) throws SerializationException { + public @NotNull Send serialize(@NotNull Entry deserialized) throws SerializationException { var keySuffix = keySuffixSerializer.serialize(deserialized.getKey()); var value = valueSerializer.serialize(deserialized.getValue()); - if (value == null && keySuffix == null) { - return null; - } else if (value == null) { - return keySuffix; - } else if (keySuffix == null) { - return value; - } else { - return LLUtils.compositeBuffer(allocator, keySuffix, value).send(); - } + return LLUtils.compositeBuffer(allocator, keySuffix, value).send(); } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java b/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java index 749ba0f..330ba68 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/ValuesSetSerializer.java @@ -45,8 +45,8 @@ class ValuesSetSerializer implements Serializer> { output.writeInt(deserialized.size()); for (X entry : deserialized) { var serializedToReceive = entrySerializer.serialize(entry); - if (serializedToReceive != null) { - try (Buffer serialized = serializedToReceive.receive()) { + try (Buffer serialized = serializedToReceive.receive()) { + if (serialized.readableBytes() > 0) { output.ensureWritable(serialized.readableBytes()); output.writeBytes(serialized); } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java index f4165da..8c53e41 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java @@ -267,10 +267,23 @@ public class LLLocalDictionary implements LLDictionary { stamp = 0; } try { + Buffer logKey; if (logger.isTraceEnabled(MARKER_ROCKSDB)) { - logger.trace(MARKER_ROCKSDB, "Reading {}", LLUtils.toStringSafe(key)); + logKey = key.copy(); + } else { + logKey = null; + } + try (logKey) { + var result = dbGet(cfh, resolveSnapshot(snapshot), key.send(), existsAlmostCertainly); + if (logger.isTraceEnabled(MARKER_ROCKSDB)) { + try (var result2 = result == null ? null : result.receive()) { + logger.trace(MARKER_ROCKSDB, "Reading {}: {}", LLUtils.toStringSafe(logKey), LLUtils.toString(result2)); + return result2 == null ? null : result2.send(); + } + } else { + return result; + } } - return dbGet(cfh, resolveSnapshot(snapshot), key.send(), existsAlmostCertainly); } finally { if (updateMode == UpdateMode.ALLOW) { lock.unlockRead(stamp); @@ -414,6 +427,8 @@ public class LLLocalDictionary implements LLDictionary { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called dbPut in a nonblocking thread"); } + assert key.isAccessible(); + assert value.isAccessible(); if (databaseOptions.allowNettyDirect()) { var keyNioBuffer = LLUtils.convertToReadableDirect(alloc, key.send()); try (var ignored1 = keyNioBuffer.buffer().receive()) { @@ -592,6 +607,8 @@ public class LLLocalDictionary implements LLDictionary { valueSend -> this.>runOnDb(() -> { try (var key = keySend.receive()) { try (var value = valueSend.receive()) { + assert key.isAccessible(); + assert value.isAccessible(); StampedLock lock; long stamp; if (updateMode == UpdateMode.ALLOW) { @@ -656,9 +673,6 @@ public class LLLocalDictionary implements LLDictionary { stamp = 0; } try { - if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Reading {}", LLUtils.toStringSafe(key)); - } while (true) { @Nullable Buffer prevData; var prevDataHolder = existsAlmostCertainly ? null : new Holder(); @@ -682,19 +696,37 @@ public class LLLocalDictionary implements LLDictionary { } else { prevData = null; } + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Reading {}: {} (before update)", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(prevData) + ); + } try { @Nullable Buffer newData; try (Buffer prevDataToSendToUpdater = prevData == null ? null : prevData.copy()) { - try (var newDataToReceive = updater.apply( - prevDataToSendToUpdater == null ? null : prevDataToSendToUpdater.send())) { - if (newDataToReceive != null) { - newData = newDataToReceive.receive(); - } else { - newData = null; + try (var sentData = prevDataToSendToUpdater == null ? null + : prevDataToSendToUpdater.send()) { + try (var newDataToReceive = updater.apply(sentData)) { + if (newDataToReceive != null) { + newData = newDataToReceive.receive(); + } else { + newData = null; + } } } } + assert newData == null || newData.isAccessible(); try { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Updating {}. previous data: {}, updated data: {}", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(prevData), + LLUtils.toStringSafe(newData) + ); + } if (prevData != null && newData == null) { //noinspection DuplicatedCode if (updateMode == UpdateMode.ALLOW) { @@ -709,7 +741,7 @@ public class LLLocalDictionary implements LLDictionary { } } if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Deleting {}", LLUtils.toStringSafe(key)); + logger.trace(MARKER_ROCKSDB, "Deleting {} (after update)", LLUtils.toStringSafe(key)); } dbDelete(cfh, null, key.send()); } else if (newData != null @@ -727,7 +759,11 @@ public class LLLocalDictionary implements LLDictionary { } } if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Writing {}: {}", LLUtils.toStringSafe(key), LLUtils.toStringSafe(newData)); + logger.trace(MARKER_ROCKSDB, + "Writing {}: {} (after update)", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(newData) + ); } Buffer dataToPut; if (updateReturnMode == UpdateReturnMode.GET_NEW_VALUE) { @@ -779,7 +815,7 @@ public class LLLocalDictionary implements LLDictionary { SerializationFunction<@Nullable Send, @Nullable Send> updater, boolean existsAlmostCertainly) { return Mono.usingWhen(keyMono, - keySend -> this.runOnDb(() -> { + keySend -> runOnDb(() -> { try (var key = keySend.receive()) { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called update in a nonblocking thread"); @@ -799,7 +835,7 @@ public class LLLocalDictionary implements LLDictionary { } try { if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Reading {}", LLUtils.toStringSafe(key)); + logger.trace(MARKER_ROCKSDB, "Reading {} (before update)", LLUtils.toStringSafe(key)); } while (true) { @Nullable Buffer prevData; @@ -824,19 +860,37 @@ public class LLLocalDictionary implements LLDictionary { } else { prevData = null; } + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Read {}: {} (before update)", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(prevData) + ); + } try { @Nullable Buffer newData; try (Buffer prevDataToSendToUpdater = prevData == null ? null : prevData.copy()) { - try (var newDataToReceive = updater.apply( - prevDataToSendToUpdater == null ? null : prevDataToSendToUpdater.send())) { - if (newDataToReceive != null) { - newData = newDataToReceive.receive(); - } else { - newData = null; + try (var sentData = prevDataToSendToUpdater == null ? null + : prevDataToSendToUpdater.send()) { + try (var newDataToReceive = updater.apply(sentData)) { + if (newDataToReceive != null) { + newData = newDataToReceive.receive(); + } else { + newData = null; + } } } } + assert newData == null || newData.isAccessible(); try { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Updating {}. previous data: {}, updated data: {}", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(prevData), + LLUtils.toStringSafe(newData) + ); + } if (prevData != null && newData == null) { //noinspection DuplicatedCode if (updateMode == UpdateMode.ALLOW) { @@ -851,7 +905,7 @@ public class LLLocalDictionary implements LLDictionary { } } if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Deleting {}", LLUtils.toStringSafe(key)); + logger.trace(MARKER_ROCKSDB, "Deleting {} (after update)", LLUtils.toStringSafe(key)); } dbDelete(cfh, null, key.send()); } else if (newData != null @@ -869,8 +923,11 @@ public class LLLocalDictionary implements LLDictionary { } } if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Writing {}: {}", - LLUtils.toStringSafe(key), LLUtils.toStringSafe(newData)); + logger.trace(MARKER_ROCKSDB, + "Writing {}: {} (after update)", + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(newData) + ); } assert key.isAccessible(); assert newData.isAccessible(); @@ -986,18 +1043,24 @@ public class LLLocalDictionary implements LLDictionary { stamp = 0; } try { - if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Reading {}", LLUtils.toArray(key)); - } var data = new Holder(); + Buffer bufferResult; if (db.keyMayExist(cfh, LLUtils.toArray(key), data)) { if (data.getValue() != null) { - return LLUtils.fromByteArray(alloc, data.getValue()).send(); + bufferResult = LLUtils.fromByteArray(alloc, data.getValue()); } else { - return dbGet(cfh, null, key.send(), true); + try (var bufferResultToReceive = dbGet(cfh, null, key.send(), true)) { + bufferResult = bufferResultToReceive == null ? null : bufferResultToReceive.receive(); + } } } else { - return null; + bufferResult = null; + } + try (bufferResult) { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Reading {}: {}", LLUtils.toStringSafe(key), LLUtils.toStringSafe(bufferResult)); + } + return bufferResult == null ? null : bufferResult.send(); } } finally { if (updateMode == UpdateMode.ALLOW) { diff --git a/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java b/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java index 608404e..c283149 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/MemorySegmentUtils.java @@ -124,6 +124,6 @@ public class MemorySegmentUtils { } public static String getSuggestedArgs() { - return "--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --enable-native-access"; + return "--enable-preview --add-modules jdk.incubator.foreign -Dforeign.restricted=permit --enable-native-access=ALL-UNNAMED"; } } diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java b/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java index cd31dc2..4ffb437 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/Serializer.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.netty.NullableBuffer; import java.nio.charset.StandardCharsets; import java.util.Objects; import org.jetbrains.annotations.NotNull; @@ -13,16 +14,16 @@ public interface Serializer { record DeserializationResult(T deserializedData, int bytesRead) {} - @NotNull DeserializationResult deserialize(@Nullable Send serialized) throws SerializationException; + @NotNull DeserializationResult deserialize(@NotNull Send serialized) throws SerializationException; - @Nullable Send serialize(@NotNull A deserialized) throws SerializationException; + @NotNull Send serialize(@NotNull A deserialized) throws SerializationException; Serializer> NOOP_SERIALIZER = new Serializer<>() { @Override - public @NotNull DeserializationResult> deserialize(@Nullable Send serialized) { - try (var serializedBuf = serialized == null ? null : serialized.receive()) { - var readableBytes = serializedBuf == null ? 0 : serializedBuf.readableBytes(); - return new DeserializationResult<>(serializedBuf == null ? null : serializedBuf.send(), readableBytes); + public @NotNull DeserializationResult> deserialize(@NotNull Send serialized) { + try (var serializedBuf = serialized.receive()) { + var readableBytes = serializedBuf.readableBytes(); + return new DeserializationResult<>(serializedBuf.send(), readableBytes); } } diff --git a/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java b/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java index d744aad..78fe656 100644 --- a/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java +++ b/src/main/java/it/cavallium/dbengine/database/serialization/SerializerFixedBinaryLength.java @@ -17,10 +17,7 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength> noop(int length) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult> deserialize(@Nullable Send serialized) { - if (length == 0 && serialized == null) { - return new DeserializationResult<>(null, 0); - } + public @NotNull DeserializationResult> deserialize(@NotNull Send serialized) { Objects.requireNonNull(serialized); try (var buf = serialized.receive()) { if (buf.readableBytes() != getSerializedBinaryLength()) { @@ -55,12 +52,8 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength utf8(BufferAllocator allocator, int length) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) + public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) throws SerializationException { - if (length == 0 && serializedToReceive == null) { - return new DeserializationResult<>(null, 0); - } - Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { throw new SerializationException( @@ -78,7 +71,9 @@ public interface SerializerFixedBinaryLength extends Serializer { // UTF-8 uses max. 3 bytes per char, so calculate the worst case. try (Buffer buf = allocator.allocate(LLUtils.utf8MaxBytes(deserialized))) { assert buf.isAccessible(); - buf.writeBytes(deserialized.getBytes(StandardCharsets.UTF_8)); + var bytes = deserialized.getBytes(StandardCharsets.UTF_8); + buf.ensureWritable(bytes.length); + buf.writeBytes(bytes); if (buf.readableBytes() != getSerializedBinaryLength()) { throw new SerializationException("Fixed serializer with " + getSerializedBinaryLength() + " bytes has tried to serialize an element with " @@ -99,10 +94,7 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength intSerializer(BufferAllocator allocator) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { - if (getSerializedBinaryLength() == 0 && serializedToReceive == null) { - return new DeserializationResult<>(null, 0); - } + public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { @@ -131,10 +123,7 @@ public interface SerializerFixedBinaryLength extends Serializer { static SerializerFixedBinaryLength longSerializer(BufferAllocator allocator) { return new SerializerFixedBinaryLength<>() { @Override - public @NotNull DeserializationResult deserialize(@Nullable Send serializedToReceive) { - if (getSerializedBinaryLength() == 0 && serializedToReceive == null) { - return new DeserializationResult<>(null, 0); - } + public @NotNull DeserializationResult deserialize(@NotNull Send serializedToReceive) { Objects.requireNonNull(serializedToReceive); try (var serialized = serializedToReceive.receive()) { if (serialized.readableBytes() != getSerializedBinaryLength()) { diff --git a/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java new file mode 100644 index 0000000..c1e3eef --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java @@ -0,0 +1,74 @@ +package it.cavallium.dbengine.netty; + +import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.client.SearchResult; +import org.jetbrains.annotations.Nullable; + +public class NullableBuffer extends ResourceSupport { + + @Nullable + private Buffer buffer; + + public NullableBuffer(@Nullable Buffer buffer, Drop drop) { + super(new CloseOnDrop(drop)); + this.buffer = buffer == null ? null : buffer.send().receive(); + } + + public NullableBuffer(@Nullable Send buffer, Drop drop) { + super(new CloseOnDrop(drop)); + this.buffer = buffer == null ? null : buffer.receive(); + } + + @Nullable + public Buffer buf() { + return buffer; + } + + @Nullable + public Send sendBuf() { + return buffer == null ? null : buffer.send(); + } + + @Override + protected RuntimeException createResourceClosedException() { + return new IllegalStateException("Closed"); + } + + @Override + protected Owned prepareSend() { + var buffer = this.buffer == null ? null : this.buffer.send(); + makeInaccessible(); + return drop -> new NullableBuffer(buffer, drop); + } + + private void makeInaccessible() { + this.buffer = null; + } + + private static class CloseOnDrop implements Drop { + + private final Drop delegate; + + public CloseOnDrop(Drop drop) { + this.delegate = drop; + } + + @Override + public void drop(NullableBuffer obj) { + try { + if (obj.buffer != null) { + if (obj.buffer.isAccessible()) { + obj.buffer.close(); + } + } + delegate.drop(obj); + } finally { + obj.makeInaccessible(); + } + } + } +} diff --git a/src/test/java/it/cavallium/dbengine/DbTestUtils.java b/src/test/java/it/cavallium/dbengine/DbTestUtils.java index afff40e..c16b139 100644 --- a/src/test/java/it/cavallium/dbengine/DbTestUtils.java +++ b/src/test/java/it/cavallium/dbengine/DbTestUtils.java @@ -32,6 +32,7 @@ import org.jetbrains.annotations.Nullable; import org.reactivestreams.Publisher; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; public class DbTestUtils { @@ -42,6 +43,38 @@ public class DbTestUtils { return "0123456789".repeat(1024); } + public static void run(Flux publisher) { + publisher.subscribeOn(Schedulers.immediate()).blockLast(); + } + + public static void runVoid(Mono publisher) { + publisher.then().subscribeOn(Schedulers.immediate()).block(); + } + + public static T run(Mono publisher) { + return publisher.subscribeOn(Schedulers.immediate()).block(); + } + + public static T run(boolean shouldFail, Mono publisher) { + return publisher.subscribeOn(Schedulers.immediate()).transform(mono -> { + if (shouldFail) { + return mono.onErrorResume(ex -> Mono.empty()); + } else { + return mono; + } + }).block(); + } + + public static void runVoid(boolean shouldFail, Mono publisher) { + publisher.then().subscribeOn(Schedulers.immediate()).transform(mono -> { + if (shouldFail) { + return mono.onErrorResume(ex -> Mono.empty()); + } else { + return mono; + } + }).block(); + } + public static record TestAllocator(PooledBufferAllocator allocator) {} public static TestAllocator newAllocator() { diff --git a/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java b/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java index f8ef0d1..f56ce1d 100644 --- a/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java +++ b/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java @@ -19,15 +19,20 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.test.StepVerifier; import reactor.test.StepVerifier.Step; +import reactor.test.util.TestLogger; +import reactor.util.Loggers; import reactor.util.function.Tuple2; import reactor.util.function.Tuples; public abstract class TestDictionaryMap { + private static final Logger log = LoggerFactory.getLogger(TestDictionaryMap.class); private TestAllocator allocator; private boolean checkLeaks = true; @@ -97,21 +102,25 @@ public abstract class TestDictionaryMap { @ParameterizedTest @MethodSource("provideArgumentsPut") public void testPut(MapType mapType, UpdateMode updateMode, String key, String value, boolean shouldFail) { - var stpVer = StepVerifier - .create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode) - .map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5)) - .flatMap(map -> map - .putValue(key, value) - .then(map.getValue(null, key)) - .doAfterTerminate(map::release) - ) - )); - if (shouldFail) { - this.checkLeaks = false; - stpVer.verifyError(); - } else { - stpVer.expectNext(value).verifyComplete(); - } + var gen = getTempDbGenerator(); + var db = run(gen.openTempDb(allocator)); + var dict = run(tempDictionary(db.db(), updateMode)); + var map = tempDatabaseMapDictionaryMap(dict, mapType, 5); + + runVoid(shouldFail, map.putValue(key, value)); + + var resultingMapSize = run(map.leavesCount(null, false)); + Assertions.assertEquals(shouldFail ? 0 : 1, resultingMapSize); + + var resultingMap = run(map.get(null)); + Assertions.assertEquals(shouldFail ? null : Map.of(key, value), resultingMap); + + runVoid(map.close()); + map.release(); + + //if (shouldFail) this.checkLeaks = false; + + gen.closeTempDb(db); } @ParameterizedTest @@ -257,26 +266,50 @@ public abstract class TestDictionaryMap { .map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5)) .flatMapMany(map -> Flux .concat( - map.updateValue(key, old -> { - assert old == null; - return "error?"; - }).then(map.getValue(null, key)), - map.updateValue(key, false, old -> { - assert Objects.equals(old, "error?"); - return "error?"; - }).then(map.getValue(null, key)), - map.updateValue(key, true, old -> { - assert Objects.equals(old, "error?"); - return "error?"; - }).then(map.getValue(null, key)), - map.updateValue(key, true, old -> { - assert Objects.equals(old, "error?"); - return value; - }).then(map.getValue(null, key)), - map.updateValue(key, true, old -> { - assert Objects.equals(old, value); - return value; - }).then(map.getValue(null, key)) + Mono + .fromRunnable(() -> log.debug("1. Updating value: {}", key)) + .then(map.updateValue(key, old -> { + assert old == null; + return "error?"; + })) + .doOnSuccess(s -> log.debug("1. Getting value: {}", key)) + .then(map.getValue(null, key)), + + Mono + .fromRunnable(() -> log.debug("2. Updating value: {}", key)) + .then(map.updateValue(key, false, old -> { + assert Objects.equals(old, "error?"); + return "error?"; + })) + .doOnSuccess(s -> log.debug("2. Getting value: {}", key)) + .then(map.getValue(null, key)), + + Mono + .fromRunnable(() -> log.debug("3. Updating value: {}", key)) + .then(map.updateValue(key, true, old -> { + assert Objects.equals(old, "error?"); + return "error?"; + })) + .doOnSuccess(s -> log.debug("3. Getting value: {}", key)) + .then(map.getValue(null, key)), + + Mono + .fromRunnable(() -> log.debug("4. Updating value: {}", key)) + .then(map.updateValue(key, true, old -> { + assert Objects.equals(old, "error?"); + return value; + })) + .doOnSuccess(s -> log.debug("4. Getting value: {}", key)) + .then(map.getValue(null, key)), + + Mono + .fromRunnable(() -> log.debug("5. Updating value: {}", key)) + .then(map.updateValue(key, true, old -> { + assert Objects.equals(old, value); + return value; + })) + .doOnSuccess(s -> log.debug("5. Getting value: {}", key)) + .then(map.getValue(null, key)) ) .doAfterTerminate(map::release) ) diff --git a/src/test/java/it/cavallium/dbengine/TestLLDictionary.java b/src/test/java/it/cavallium/dbengine/TestLLDictionary.java new file mode 100644 index 0000000..05fded5 --- /dev/null +++ b/src/test/java/it/cavallium/dbengine/TestLLDictionary.java @@ -0,0 +1,311 @@ +package it.cavallium.dbengine; + +import static it.cavallium.dbengine.DbTestUtils.destroyAllocator; +import static it.cavallium.dbengine.DbTestUtils.ensureNoLeaks; +import static it.cavallium.dbengine.DbTestUtils.newAllocator; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Send; +import it.cavallium.dbengine.DbTestUtils.TempDb; +import it.cavallium.dbengine.DbTestUtils.TestAllocator; +import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLDictionaryResultType; +import it.cavallium.dbengine.database.LLKeyValueDatabase; +import it.cavallium.dbengine.database.LLRange; +import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.UpdateMode; +import it.cavallium.dbengine.database.UpdateReturnMode; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Objects; +import java.util.stream.Stream; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; + +public abstract class TestLLDictionary { + + private final Logger log = LoggerFactory.getLogger(this.getClass()); + private static final Mono> RANGE_ALL = Mono.fromCallable(() -> LLRange.all().send()); + private TestAllocator allocator; + private TempDb tempDb; + private LLKeyValueDatabase db; + + protected abstract TemporaryDbGenerator getTempDbGenerator(); + + @BeforeEach + public void beforeEach() { + this.allocator = newAllocator(); + ensureNoLeaks(allocator.allocator(), false, false); + tempDb = Objects.requireNonNull(getTempDbGenerator().openTempDb(allocator).block(), "TempDB"); + db = tempDb.db(); + } + + @AfterEach + public void afterEach() { + getTempDbGenerator().closeTempDb(tempDb).block(); + ensureNoLeaks(allocator.allocator(), true, false); + destroyAllocator(allocator); + } + + public static Stream provideArguments() { + return Arrays.stream(UpdateMode.values()).map(Arguments::of); + } + + public static Stream providePutArguments() { + var updateModes = Arrays.stream(UpdateMode.values()); + return updateModes.flatMap(updateMode -> { + var resultTypes = Arrays.stream(LLDictionaryResultType.values()); + return resultTypes.map(resultType -> Arguments.of(updateMode, resultType)); + }); + } + + public static Stream provideUpdateArguments() { + var updateModes = Arrays.stream(UpdateMode.values()); + return updateModes.flatMap(updateMode -> { + var resultTypes = Arrays.stream(UpdateReturnMode.values()); + return resultTypes.map(resultType -> Arguments.of(updateMode, resultType)); + }); + } + + private LLDictionary getDict(UpdateMode updateMode) { + var dict = DbTestUtils.tempDictionary(db, updateMode).blockOptional().orElseThrow(); + var key1 = Mono.fromCallable(() -> fromString("test-key-1")); + var key2 = Mono.fromCallable(() -> fromString("test-key-2")); + var key3 = Mono.fromCallable(() -> fromString("test-key-3")); + var key4 = Mono.fromCallable(() -> fromString("test-key-4")); + var value = Mono.fromCallable(() -> fromString("test-value")); + dict.put(key1, value, LLDictionaryResultType.VOID).block(); + dict.put(key2, value, LLDictionaryResultType.VOID).block(); + dict.put(key3, value, LLDictionaryResultType.VOID).block(); + dict.put(key4, value, LLDictionaryResultType.VOID).block(); + return dict; + } + + private Send fromString(String s) { + var sb = s.getBytes(StandardCharsets.UTF_8); + try (var b = db.getAllocator().allocate(sb.length + 3 + 13)) { + assert b.writerOffset() == 0; + assert b.readerOffset() == 0; + b.writerOffset(3).writeBytes(sb); + b.readerOffset(3); + assert b.readableBytes() == sb.length; + + var part1 = b.split(); + + return LLUtils.compositeBuffer(db.getAllocator(), part1.send(), b.send()).send(); + } + } + + private String toString(Send b) { + try (var bb = b.receive()) { + byte[] data = new byte[bb.readableBytes()]; + bb.copyInto(bb.readerOffset(), data, 0, data.length); + return new String(data, StandardCharsets.UTF_8); + } + } + + private void run(Flux publisher) { + publisher.subscribeOn(Schedulers.immediate()).blockLast(); + } + + private void runVoid(Mono publisher) { + publisher.then().subscribeOn(Schedulers.immediate()).block(); + } + + private T run(Mono publisher) { + return publisher.subscribeOn(Schedulers.immediate()).block(); + } + + private T run(boolean shouldFail, Mono publisher) { + return publisher.subscribeOn(Schedulers.immediate()).transform(mono -> { + if (shouldFail) { + return mono.onErrorResume(ex -> Mono.empty()); + } else { + return mono; + } + }).block(); + } + + private void runVoid(boolean shouldFail, Mono publisher) { + publisher.then().subscribeOn(Schedulers.immediate()).transform(mono -> { + if (shouldFail) { + return mono.onErrorResume(ex -> Mono.empty()); + } else { + return mono; + } + }).block(); + } + + @Test + public void testNoOp() { + } + + @Test + public void testNoOpAllocation() { + for (int i = 0; i < 10; i++) { + var a = allocator.allocator().allocate(i * 512); + a.send().receive().close(); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetDict(UpdateMode updateMode) { + var dict = getDict(updateMode); + Assertions.assertNotNull(dict); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetColumnName(UpdateMode updateMode) { + var dict = getDict(updateMode); + Assertions.assertEquals("hash_map_testmap", dict.getColumnName()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetAllocator(UpdateMode updateMode) { + var dict = getDict(updateMode); + var alloc = dict.getAllocator(); + Assertions.assertEquals(alloc, alloc); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGet(UpdateMode updateMode) { + var dict = getDict(updateMode); + var keyEx = Mono.fromCallable(() -> fromString("test-key-1")); + var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent")); + Assertions.assertEquals("test-value", run(dict.get(null, keyEx).map(this::toString).transform(LLUtils::handleDiscard))); + Assertions.assertEquals("test-value", run(dict.get(null, keyEx, true).map(this::toString).transform(LLUtils::handleDiscard))); + Assertions.assertEquals("test-value", run(dict.get(null, keyEx, false).map(this::toString).transform(LLUtils::handleDiscard))); + Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx).map(this::toString).transform(LLUtils::handleDiscard))); + Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx, true).map(this::toString).transform(LLUtils::handleDiscard))); + Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx, false).map(this::toString).transform(LLUtils::handleDiscard))); + } + + @ParameterizedTest + @MethodSource("providePutArguments") + public void testPutExisting(UpdateMode updateMode, LLDictionaryResultType resultType) { + var dict = getDict(updateMode); + var keyEx = Mono.fromCallable(() -> fromString("test-key-1")); + var value = Mono.fromCallable(() -> fromString("test-value")); + + var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false)); + + runVoid(dict.put(keyEx, value, resultType).then().doOnDiscard(Send.class, Send::close)); + + var afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + Assertions.assertEquals(0, afterSize - beforeSize); + } + + @ParameterizedTest + @MethodSource("providePutArguments") + public void testPutNew(UpdateMode updateMode, LLDictionaryResultType resultType) { + var dict = getDict(updateMode); + var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent")); + var value = Mono.fromCallable(() -> fromString("test-value")); + + var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false)); + + runVoid(dict.put(keyNonEx, value, resultType).then().doOnDiscard(Send.class, Send::close)); + + var afterSize = run(dict.sizeRange(null, Mono.fromCallable(() -> LLRange.all().send()), false)); + Assertions.assertEquals(1, afterSize - beforeSize); + + Assertions.assertTrue(run(dict.getRangeKeys(null, RANGE_ALL).map(this::toString).collectList()).contains("test-nonexistent")); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetUpdateMode(UpdateMode updateMode) { + var dict = getDict(updateMode); + assertEquals(updateMode, run(dict.getUpdateMode())); + } + + @ParameterizedTest + @MethodSource("provideUpdateArguments") + public void testUpdateExisting(UpdateMode updateMode, UpdateReturnMode updateReturnMode) { + var dict = getDict(updateMode); + var keyEx = Mono.fromCallable(() -> fromString("test-key-1")); + var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false)); + long afterSize; + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, true).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(0, afterSize - beforeSize); + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, false).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(0, afterSize - beforeSize); + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(0, afterSize - beforeSize); + } + + @ParameterizedTest + @MethodSource("provideUpdateArguments") + public void testUpdateNew(UpdateMode updateMode, UpdateReturnMode updateReturnMode) { + int expected = updateMode == UpdateMode.DISALLOW ? 0 : 1; + var dict = getDict(updateMode); + var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent")); + var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false)); + long afterSize; + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, true).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(expected, afterSize - beforeSize); + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, false).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(expected, afterSize - beforeSize); + runVoid(updateMode == UpdateMode.DISALLOW, + dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).then().transform(LLUtils::handleDiscard) + ); + afterSize = run(dict.sizeRange(null, RANGE_ALL, false)); + assertEquals(expected, afterSize - beforeSize); + + if (updateMode != UpdateMode.DISALLOW) { + Assertions.assertTrue(run(dict.getRangeKeys(null, RANGE_ALL).map(this::toString).collectList()).contains( + "test-nonexistent")); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testUpdateAndGetDelta(UpdateMode updateMode) { + log.warn("Test not implemented"); + //todo: implement + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testClear(UpdateMode updateMode) { + log.warn("Test not implemented"); + //todo: implement + } + + @ParameterizedTest + @MethodSource("providePutArguments") + public void testRemove(UpdateMode updateMode, LLDictionaryResultType resultType) { + log.warn("Test not implemented"); + //todo: implement + } +} diff --git a/src/test/java/it/cavallium/dbengine/TestLLDictionaryLeaks.java b/src/test/java/it/cavallium/dbengine/TestLLDictionaryLeaks.java index 3a15e47..026107d 100644 --- a/src/test/java/it/cavallium/dbengine/TestLLDictionaryLeaks.java +++ b/src/test/java/it/cavallium/dbengine/TestLLDictionaryLeaks.java @@ -26,9 +26,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Schedulers; public abstract class TestLLDictionaryLeaks { @@ -90,43 +88,12 @@ public abstract class TestLLDictionaryLeaks { private Send fromString(String s) { var sb = s.getBytes(StandardCharsets.UTF_8); try (var b = db.getAllocator().allocate(sb.length)) { - b.writeBytes(b); + b.writeBytes(sb); + assert b.readableBytes() == sb.length; return b.send(); } } - private void run(Flux publisher) { - publisher.subscribeOn(Schedulers.immediate()).blockLast(); - } - - private void runVoid(Mono publisher) { - publisher.then().subscribeOn(Schedulers.immediate()).block(); - } - - private T run(Mono publisher) { - return publisher.subscribeOn(Schedulers.immediate()).block(); - } - - private T run(boolean shouldFail, Mono publisher) { - return publisher.subscribeOn(Schedulers.immediate()).transform(mono -> { - if (shouldFail) { - return mono.onErrorResume(ex -> Mono.empty()); - } else { - return mono; - } - }).block(); - } - - private void runVoid(boolean shouldFail, Mono publisher) { - publisher.then().subscribeOn(Schedulers.immediate()).transform(mono -> { - if (shouldFail) { - return mono.onErrorResume(ex -> Mono.empty()); - } else { - return mono; - } - }).block(); - } - @Test public void testNoOp() { } @@ -164,9 +131,9 @@ public abstract class TestLLDictionaryLeaks { public void testGet(UpdateMode updateMode) { var dict = getDict(updateMode); var key = Mono.fromCallable(() -> fromString("test")); - runVoid(dict.get(null, key).then().transform(LLUtils::handleDiscard)); - runVoid(dict.get(null, key, true).then().transform(LLUtils::handleDiscard)); - runVoid(dict.get(null, key, false).then().transform(LLUtils::handleDiscard)); + DbTestUtils.runVoid(dict.get(null, key).then().transform(LLUtils::handleDiscard)); + DbTestUtils.runVoid(dict.get(null, key, true).then().transform(LLUtils::handleDiscard)); + DbTestUtils.runVoid(dict.get(null, key, false).then().transform(LLUtils::handleDiscard)); } @ParameterizedTest @@ -175,14 +142,14 @@ public abstract class TestLLDictionaryLeaks { var dict = getDict(updateMode); var key = Mono.fromCallable(() -> fromString("test-key")); var value = Mono.fromCallable(() -> fromString("test-value")); - runVoid(dict.put(key, value, resultType).then().doOnDiscard(Send.class, Send::close)); + DbTestUtils.runVoid(dict.put(key, value, resultType).then().doOnDiscard(Send.class, Send::close)); } @ParameterizedTest @MethodSource("provideArguments") public void testGetUpdateMode(UpdateMode updateMode) { var dict = getDict(updateMode); - assertEquals(updateMode, run(dict.getUpdateMode())); + assertEquals(updateMode, DbTestUtils.run(dict.getUpdateMode())); } @ParameterizedTest @@ -190,13 +157,13 @@ public abstract class TestLLDictionaryLeaks { public void testUpdate(UpdateMode updateMode, UpdateReturnMode updateReturnMode) { var dict = getDict(updateMode); var key = Mono.fromCallable(() -> fromString("test-key")); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.update(key, old -> old, updateReturnMode, true).then().transform(LLUtils::handleDiscard) ); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.update(key, old -> old, updateReturnMode, false).then().transform(LLUtils::handleDiscard) ); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.update(key, old -> old, updateReturnMode).then().transform(LLUtils::handleDiscard) ); } @@ -206,13 +173,13 @@ public abstract class TestLLDictionaryLeaks { public void testUpdateAndGetDelta(UpdateMode updateMode) { var dict = getDict(updateMode); var key = Mono.fromCallable(() -> fromString("test-key")); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.updateAndGetDelta(key, old -> old, true).then().transform(LLUtils::handleDiscard) ); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.updateAndGetDelta(key, old -> old, false).then().transform(LLUtils::handleDiscard) ); - runVoid(updateMode == UpdateMode.DISALLOW, + DbTestUtils.runVoid(updateMode == UpdateMode.DISALLOW, dict.updateAndGetDelta(key, old -> old).then().transform(LLUtils::handleDiscard) ); } @@ -221,7 +188,7 @@ public abstract class TestLLDictionaryLeaks { @MethodSource("provideArguments") public void testClear(UpdateMode updateMode) { var dict = getDict(updateMode); - runVoid(dict.clear()); + DbTestUtils.runVoid(dict.clear()); } @ParameterizedTest @@ -229,6 +196,6 @@ public abstract class TestLLDictionaryLeaks { public void testRemove(UpdateMode updateMode, LLDictionaryResultType resultType) { var dict = getDict(updateMode); var key = Mono.fromCallable(() -> fromString("test-key")); - runVoid(dict.remove(key, resultType).then().doOnDiscard(Send.class, Send::close)); + DbTestUtils.runVoid(dict.remove(key, resultType).then().doOnDiscard(Send.class, Send::close)); } } diff --git a/src/test/java/it/cavallium/dbengine/TestLocalLLDictionary.java b/src/test/java/it/cavallium/dbengine/TestLocalLLDictionary.java new file mode 100644 index 0000000..abd01de --- /dev/null +++ b/src/test/java/it/cavallium/dbengine/TestLocalLLDictionary.java @@ -0,0 +1,11 @@ +package it.cavallium.dbengine; + +public class TestLocalLLDictionary extends TestLLDictionary { + + private static final TemporaryDbGenerator GENERATOR = new LocalTemporaryDbGenerator(); + + @Override + protected TemporaryDbGenerator getTempDbGenerator() { + return GENERATOR; + } +} diff --git a/src/test/java/it/cavallium/dbengine/TestMemoryLLDictionary.java b/src/test/java/it/cavallium/dbengine/TestMemoryLLDictionary.java new file mode 100644 index 0000000..ce032a6 --- /dev/null +++ b/src/test/java/it/cavallium/dbengine/TestMemoryLLDictionary.java @@ -0,0 +1,11 @@ +package it.cavallium.dbengine; + +public class TestMemoryLLDictionary extends TestLLDictionary { + + private static final TemporaryDbGenerator GENERATOR = new MemoryTemporaryDbGenerator(); + + @Override + protected TemporaryDbGenerator getTempDbGenerator() { + return GENERATOR; + } +} diff --git a/src/test/resources/log4j2.xml b/src/test/resources/log4j2.xml new file mode 100644 index 0000000..9731f37 --- /dev/null +++ b/src/test/resources/log4j2.xml @@ -0,0 +1,20 @@ + + + + + + + + + + + + + + \ No newline at end of file From 552b5f78cef668bea0b217e67f874663e4a657c4 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 02:22:30 +0200 Subject: [PATCH 15/23] Fix size check assertion --- .../database/collections/DatabaseMapDictionaryDeep.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java index 2dff876..b02277c 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java @@ -250,9 +250,11 @@ public class DatabaseMapDictionaryDeep> implem * Removes the prefix from the key */ protected void removePrefix(Buffer key) { - assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; + assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength + || key.readableBytes() == keyPrefixLength + keySuffixLength; key.readerOffset(key.readerOffset() + this.keyPrefixLength).compact(); - assert key.readableBytes() == keySuffixLength + keyExtLength; + assert key.readableBytes() == keySuffixLength + keyExtLength + || key.readableBytes() == keySuffixLength; } /** From b77b441515012638660c1c4be5e3bb9ffba083b0 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 11:30:44 +0200 Subject: [PATCH 16/23] Fix all errors --- .../cavallium/dbengine/database/LLUtils.java | 38 ++++++++++- .../DatabaseMapDictionaryDeep.java | 4 +- .../DatabaseMapDictionaryHashed.java | 2 +- .../collections/DatabaseSetDictionary.java | 3 +- .../DatabaseSetDictionaryHashed.java | 6 +- .../collections/DatabaseStageMap.java | 3 +- .../database/disk/LLLocalDictionary.java | 5 +- .../LLLocalGroupedReactiveRocksIterator.java | 24 +++++++ ...LLLocalKeyPrefixReactiveRocksIterator.java | 24 +++++++ .../disk/LLLocalReactiveRocksIterator.java | 23 +++++++ .../dbengine/TestDictionaryMapDeep.java | 68 ++++++++++++++----- 11 files changed, 171 insertions(+), 29 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index c251e19..7c3a1d1 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -193,6 +193,34 @@ public class LLUtils { } } + public static String toStringSafe(@Nullable LLRange range) { + try { + if (range == null || range.isAccessible()) { + return toString(range); + } else { + return "(released)"; + } + } catch (IllegalReferenceCountException ex) { + return "(released)"; + } + } + + public static String toString(@Nullable LLRange range) { + if (range == null) { + return "null"; + } else if (range.isAll()) { + return "ξ"; + } else if (range.hasMin() && range.hasMax()) { + return "[" + toStringSafe(range.getMinUnsafe()) + "," + toStringSafe(range.getMaxUnsafe()) + ")"; + } else if (range.hasMin()) { + return "[" + toStringSafe(range.getMinUnsafe()) + ",*)"; + } else if (range.hasMax()) { + return "[*," + toStringSafe(range.getMaxUnsafe()) + ")"; + } else { + return "∅"; + } + } + public static String toString(@Nullable Buffer key) { if (key == null) { return "null"; @@ -215,6 +243,8 @@ public class LLUtils { if (isAscii) { if (byteVal >= 32 && byteVal < 127) { asciiSB.append((char) byteVal); + } else if (byteVal == 0) { + asciiSB.append('␀'); } else { isAscii = false; asciiSB = null; @@ -477,16 +507,20 @@ public class LLUtils { @NotNull public static DirectBuffer convertToReadableDirect(BufferAllocator allocator, Send content) { try (var buf = content.receive()) { + DirectBuffer result; if (buf.countComponents() == 1) { var direct = obtainDirect(buf, false); - return new DirectBuffer(buf.send(), direct); + result = new DirectBuffer(buf.send(), direct); } else { var direct = newDirect(allocator, buf.readableBytes()); try (var buf2 = direct.buffer().receive()) { buf.copyInto(buf.readerOffset(), buf2, buf2.writerOffset(), buf.readableBytes()); - return new DirectBuffer(buf2.send(), direct.byteBuffer()); + buf2.writerOffset(buf2.writerOffset() + buf.readableBytes()); + assert buf2.readableBytes() == buf.readableBytes(); + result = new DirectBuffer(buf2.send(), direct.byteBuffer()); } } + return result; } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java index b02277c..baf3492 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java @@ -252,7 +252,7 @@ public class DatabaseMapDictionaryDeep> implem protected void removePrefix(Buffer key) { assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength || key.readableBytes() == keyPrefixLength + keySuffixLength; - key.readerOffset(key.readerOffset() + this.keyPrefixLength).compact(); + key.readerOffset(key.readerOffset() + this.keyPrefixLength); assert key.readableBytes() == keySuffixLength + keyExtLength || key.readableBytes() == keySuffixLength; } @@ -262,7 +262,7 @@ public class DatabaseMapDictionaryDeep> implem */ protected void removeExt(Buffer key) { assert key.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength; - key.writerOffset(keyPrefixLength + keySuffixLength).compact(); + key.writerOffset(keyPrefixLength + keySuffixLength); assert key.readableBytes() == keyPrefixLength + keySuffixLength; } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java index 653cc35..66815c3 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java @@ -31,7 +31,7 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap keySuffixHashFunction; protected DatabaseMapDictionaryHashed(LLDictionary dictionary, - Send prefixKey, + @NotNull Send prefixKey, Serializer keySuffixSerializer, Serializer valueSerializer, Function keySuffixHashFunction, diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java index f1903f8..260ff51 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.collections.DatabaseEmpty.Nothing; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.util.HashMap; @@ -23,7 +24,7 @@ public class DatabaseSetDictionary extends DatabaseMapDictionary public static DatabaseSetDictionary simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer) { - return new DatabaseSetDictionary<>(dictionary, null, keySerializer); + return new DatabaseSetDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer); } public static DatabaseSetDictionary tail(LLDictionary dictionary, diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java index c9c6a38..16a9f3d 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.collections.DatabaseEmpty.Nothing; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; @@ -11,6 +12,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.function.Function; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Mono; @@ -18,7 +20,7 @@ import reactor.core.publisher.Mono; public class DatabaseSetDictionaryHashed extends DatabaseMapDictionaryHashed { protected DatabaseSetDictionaryHashed(LLDictionary dictionary, - Send prefixKey, + @NotNull Send prefixKey, Serializer keySuffixSerializer, Function keySuffixHashFunction, SerializerFixedBinaryLength keySuffixHashSerializer) { @@ -36,7 +38,7 @@ public class DatabaseSetDictionaryHashed extends DatabaseMapDictionaryHas Function keyHashFunction, SerializerFixedBinaryLength keyHashSerializer) { return new DatabaseSetDictionaryHashed<>(dictionary, - null, + LLUtils.empty(dictionary.getAllocator()), keySerializer, keyHashFunction, keyHashSerializer diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java index 42cf458..fdc4c2e 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java @@ -287,7 +287,8 @@ public interface DatabaseStageMap> extends Dat @Override default Mono> get(@Nullable CompositeSnapshot snapshot, boolean existsAlmostCertainly) { return getAllValues(snapshot) - .collectMap(Entry::getKey, Entry::getValue, HashMap::new); + .collectMap(Entry::getKey, Entry::getValue, HashMap::new) + .filter(map -> !map.isEmpty()); } @Override diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java index 8c53e41..075fa91 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java @@ -834,9 +834,6 @@ public class LLLocalDictionary implements LLDictionary { stamp = 0; } try { - if (logger.isTraceEnabled()) { - logger.trace(MARKER_ROCKSDB, "Reading {} (before update)", LLUtils.toStringSafe(key)); - } while (true) { @Nullable Buffer prevData; var prevDataHolder = existsAlmostCertainly ? null : new Holder(); @@ -862,7 +859,7 @@ public class LLLocalDictionary implements LLDictionary { } if (logger.isTraceEnabled()) { logger.trace(MARKER_ROCKSDB, - "Read {}: {} (before update)", + "Reading {}: {} (before update)", LLUtils.toStringSafe(key), LLUtils.toStringSafe(prevData) ); diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java index e2c82f5..e4db431 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalGroupedReactiveRocksIterator.java @@ -1,5 +1,7 @@ package it.cavallium.dbengine.database.disk; +import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB; + import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.Send; @@ -12,10 +14,13 @@ import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; public abstract class LLLocalGroupedReactiveRocksIterator { + protected static final Logger logger = LoggerFactory.getLogger(LLLocalGroupedReactiveRocksIterator.class); private final RocksDB db; private final BufferAllocator alloc; private final ColumnFamilyHandle cfh; @@ -52,6 +57,9 @@ public abstract class LLLocalGroupedReactiveRocksIterator { .generate(() -> { var readOptions = new ReadOptions(this.readOptions); readOptions.setFillCache(canFillCache && range.hasMin() && range.hasMax()); + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range)); + } return LLLocalDictionary.getRocksIterator(alloc, allowNettyDirect, readOptions, range.copy().send(), db, cfh); }, (tuple, sink) -> { try { @@ -74,6 +82,16 @@ public abstract class LLLocalGroupedReactiveRocksIterator { } else { value = null; } + + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Range {} is reading {}: {}", + LLUtils.toStringSafe(range), + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(value) + ); + } + try { rocksIterator.next(); rocksIterator.status(); @@ -94,9 +112,15 @@ public abstract class LLLocalGroupedReactiveRocksIterator { if (!values.isEmpty()) { sink.next(values); } else { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(range)); + } sink.complete(); } } catch (RocksDBException ex) { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(range)); + } sink.error(ex); } return tuple; diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java index 818882d..d82e570 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalKeyPrefixReactiveRocksIterator.java @@ -1,5 +1,7 @@ package it.cavallium.dbengine.database.disk; +import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB; + import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; import io.net5.buffer.api.Send; @@ -9,10 +11,13 @@ import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; public class LLLocalKeyPrefixReactiveRocksIterator { + protected static final Logger logger = LoggerFactory.getLogger(LLLocalKeyPrefixReactiveRocksIterator.class); private final RocksDB db; private final BufferAllocator alloc; private final ColumnFamilyHandle cfh; @@ -54,6 +59,9 @@ public class LLLocalKeyPrefixReactiveRocksIterator { readOptions.setReadaheadSize(32 * 1024); // 32KiB readOptions.setFillCache(canFillCache); } + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range)); + } return LLLocalDictionary.getRocksIterator(alloc, allowNettyDirect, readOptions, rangeSend, db, cfh); }, (tuple, sink) -> { try { @@ -79,11 +87,24 @@ public class LLLocalKeyPrefixReactiveRocksIterator { rocksIterator.status(); } } + if (firstGroupKey != null) { var groupKeyPrefix = firstGroupKey.copy(firstGroupKey.readerOffset(), prefixLength); assert groupKeyPrefix.isAccessible(); + + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Range {} is reading prefix {}", + LLUtils.toStringSafe(range), + LLUtils.toStringSafe(groupKeyPrefix) + ); + } + sink.next(groupKeyPrefix.send()); } else { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(range)); + } sink.complete(); } } finally { @@ -92,6 +113,9 @@ public class LLLocalKeyPrefixReactiveRocksIterator { } } } catch (RocksDBException ex) { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(range)); + } sink.error(ex); } return tuple; diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java index 70428da..a1d5d32 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLLocalReactiveRocksIterator.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database.disk; +import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB; import static it.cavallium.dbengine.database.disk.LLLocalDictionary.getRocksIterator; import io.net5.buffer.api.Buffer; @@ -14,10 +15,13 @@ import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; public abstract class LLLocalReactiveRocksIterator { + protected static final Logger logger = LoggerFactory.getLogger(LLLocalReactiveRocksIterator.class); private final AtomicBoolean released = new AtomicBoolean(false); private final RocksDB db; private final BufferAllocator alloc; @@ -51,6 +55,9 @@ public abstract class LLLocalReactiveRocksIterator { readOptions.setReadaheadSize(32 * 1024); // 32KiB readOptions.setFillCache(false); } + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range)); + } return getRocksIterator(alloc, allowNettyDirect, readOptions, range.copy().send(), db, cfh); }, (tuple, sink) -> { try { @@ -74,6 +81,16 @@ public abstract class LLLocalReactiveRocksIterator { } else { value = null; } + + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, + "Range {} is reading {}: {}", + LLUtils.toStringSafe(range), + LLUtils.toStringSafe(key), + LLUtils.toStringSafe(value) + ); + } + try { rocksIterator.next(); rocksIterator.status(); @@ -85,9 +102,15 @@ public abstract class LLLocalReactiveRocksIterator { } } } else { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(range)); + } sink.complete(); } } catch (RocksDBException ex) { + if (logger.isTraceEnabled()) { + logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(range)); + } sink.error(ex); } return tuple; diff --git a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java index 2238a62..d7073c7 100644 --- a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java +++ b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java @@ -5,7 +5,10 @@ import static it.cavallium.dbengine.DbTestUtils.ensureNoLeaks; import static it.cavallium.dbengine.DbTestUtils.isCIMode; import static it.cavallium.dbengine.DbTestUtils.newAllocator; import static it.cavallium.dbengine.DbTestUtils.destroyAllocator; +import static it.cavallium.dbengine.DbTestUtils.run; +import static it.cavallium.dbengine.DbTestUtils.runVoid; import static it.cavallium.dbengine.DbTestUtils.tempDatabaseMapDictionaryDeepMap; +import static it.cavallium.dbengine.DbTestUtils.tempDatabaseMapDictionaryMap; import static it.cavallium.dbengine.DbTestUtils.tempDb; import static it.cavallium.dbengine.DbTestUtils.tempDictionary; @@ -23,12 +26,15 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; import java.util.stream.Stream; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.MethodOrderer; import org.junit.jupiter.api.TestMethodOrder; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.test.StepVerifier; @@ -42,6 +48,7 @@ import reactor.util.function.Tuples; @TestMethodOrder(MethodOrderer.MethodName.class) public abstract class TestDictionaryMapDeep { + private final Logger log = LoggerFactory.getLogger(this.getClass()); private TestAllocator allocator; private boolean checkLeaks = true; @@ -174,22 +181,51 @@ public abstract class TestDictionaryMapDeep { @ParameterizedTest @MethodSource("provideArgumentsSet") - public void testSetValueGetValue(UpdateMode updateMode, String key, Map value, boolean shouldFail) { - var stpVer = StepVerifier - .create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode) - .map(dict -> tempDatabaseMapDictionaryDeepMap(dict, 5, 6)) - .flatMap(map -> map - .putValue(key, value) - .then(map.getValue(null, key)) - .doAfterTerminate(map::release) - ) - )); - if (shouldFail) { - this.checkLeaks = false; - stpVer.verifyError(); - } else { - stpVer.expectNext(value).verifyComplete(); - } + public void testPutValue(UpdateMode updateMode, String key, Map value, boolean shouldFail) { + var gen = getTempDbGenerator(); + var db = run(gen.openTempDb(allocator)); + var dict = run(tempDictionary(db.db(), updateMode)); + var map = tempDatabaseMapDictionaryDeepMap(dict, 5, 6); + + log.debug("Put \"{}\" = \"{}\"", key, value); + runVoid(shouldFail, map.putValue(key, value)); + + var resultingMapSize = run(map.leavesCount(null, false)); + Assertions.assertEquals(shouldFail ? 0 : value.size(), resultingMapSize); + + var resultingMap = run(map.get(null)); + Assertions.assertEquals(shouldFail ? null : Map.of(key, value), resultingMap); + + runVoid(map.close()); + map.release(); + + //if (shouldFail) this.checkLeaks = false; + + gen.closeTempDb(db); + } + + @ParameterizedTest + @MethodSource("provideArgumentsSet") + public void testGetValue(UpdateMode updateMode, String key, Map value, boolean shouldFail) { + var gen = getTempDbGenerator(); + var db = run(gen.openTempDb(allocator)); + var dict = run(tempDictionary(db.db(), updateMode)); + var map = tempDatabaseMapDictionaryDeepMap(dict, 5, 6); + + log.debug("Put \"{}\" = \"{}\"", key, value); + runVoid(shouldFail, map.putValue(key, value)); + + log.debug("Get \"{}\"", key); + var returnedValue = run(shouldFail, map.getValue(null, key)); + + Assertions.assertEquals(shouldFail ? null : value, returnedValue); + + runVoid(map.close()); + map.release(); + + //if (shouldFail) this.checkLeaks = false; + + gen.closeTempDb(db); } @ParameterizedTest From d4c977047c44283e39a36388e79a150b5668fee4 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 14:50:15 +0200 Subject: [PATCH 17/23] Allow sorted/scored counting --- .../searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index a1c7503..fd88002 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -27,11 +27,11 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea var indexSearchersResource = Mono .fromRunnable(() -> { LLUtils.ensureBlocking(); - if (queryParams.isSorted()) { + if (queryParams.isSorted() && queryParams.limit() > 0) { throw new UnsupportedOperationException("Sorted queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } - if (queryParams.isScored()) { + if (queryParams.isScored() && queryParams.limit() > 0) { throw new UnsupportedOperationException("Scored queries are not supported" + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } From 6c849fae78c15b143fa800e38ad2a1b6e328221d Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 15:34:56 +0200 Subject: [PATCH 18/23] Change makeInaccessible visibility --- .../dbengine/client/SearchResult.java | 23 ++----------- .../dbengine/client/SearchResultKeys.java | 22 ++----------- .../cavallium/dbengine/database/LLRange.java | 4 +-- .../database/LLSearchResultShard.java | 23 ++----------- .../database/disk/LLIndexSearcher.java | 23 ++----------- .../database/disk/LLIndexSearchers.java | 32 +++++++------------ .../lucene/searcher/LuceneSearchResult.java | 23 ++----------- .../dbengine/netty/NullableBuffer.java | 15 +++------ 8 files changed, 27 insertions(+), 138 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResult.java b/src/main/java/it/cavallium/dbengine/client/SearchResult.java index 5026ac4..efbca1c 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResult.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResult.java @@ -18,7 +18,7 @@ public final class SearchResult extends ResourceSupport public SearchResult(Flux> results, TotalHitsCount totalHitsCount, Drop> drop) { - super(new CloseOnDrop<>(drop)); + super(drop); this.results = results; this.totalHitsCount = totalHitsCount; } @@ -49,30 +49,11 @@ public final class SearchResult extends ResourceSupport protected Owned> prepareSend() { var results = this.results; var totalHitsCount = this.totalHitsCount; - makeInaccessible(); return drop -> new SearchResult<>(results, totalHitsCount, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.results = null; this.totalHitsCount = null; } - - private static class CloseOnDrop implements Drop> { - - private final Drop> delegate; - - public CloseOnDrop(Drop> drop) { - this.delegate = drop; - } - - @Override - public void drop(SearchResult obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } } diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java index c22cbdf..f071a1c 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java @@ -23,7 +23,7 @@ public final class SearchResultKeys extends ResourceSupport> results, TotalHitsCount totalHitsCount, Drop> drop) { - super(new SearchResultKeys.CloseOnDrop<>(drop)); + super(drop); this.results = results; this.totalHitsCount = totalHitsCount; } @@ -65,27 +65,9 @@ public final class SearchResultKeys extends ResourceSupport new SearchResultKeys<>(results, totalHitsCount, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.results = null; this.totalHitsCount = null; } - private static class CloseOnDrop implements Drop> { - - private final Drop> delegate; - - public CloseOnDrop(Drop> drop) { - this.delegate = drop; - } - - @Override - public void drop(SearchResultKeys obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } - } diff --git a/src/main/java/it/cavallium/dbengine/database/LLRange.java b/src/main/java/it/cavallium/dbengine/database/LLRange.java index cbbdfb9..ef6199e 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLRange.java +++ b/src/main/java/it/cavallium/dbengine/database/LLRange.java @@ -193,11 +193,10 @@ public class LLRange extends ResourceSupport { minSend = this.min != null ? this.min.send() : null; maxSend = this.max != null ? this.max.send() : null; singleSend = this.single != null ? this.single.send() : null; - this.makeInaccessible(); return drop -> new LLRange(minSend, maxSend, singleSend, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.min = null; this.max = null; this.single = null; @@ -216,7 +215,6 @@ public class LLRange extends ResourceSupport { if (obj.min != null) obj.min.close(); if (obj.max != null) obj.max.close(); if (obj.single != null) obj.single.close(); - obj.makeInaccessible(); delegate.drop(obj); } } diff --git a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java index 82ac4f7..ad07bec 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java +++ b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java @@ -17,7 +17,7 @@ public final class LLSearchResultShard extends ResourceSupport results, TotalHitsCount totalHitsCount, Drop drop) { - super(new LLSearchResultShard.CloseOnDrop(drop)); + super(drop); this.results = results; this.totalHitsCount = totalHitsCount; } @@ -65,30 +65,11 @@ public final class LLSearchResultShard extends ResourceSupport prepareSend() { var results = this.results; var totalHitsCount = this.totalHitsCount; - makeInaccessible(); return drop -> new LLSearchResultShard(results, totalHitsCount, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.results = null; this.totalHitsCount = null; } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(LLSearchResultShard obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java index 093eccd..f5b6e5e 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -16,7 +16,7 @@ public class LLIndexSearcher extends ResourceSupport drop) { - super(new LLIndexSearcher.CloseOnDrop(drop)); + super(drop); this.indexSearcher = indexSearcher; } @@ -42,29 +42,10 @@ public class LLIndexSearcher extends ResourceSupport prepareSend() { var indexSearcher = this.indexSearcher; - makeInaccessible(); return drop -> new LLIndexSearcher(indexSearcher, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.indexSearcher = null; } - - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(LLIndexSearcher obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java index 684385b..04203dd 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java @@ -80,11 +80,10 @@ public interface LLIndexSearchers extends Resource { @Override protected Owned prepareSend() { Send indexSearcher = this.indexSearcher.send(); - this.makeInaccessible(); return drop -> new UnshardedIndexSearchers(indexSearcher, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.indexSearcher = null; } @@ -98,12 +97,8 @@ public interface LLIndexSearchers extends Resource { @Override public void drop(UnshardedIndexSearchers obj) { - try { - if (obj.indexSearcher != null) obj.indexSearcher.close(); - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } + if (obj.indexSearcher != null) obj.indexSearcher.close(); + delegate.drop(obj); } } } @@ -176,11 +171,10 @@ public interface LLIndexSearchers extends Resource { for (LLIndexSearcher indexSearcher : this.indexSearchers) { indexSearchers.add(indexSearcher.send()); } - this.makeInaccessible(); return drop -> new ShardedIndexSearchers(indexSearchers, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.indexSearchers = null; this.indexSearchersVals = null; } @@ -196,20 +190,16 @@ public interface LLIndexSearchers extends Resource { @Override public void drop(ShardedIndexSearchers obj) { - try { - assert !dropped; - if (obj.indexSearchers != null) { - for (LLIndexSearcher indexSearcher : obj.indexSearchers) { - if (indexSearcher.isAccessible()) { - indexSearcher.close(); - } + assert !dropped; + if (obj.indexSearchers != null) { + for (LLIndexSearcher indexSearcher : obj.indexSearchers) { + if (indexSearcher.isAccessible()) { + indexSearcher.close(); } } - dropped = true; - delegate.drop(obj); - } finally { - obj.makeInaccessible(); } + dropped = true; + delegate.drop(obj); } } } diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java index 3e2037a..9a52926 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java @@ -22,7 +22,7 @@ public final class LuceneSearchResult extends ResourceSupport results; public LuceneSearchResult(TotalHitsCount totalHitsCount, Flux results, Drop drop) { - super(new LuceneSearchResult.CloseOnDrop(drop)); + super(drop); this.totalHitsCount = totalHitsCount; this.results = results; } @@ -70,31 +70,12 @@ public final class LuceneSearchResult extends ResourceSupport prepareSend() { var totalHitsCount = this.totalHitsCount; var results = this.results; - makeInaccessible(); return drop -> new LuceneSearchResult(totalHitsCount, results, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.totalHitsCount = null; this.results = null; } - private static class CloseOnDrop implements Drop { - - private final Drop delegate; - - public CloseOnDrop(Drop drop) { - this.delegate = drop; - } - - @Override - public void drop(LuceneSearchResult obj) { - try { - delegate.drop(obj); - } finally { - obj.makeInaccessible(); - } - } - } - } diff --git a/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java index c1e3eef..43f4a25 100644 --- a/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java +++ b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java @@ -41,11 +41,10 @@ public class NullableBuffer extends ResourceSupport prepareSend() { var buffer = this.buffer == null ? null : this.buffer.send(); - makeInaccessible(); return drop -> new NullableBuffer(buffer, drop); } - private void makeInaccessible() { + protected void makeInaccessible() { this.buffer = null; } @@ -59,16 +58,12 @@ public class NullableBuffer extends ResourceSupport Date: Thu, 23 Sep 2021 15:37:13 +0200 Subject: [PATCH 19/23] Improve logging --- .../lucene/searcher/ScoredSimpleLuceneShardSearcher.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java index cef1b48..46b9376 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/ScoredSimpleLuceneShardSearcher.java @@ -8,6 +8,7 @@ import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.disk.LLIndexSearcher; import it.cavallium.dbengine.database.disk.LLIndexSearchers; +import it.cavallium.dbengine.database.disk.LLLocalGroupedReactiveRocksIterator; import it.cavallium.dbengine.lucene.LuceneUtils; import java.util.Arrays; import java.util.List; @@ -16,12 +17,16 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Sort; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Schedulers; public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { + protected static final Logger logger = LoggerFactory.getLogger(ScoredSimpleLuceneShardSearcher.class); + public ScoredSimpleLuceneShardSearcher() { } @@ -127,10 +132,10 @@ public class ScoredSimpleLuceneShardSearcher implements LuceneMultiSearcher { AtomicReference currentPageInfoRef = new AtomicReference<>(secondPageInfo); return Mono .fromSupplier(currentPageInfoRef::get) - .doOnNext(s -> System.err.println("Current page info: " + s)) + .doOnNext(s -> logger.debug("Current page info: {}", s)) .flatMap(currentPageInfo -> this.searchPage(queryParams, indexSearchers, true, queryParams.pageLimits(), 0, currentPageInfo)) - .doOnNext(s -> System.err.println("Next page info: " + s.nextPageInfo())) + .doOnNext(s -> logger.debug("Next page info: {}", s.nextPageInfo())) .doOnNext(s -> currentPageInfoRef.set(s.nextPageInfo())) .repeatWhen(s -> s.takeWhile(n -> n > 0)); }) From 247207febee9de8184f9eece1eff098159f295d7 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 15:48:27 +0200 Subject: [PATCH 20/23] Temporary --- .../java/it/cavallium/dbengine/database/LLUtils.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 7c3a1d1..5b572ff 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -479,10 +479,14 @@ public class LLUtils { } public static Send empty(BufferAllocator allocator) { - try (var empty = CompositeBuffer.compose(allocator)) { - assert empty.readableBytes() == 0; - assert empty.capacity() == 0; - return empty.send(); + try { + return allocator.allocate(0).send(); + } catch (Exception ex) { + try (var empty = CompositeBuffer.compose(allocator)) { + assert empty.readableBytes() == 0; + assert empty.capacity() == 0; + return empty.send(); + } } } From 3a544d4297ef91f07a4c89a56976f3dd0e244a63 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 20:57:28 +0200 Subject: [PATCH 21/23] Partial migration to ResourceSupport --- .../dbengine/client/SearchResult.java | 3 +- .../dbengine/client/SearchResultKeys.java | 3 +- .../cavallium/dbengine/database/LLDelta.java | 2 +- .../cavallium/dbengine/database/LLEntry.java | 2 +- .../cavallium/dbengine/database/LLRange.java | 8 +- .../database/LLSearchResultShard.java | 2 +- .../cavallium/dbengine/database/LLUtils.java | 23 ++- .../database/LiveResourceSupport.java | 33 ++++ .../database/collections/DatabaseEmpty.java | 8 +- .../collections/DatabaseMapDictionary.java | 24 +-- .../DatabaseMapDictionaryDeep.java | 179 ++++++++++-------- .../DatabaseMapDictionaryHashed.java | 97 +++++++--- .../collections/DatabaseSetDictionary.java | 16 +- .../DatabaseSetDictionaryHashed.java | 18 +- .../database/collections/DatabaseSingle.java | 50 ++++- .../collections/DatabaseSingleBucket.java | 71 +++++-- .../collections/DatabaseSingleMapped.java | 65 +++++-- .../database/collections/DatabaseStage.java | 9 +- .../collections/DatabaseStageEntry.java | 1 + .../collections/DatabaseStageMap.java | 82 +++----- .../collections/DatabaseStageWithEntry.java | 1 + .../collections/SubStageGetterHashMap.java | 18 +- .../collections/SubStageGetterHashSet.java | 9 +- .../collections/SubStageGetterMap.java | 7 +- .../collections/SubStageGetterMapDeep.java | 9 +- .../collections/SubStageGetterSet.java | 2 +- .../collections/SubStageGetterSingle.java | 7 +- .../database/disk/LLIndexSearcher.java | 3 +- .../database/disk/LLIndexSearchers.java | 5 +- .../dbengine/lucene/LuceneUtils.java | 6 +- .../lucene/searcher/LuceneSearchResult.java | 3 +- .../dbengine/netty/NullableBuffer.java | 3 +- .../it/cavallium/dbengine/DbTestUtils.java | 15 +- .../cavallium/dbengine/TestDictionaryMap.java | 49 +++-- .../dbengine/TestDictionaryMapDeep.java | 105 +++++----- .../TestDictionaryMapDeepHashMap.java | 4 +- 36 files changed, 576 insertions(+), 366 deletions(-) create mode 100644 src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResult.java b/src/main/java/it/cavallium/dbengine/client/SearchResult.java index efbca1c..0a9bec2 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResult.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResult.java @@ -5,13 +5,14 @@ import io.net5.buffer.api.Owned; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLSearchResultShard; +import it.cavallium.dbengine.database.LiveResourceSupport; import java.util.Objects; import org.warp.commonutils.log.Logger; import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SearchResult extends ResourceSupport, SearchResult> { +public final class SearchResult extends LiveResourceSupport, SearchResult> { private Flux> results; private TotalHitsCount totalHitsCount; diff --git a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java index f071a1c..7478031 100644 --- a/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java +++ b/src/main/java/it/cavallium/dbengine/client/SearchResultKeys.java @@ -5,6 +5,7 @@ import io.net5.buffer.api.Owned; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLSearchResultShard; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.cavallium.dbengine.database.collections.ValueGetter; import java.util.Objects; import org.reactivestreams.Publisher; @@ -14,7 +15,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @SuppressWarnings("unused") -public final class SearchResultKeys extends ResourceSupport, SearchResultKeys> { +public final class SearchResultKeys extends LiveResourceSupport, SearchResultKeys> { private static final Logger logger = LoggerFactory.getLogger(SearchResultKeys.class); diff --git a/src/main/java/it/cavallium/dbengine/database/LLDelta.java b/src/main/java/it/cavallium/dbengine/database/LLDelta.java index dbf445b..8052c56 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLDelta.java +++ b/src/main/java/it/cavallium/dbengine/database/LLDelta.java @@ -8,7 +8,7 @@ import io.net5.buffer.api.internal.ResourceSupport; import java.util.StringJoiner; import org.jetbrains.annotations.Nullable; -public class LLDelta extends ResourceSupport { +public class LLDelta extends LiveResourceSupport { @Nullable private final Buffer previous; @Nullable diff --git a/src/main/java/it/cavallium/dbengine/database/LLEntry.java b/src/main/java/it/cavallium/dbengine/database/LLEntry.java index e93552a..3413008 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLEntry.java +++ b/src/main/java/it/cavallium/dbengine/database/LLEntry.java @@ -9,7 +9,7 @@ import java.util.StringJoiner; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -public class LLEntry extends ResourceSupport { +public class LLEntry extends LiveResourceSupport { @NotNull private final Buffer key; @NotNull diff --git a/src/main/java/it/cavallium/dbengine/database/LLRange.java b/src/main/java/it/cavallium/dbengine/database/LLRange.java index ef6199e..664c249 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLRange.java +++ b/src/main/java/it/cavallium/dbengine/database/LLRange.java @@ -12,7 +12,7 @@ import java.util.StringJoiner; /** * Range of data, from min (inclusive),to max (exclusive) */ -public class LLRange extends ResourceSupport { +public class LLRange extends LiveResourceSupport { private static final LLRange RANGE_ALL = new LLRange(null, null, null, d -> {}); private Buffer min; @@ -212,9 +212,9 @@ public class LLRange extends ResourceSupport { @Override public void drop(LLRange obj) { - if (obj.min != null) obj.min.close(); - if (obj.max != null) obj.max.close(); - if (obj.single != null) obj.single.close(); + if (obj.min != null && obj.min.isAccessible()) obj.min.close(); + if (obj.max != null && obj.max.isAccessible()) obj.max.close(); + if (obj.single != null && obj.single.isAccessible()) obj.single.close(); delegate.drop(obj); } } diff --git a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java index ad07bec..2ffdbfc 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java +++ b/src/main/java/it/cavallium/dbengine/database/LLSearchResultShard.java @@ -9,7 +9,7 @@ import org.warp.commonutils.log.Logger; import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; -public final class LLSearchResultShard extends ResourceSupport { +public final class LLSearchResultShard extends LiveResourceSupport { private static final Logger logger = LoggerFactory.getLogger(LLSearchResultShard.class); diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 5b572ff..16f38cb 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -433,6 +433,25 @@ public class LLUtils { .doOnDiscard(Send.class, send -> send.close()); } + /** + * cleanup resource + * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful + */ + public static , V extends T> Flux usingEachResource(Flux resourceSupplier, + Function> resourceClosure, + boolean cleanupOnSuccess) { + return resourceSupplier + .concatMap(resource -> Mono.usingWhen(Mono.just(resource), resourceClosure, r -> { + if (cleanupOnSuccess) { + return Mono.fromRunnable(() -> r.close()); + } else { + return Mono.empty(); + } + }, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close()))) + .doOnDiscard(Resource.class, resource -> resource.close()) + .doOnDiscard(Send.class, send -> send.close()); + } + /** * cleanup resource * @param cleanupOnSuccess if true the resource will be cleaned up if the function is successful @@ -936,7 +955,9 @@ public class LLUtils { } private static void discardStage(DatabaseStage stage) { - stage.release(); + if (stage != null && stage.isAccessible()) { + stage.close(); + } } public static boolean isDirect(Buffer key) { diff --git a/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java b/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java new file mode 100644 index 0000000..88b1c36 --- /dev/null +++ b/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java @@ -0,0 +1,33 @@ +package it.cavallium.dbengine.database; + +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Resource; +import io.net5.buffer.api.internal.LifecycleTracer; +import io.net5.buffer.api.internal.ResourceSupport; +import org.warp.commonutils.log.Logger; +import org.warp.commonutils.log.LoggerFactory; + +public abstract class LiveResourceSupport, T extends LiveResourceSupport> extends ResourceSupport { + + private static final Logger logger = LoggerFactory.getLogger(LiveResourceSupport.class); + + protected LiveResourceSupport(Drop drop) { + super(drop); + } + + @Override + protected void finalize() throws Throwable { + if (this.isAccessible()) { + try { + this.close(); + } catch (Throwable ignored) { + } finally { + var ise = new IllegalStateException("Resource not released"); + ise.setStackTrace(new StackTraceElement[0]); + logger.error("Resource not released: {}", this, attachTrace(ise)); + } + } + super.finalize(); + } +} diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java index 6769af6..cdd9dbd 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseEmpty.java @@ -2,7 +2,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; -import io.net5.buffer.api.CompositeBuffer; +import io.net5.buffer.api.Drop; import io.net5.buffer.api.Send; import it.cavallium.dbengine.database.LLDictionary; import it.cavallium.dbengine.database.LLUtils; @@ -36,8 +36,10 @@ public class DatabaseEmpty { private DatabaseEmpty() { } - public static DatabaseStageEntry create(LLDictionary dictionary, Send key) { - return new DatabaseSingle<>(dictionary, key, nothingSerializer(dictionary.getAllocator())); + public static DatabaseStageEntry create(LLDictionary dictionary, + Send key, + Drop> drop) { + return new DatabaseSingle<>(dictionary, key, nothingSerializer(dictionary.getAllocator()), drop); } public static final class Nothing { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java index fd931c7..5ee7a61 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionary.java @@ -1,6 +1,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.CompositeSnapshot; @@ -42,23 +43,27 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep prefixKey, SerializerFixedBinaryLength keySuffixSerializer, - Serializer valueSerializer) { + Serializer valueSerializer, + Drop>> drop) { // Do not retain or release or use the prefixKey here - super(dictionary, prefixKey, keySuffixSerializer, new SubStageGetterSingle<>(valueSerializer), 0); + super(dictionary, prefixKey, keySuffixSerializer, new SubStageGetterSingle<>(valueSerializer), 0, drop); this.valueSerializer = valueSerializer; } public static DatabaseMapDictionary simple(LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, - Serializer valueSerializer) { - return new DatabaseMapDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, valueSerializer); + Serializer valueSerializer, + Drop>> drop) { + return new DatabaseMapDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, + valueSerializer, drop); } public static DatabaseMapDictionary tail(LLDictionary dictionary, Send prefixKey, SerializerFixedBinaryLength keySuffixSerializer, - Serializer valueSerializer) { - return new DatabaseMapDictionary<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer); + Serializer valueSerializer, + Drop>> drop) { + return new DatabaseMapDictionary<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer, drop); } private Send toKey(Send suffixKeyToSend) { @@ -147,7 +152,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep> at(@Nullable CompositeSnapshot snapshot, T keySuffix) { return Mono.fromCallable(() -> - new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix)), valueSerializer)); + new DatabaseSingle<>(dictionary, toKey(serializeSuffix(keySuffix)), valueSerializer, d -> {})); } @Override @@ -396,10 +401,7 @@ public class DatabaseMapDictionary extends DatabaseMapDictionaryDeep(dictionary, - toKey(keyBuf.send()), - valueSerializer - ) + new DatabaseSingle<>(dictionary, toKey(keyBuf.send()), valueSerializer, d -> {}) )); } catch (SerializationException ex) { sink.error(ex); diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java index baf3492..b62afdc 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryDeep.java @@ -2,8 +2,11 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; import io.net5.util.IllegalReferenceCountException; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; @@ -12,6 +15,7 @@ import it.cavallium.dbengine.database.LLDictionaryResultType; import it.cavallium.dbengine.database.LLRange; import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.cavallium.dbengine.database.UpdateMode; import it.cavallium.dbengine.database.serialization.SerializationException; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; @@ -24,20 +28,21 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; // todo: implement optimized methods (which?) -public class DatabaseMapDictionaryDeep> implements DatabaseStageMap { +public class DatabaseMapDictionaryDeep> extends + LiveResourceSupport>, DatabaseMapDictionaryDeep> + implements DatabaseStageMap { protected final LLDictionary dictionary; private final BufferAllocator alloc; protected final SubStageGetter subStageGetter; protected final SerializerFixedBinaryLength keySuffixSerializer; - @NotNull - protected final Buffer keyPrefix; protected final int keyPrefixLength; protected final int keySuffixLength; protected final int keyExtLength; - protected final LLRange range; protected final Mono> rangeMono; - private volatile boolean released; + + protected LLRange range; + protected Buffer keyPrefix; private static void incrementPrefix(Buffer prefix, int prefixLength) { assert prefix.readableBytes() >= prefixLength; @@ -71,18 +76,12 @@ public class DatabaseMapDictionaryDeep> implem } } - static Buffer firstRangeKey(BufferAllocator alloc, - Send prefixKey, - int prefixLength, - int suffixLength, + static Buffer firstRangeKey(BufferAllocator alloc, Send prefixKey, int prefixLength, int suffixLength, int extLength) { return zeroFillKeySuffixAndExt(alloc, prefixKey, prefixLength, suffixLength, extLength); } - static Buffer nextRangeKey(BufferAllocator alloc, - Send prefixKey, - int prefixLength, - int suffixLength, + static Buffer nextRangeKey(BufferAllocator alloc, Send prefixKey, int prefixLength, int suffixLength, int extLength) { try (prefixKey) { Buffer nonIncremented = zeroFillKeySuffixAndExt(alloc, prefixKey, prefixLength, suffixLength, extLength); @@ -91,11 +90,8 @@ public class DatabaseMapDictionaryDeep> implem } } - protected static Buffer zeroFillKeySuffixAndExt(BufferAllocator alloc, - @NotNull Send prefixKeySend, - int prefixLength, - int suffixLength, - int extLength) { + protected static Buffer zeroFillKeySuffixAndExt(BufferAllocator alloc, @NotNull Send prefixKeySend, + int prefixLength, int suffixLength, int extLength) { var result = prefixKeySend.receive(); if (result == null) { assert prefixLength == 0; @@ -115,41 +111,20 @@ public class DatabaseMapDictionaryDeep> implem } } - static Buffer firstRangeKey( - BufferAllocator alloc, - Send prefixKey, - Send suffixKey, - int prefixLength, - int suffixLength, - int extLength) { + static Buffer firstRangeKey(BufferAllocator alloc, Send prefixKey, Send suffixKey, int prefixLength, + int suffixLength, int extLength) { return zeroFillKeyExt(alloc, prefixKey, suffixKey, prefixLength, suffixLength, extLength); } - static Buffer nextRangeKey( - BufferAllocator alloc, - Send prefixKey, - Send suffixKey, - int prefixLength, - int suffixLength, - int extLength) { - Buffer nonIncremented = zeroFillKeyExt(alloc, - prefixKey, - suffixKey, - prefixLength, - suffixLength, - extLength - ); + static Buffer nextRangeKey(BufferAllocator alloc, Send prefixKey, Send suffixKey, int prefixLength, + int suffixLength, int extLength) { + Buffer nonIncremented = zeroFillKeyExt(alloc, prefixKey, suffixKey, prefixLength, suffixLength, extLength); incrementPrefix(nonIncremented, prefixLength + suffixLength); return nonIncremented; } - protected static Buffer zeroFillKeyExt( - BufferAllocator alloc, - Send prefixKeySend, - Send suffixKeySend, - int prefixLength, - int suffixLength, - int extLength) { + protected static Buffer zeroFillKeyExt(BufferAllocator alloc, Send prefixKeySend, Send suffixKeySend, + int prefixLength, int suffixLength, int extLength) { try (var prefixKey = prefixKeySend.receive()) { try (var suffixKey = suffixKeySend.receive()) { assert prefixKey.readableBytes() == prefixLength; @@ -174,36 +149,30 @@ public class DatabaseMapDictionaryDeep> implem */ @Deprecated public static DatabaseMapDictionaryDeep> simple(LLDictionary dictionary, - SerializerFixedBinaryLength keySerializer, - SubStageGetterSingle subStageGetter) { - return new DatabaseMapDictionaryDeep<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, subStageGetter, 0); + SerializerFixedBinaryLength keySerializer, SubStageGetterSingle subStageGetter, + Drop>> drop) { + return new DatabaseMapDictionaryDeep<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, + subStageGetter, 0, drop); } - public static > DatabaseMapDictionaryDeep deepTail(LLDictionary dictionary, - SerializerFixedBinaryLength keySerializer, - int keyExtLength, - SubStageGetter subStageGetter) { - return new DatabaseMapDictionaryDeep<>(dictionary, - LLUtils.empty(dictionary.getAllocator()), - keySerializer, - subStageGetter, - keyExtLength - ); + public static > DatabaseMapDictionaryDeep deepTail( + LLDictionary dictionary, SerializerFixedBinaryLength keySerializer, int keyExtLength, + SubStageGetter subStageGetter, Drop> drop) { + return new DatabaseMapDictionaryDeep<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, + subStageGetter, keyExtLength, drop); } - public static > DatabaseMapDictionaryDeep deepIntermediate(LLDictionary dictionary, - Send prefixKey, - SerializerFixedBinaryLength keySuffixSerializer, - SubStageGetter subStageGetter, - int keyExtLength) { - return new DatabaseMapDictionaryDeep<>(dictionary, prefixKey, keySuffixSerializer, subStageGetter, keyExtLength); + public static > DatabaseMapDictionaryDeep deepIntermediate( + LLDictionary dictionary, Send prefixKey, SerializerFixedBinaryLength keySuffixSerializer, + SubStageGetter subStageGetter, int keyExtLength, Drop> drop) { + return new DatabaseMapDictionaryDeep<>(dictionary, prefixKey, keySuffixSerializer, subStageGetter, + keyExtLength, drop); } - protected DatabaseMapDictionaryDeep(LLDictionary dictionary, - @NotNull Send prefixKeyToReceive, - SerializerFixedBinaryLength keySuffixSerializer, - SubStageGetter subStageGetter, - int keyExtLength) { + protected DatabaseMapDictionaryDeep(LLDictionary dictionary, @NotNull Send prefixKeyToReceive, + SerializerFixedBinaryLength keySuffixSerializer, SubStageGetter subStageGetter, int keyExtLength, + Drop> drop) { + super(new CloseOnDrop<>(drop)); try (var prefixKey = prefixKeyToReceive.receive()) { this.dictionary = dictionary; this.alloc = dictionary.getAllocator(); @@ -231,6 +200,31 @@ public class DatabaseMapDictionaryDeep> implem } } + private DatabaseMapDictionaryDeep(LLDictionary dictionary, + BufferAllocator alloc, + SubStageGetter subStageGetter, + SerializerFixedBinaryLength keySuffixSerializer, + int keyPrefixLength, + int keySuffixLength, + int keyExtLength, + Mono> rangeMono, + Send range, + Send keyPrefix, + Drop> drop) { + super(new CloseOnDrop<>(drop)); + this.dictionary = dictionary; + this.alloc = alloc; + this.subStageGetter = subStageGetter; + this.keySuffixSerializer = keySuffixSerializer; + this.keyPrefixLength = keyPrefixLength; + this.keySuffixLength = keySuffixLength; + this.keyExtLength = keyExtLength; + this.rangeMono = rangeMono; + + this.range = range.receive(); + this.keyPrefix = keyPrefix.receive(); + } + @SuppressWarnings("unused") protected boolean suffixKeyConsistency(int keySuffixLength) { return this.keySuffixLength == keySuffixLength; @@ -301,7 +295,7 @@ public class DatabaseMapDictionaryDeep> implem return this.subStageGetter .subStage(dictionary, snapshot, suffixKeyWithoutExt) .transform(LLUtils::handleDiscard) - .doOnDiscard(DatabaseStage.class, DatabaseStage::release); + .doOnDiscard(DatabaseStage.class, DatabaseStage::close); } @Override @@ -415,13 +409,42 @@ public class DatabaseMapDictionaryDeep> implem } @Override - public void release() { - if (!released) { - released = true; - this.range.close(); - this.keyPrefix.close(); - } else { - throw new IllegalReferenceCountException(0, -1); + protected RuntimeException createResourceClosedException() { + throw new IllegalStateException("Closed"); + } + + @Override + protected Owned> prepareSend() { + var keyPrefix = this.keyPrefix.send(); + var range = this.range.send(); + return drop -> new DatabaseMapDictionaryDeep<>(dictionary, alloc, subStageGetter, keySuffixSerializer, + keyPrefixLength, keySuffixLength, keyExtLength, rangeMono, range, keyPrefix, drop); + } + + @Override + protected void makeInaccessible() { + this.keyPrefix = null; + this.range = null; + } + + private static class CloseOnDrop> implements + Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(DatabaseMapDictionaryDeep obj) { + if (obj.range != null) { + obj.range.close(); + } + if (obj.keyPrefix != null) { + obj.keyPrefix.close(); + } + delegate.drop(obj); } } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java index 66815c3..daec688 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseMapDictionaryHashed.java @@ -2,11 +2,14 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; import io.net5.buffer.api.BufferAllocator; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.cavallium.dbengine.database.UpdateMode; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; @@ -24,18 +27,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @SuppressWarnings("unused") -public class DatabaseMapDictionaryHashed implements DatabaseStageMap> { +public class DatabaseMapDictionaryHashed extends + LiveResourceSupport>, DatabaseMapDictionaryHashed> + implements DatabaseStageMap> { private final BufferAllocator alloc; - private final DatabaseMapDictionary>> subDictionary; private final Function keySuffixHashFunction; + private DatabaseMapDictionary>> subDictionary; + protected DatabaseMapDictionaryHashed(LLDictionary dictionary, @NotNull Send prefixKey, Serializer keySuffixSerializer, Serializer valueSerializer, Function keySuffixHashFunction, - SerializerFixedBinaryLength keySuffixHashSerializer) { + SerializerFixedBinaryLength keySuffixHashSerializer, + Drop> drop) { + super(new DatabaseMapDictionaryHashed.CloseOnDrop<>(drop)); if (dictionary.getUpdateMode().block() != UpdateMode.ALLOW) { throw new IllegalArgumentException("Hashed maps only works when UpdateMode is ALLOW"); } @@ -44,26 +52,36 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap(alloc, keySuffixSerializer, valueSerializer); ValuesSetSerializer> valuesSetSerializer = new ValuesSetSerializer<>(alloc, valueWithHashSerializer); - this.subDictionary = DatabaseMapDictionary.tail(dictionary, - prefixKey, - keySuffixHashSerializer, - valuesSetSerializer - ); + this.subDictionary = DatabaseMapDictionary.tail(dictionary, prefixKey, keySuffixHashSerializer, + valuesSetSerializer, d -> {}); this.keySuffixHashFunction = keySuffixHashFunction; } + private DatabaseMapDictionaryHashed(BufferAllocator alloc, + Function keySuffixHashFunction, + Send>>>> subDictionary, + Drop> drop) { + super(new CloseOnDrop<>(drop)); + this.alloc = alloc; + this.keySuffixHashFunction = keySuffixHashFunction; + + this.subDictionary = (DatabaseMapDictionary>>) subDictionary.receive(); + } + public static DatabaseMapDictionaryHashed simple(LLDictionary dictionary, Serializer keySerializer, Serializer valueSerializer, Function keyHashFunction, - SerializerFixedBinaryLength keyHashSerializer) { + SerializerFixedBinaryLength keyHashSerializer, + Drop> drop) { return new DatabaseMapDictionaryHashed<>( dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, valueSerializer, keyHashFunction, - keyHashSerializer + keyHashSerializer, + drop ); } @@ -72,13 +90,15 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap keySuffixSerializer, Serializer valueSerializer, Function keySuffixHashFunction, - SerializerFixedBinaryLength keySuffixHashSerializer) { + SerializerFixedBinaryLength keySuffixHashSerializer, + Drop> drop) { return new DatabaseMapDictionaryHashed<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer, keySuffixHashFunction, - keySuffixHashSerializer + keySuffixHashSerializer, + drop ); } @@ -125,11 +145,6 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap close() { - return subDictionary.close(); - } - @Override public Mono isEmpty(@Nullable CompositeSnapshot snapshot) { return subDictionary.isEmpty(snapshot); @@ -145,11 +160,6 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap> at(@Nullable CompositeSnapshot snapshot, T key) { return this @@ -160,7 +170,7 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap> atPrivate(@Nullable CompositeSnapshot snapshot, T key, TH hash) { return subDictionary .at(snapshot, hash) - .map(entry -> new DatabaseSingleBucket<>(entry, key)); + .map(entry -> new DatabaseSingleBucket<>(entry, key, d -> {})); } @Override @@ -193,13 +203,11 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap> setAllValuesAndGetPrevious(Flux> entries) { return entries - .flatMap(entry -> Flux.usingWhen( - this.at(null, entry.getKey()), + .flatMap(entry -> LLUtils.usingResource(this.at(null, entry.getKey()), stage -> stage .setAndGetPrevious(entry.getValue()) - .map(prev -> Map.entry(entry.getKey(), prev)), - stage -> Mono.fromRunnable(stage::release) - )); + .map(prev -> Map.entry(entry.getKey(), prev)), true) + ); } @Override @@ -297,4 +305,37 @@ public class DatabaseMapDictionaryHashed implements DatabaseStageMap> prepareSend() { + var subDictionary = this.subDictionary.send(); + return drop -> new DatabaseMapDictionaryHashed<>(alloc, keySuffixHashFunction, subDictionary, drop); + } + + @Override + protected void makeInaccessible() { + this.subDictionary = null; + } + + private static class CloseOnDrop implements Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(DatabaseMapDictionaryHashed obj) { + if (obj.subDictionary != null) { + obj.subDictionary.close(); + } + delegate.drop(obj); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java index 260ff51..c082547 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionary.java @@ -1,6 +1,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; @@ -18,19 +19,22 @@ public class DatabaseSetDictionary extends DatabaseMapDictionary protected DatabaseSetDictionary(LLDictionary dictionary, Send prefixKey, - SerializerFixedBinaryLength keySuffixSerializer) { - super(dictionary, prefixKey, keySuffixSerializer, DatabaseEmpty.nothingSerializer(dictionary.getAllocator())); + SerializerFixedBinaryLength keySuffixSerializer, + Drop>> drop) { + super(dictionary, prefixKey, keySuffixSerializer, DatabaseEmpty.nothingSerializer(dictionary.getAllocator()), drop); } public static DatabaseSetDictionary simple(LLDictionary dictionary, - SerializerFixedBinaryLength keySerializer) { - return new DatabaseSetDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer); + SerializerFixedBinaryLength keySerializer, + Drop>> drop) { + return new DatabaseSetDictionary<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, drop); } public static DatabaseSetDictionary tail(LLDictionary dictionary, Send prefixKey, - SerializerFixedBinaryLength keySuffixSerializer) { - return new DatabaseSetDictionary<>(dictionary, prefixKey, keySuffixSerializer); + SerializerFixedBinaryLength keySuffixSerializer, + Drop>> drop) { + return new DatabaseSetDictionary<>(dictionary, prefixKey, keySuffixSerializer, drop); } public Mono> getKeySet(@Nullable CompositeSnapshot snapshot) { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java index 16a9f3d..68bf230 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSetDictionaryHashed.java @@ -1,6 +1,7 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; @@ -23,25 +24,29 @@ public class DatabaseSetDictionaryHashed extends DatabaseMapDictionaryHas @NotNull Send prefixKey, Serializer keySuffixSerializer, Function keySuffixHashFunction, - SerializerFixedBinaryLength keySuffixHashSerializer) { + SerializerFixedBinaryLength keySuffixHashSerializer, + Drop> drop) { super(dictionary, prefixKey, keySuffixSerializer, DatabaseEmpty.nothingSerializer(dictionary.getAllocator()), keySuffixHashFunction, - keySuffixHashSerializer + keySuffixHashSerializer, + drop ); } public static DatabaseSetDictionaryHashed simple(LLDictionary dictionary, Serializer keySerializer, Function keyHashFunction, - SerializerFixedBinaryLength keyHashSerializer) { + SerializerFixedBinaryLength keyHashSerializer, + Drop> drop) { return new DatabaseSetDictionaryHashed<>(dictionary, LLUtils.empty(dictionary.getAllocator()), keySerializer, keyHashFunction, - keyHashSerializer + keyHashSerializer, + drop ); } @@ -49,12 +54,13 @@ public class DatabaseSetDictionaryHashed extends DatabaseMapDictionaryHas Send prefixKey, Serializer keySuffixSerializer, Function keyHashFunction, - SerializerFixedBinaryLength keyHashSerializer) { + SerializerFixedBinaryLength keyHashSerializer, Drop> drop) { return new DatabaseSetDictionaryHashed<>(dictionary, prefixKey, keySuffixSerializer, keyHashFunction, - keyHashSerializer + keyHashSerializer, + drop ); } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingle.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingle.java index 8dcd6ab..05b9f6e 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingle.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingle.java @@ -1,6 +1,8 @@ package it.cavallium.dbengine.database.collections; import io.net5.buffer.api.Buffer; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.BadBlock; @@ -20,14 +22,18 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.publisher.SynchronousSink; -public class DatabaseSingle implements DatabaseStageEntry { +public class DatabaseSingle extends ResourceSupport, DatabaseSingle> implements + DatabaseStageEntry { private final LLDictionary dictionary; - private final Buffer key; private final Mono> keyMono; private final Serializer serializer; - public DatabaseSingle(LLDictionary dictionary, Send key, Serializer serializer) { + private Buffer key; + + public DatabaseSingle(LLDictionary dictionary, Send key, Serializer serializer, + Drop> drop) { + super(new CloseOnDrop<>(drop)); try (key) { this.dictionary = dictionary; this.key = key.receive(); @@ -124,13 +130,41 @@ public class DatabaseSingle implements DatabaseStageEntry { .isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send)); } - @Override - public void release() { - key.close(); - } - @Override public Flux badBlocks() { return dictionary.badBlocks(keyMono.map(LLRange::single).map(ResourceSupport::send)); } + + @Override + protected RuntimeException createResourceClosedException() { + throw new IllegalStateException("Closed"); + } + + @Override + protected Owned> prepareSend() { + var key = this.key.send(); + return drop -> new DatabaseSingle<>(dictionary, key, serializer, drop); + } + + @Override + protected void makeInaccessible() { + this.key = null; + } + + private static class CloseOnDrop implements Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(DatabaseSingle obj) { + if (obj.key != null) { + obj.key.close(); + } + delegate.drop(obj); + } + } } \ No newline at end of file diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleBucket.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleBucket.java index a60934d..b3e577f 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleBucket.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleBucket.java @@ -1,10 +1,14 @@ package it.cavallium.dbengine.database.collections; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.Column; import it.cavallium.dbengine.database.Delta; import it.cavallium.dbengine.database.LLUtils; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.cavallium.dbengine.database.UpdateReturnMode; import it.cavallium.dbengine.database.serialization.SerializationFunction; import it.unimi.dsi.fastutil.objects.ObjectArraySet; @@ -23,14 +27,26 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @SuppressWarnings("unused") -public class DatabaseSingleBucket implements DatabaseStageEntry { +public class DatabaseSingleBucket + extends LiveResourceSupport, DatabaseSingleBucket> + implements DatabaseStageEntry { - private final DatabaseStageEntry>> bucketStage; private final K key; - public DatabaseSingleBucket(DatabaseStageEntry>> bucketStage, K key) { - this.bucketStage = bucketStage; + private DatabaseStageEntry>> bucketStage; + + public DatabaseSingleBucket(DatabaseStageEntry>> bucketStage, K key, + Drop> drop) { + super(new CloseOnDrop<>(drop)); this.key = key; + this.bucketStage = bucketStage; + } + + private DatabaseSingleBucket(Send>>> bucketStage, K key, + Drop> drop) { + super(new CloseOnDrop<>(drop)); + this.key = key; + this.bucketStage = (DatabaseStageEntry>>) bucketStage.receive(); } @Override @@ -77,7 +93,8 @@ public class DatabaseSingleBucket implements DatabaseStageEntry { } @Override - public Mono> updateAndGetDelta(SerializationFunction<@Nullable V, @Nullable V> updater, boolean existsAlmostCertainly) { + public Mono> updateAndGetDelta(SerializationFunction<@Nullable V, @Nullable V> updater, + boolean existsAlmostCertainly) { return bucketStage .updateAndGetDelta(oldBucket -> { V oldValue = extractValue(oldBucket); @@ -106,11 +123,6 @@ public class DatabaseSingleBucket implements DatabaseStageEntry { return this.updateAndGetDelta(prev -> null).map(LLUtils::isDeltaChanged); } - @Override - public Mono close() { - return bucketStage.close(); - } - @Override public Mono leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) { return this.get(snapshot).map(prev -> 1L).defaultIfEmpty(0L); @@ -131,11 +143,6 @@ public class DatabaseSingleBucket implements DatabaseStageEntry { return bucketStage.badBlocks(); } - @Override - public void release() { - bucketStage.release(); - } - private Mono extractValueTransformation(Set> entries) { return Mono.fromCallable(() -> extractValue(entries)); } @@ -193,4 +200,38 @@ public class DatabaseSingleBucket implements DatabaseStageEntry { return null; } } + + @Override + protected RuntimeException createResourceClosedException() { + throw new IllegalStateException("Closed"); + } + + @Override + protected Owned> prepareSend() { + var bucketStage = this.bucketStage.send(); + return drop -> new DatabaseSingleBucket<>(bucketStage, key, drop); + } + + @Override + protected void makeInaccessible() { + this.bucketStage = null; + } + + private static class CloseOnDrop implements + Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(DatabaseSingleBucket obj) { + if (obj.bucketStage != null) { + obj.bucketStage.close(); + } + delegate.drop(obj); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleMapped.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleMapped.java index c17fcc9..935c0e4 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleMapped.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseSingleMapped.java @@ -1,5 +1,9 @@ package it.cavallium.dbengine.database.collections; +import io.net5.buffer.api.Drop; +import io.net5.buffer.api.Owned; +import io.net5.buffer.api.Send; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.client.Mapper; @@ -14,16 +18,28 @@ import reactor.core.publisher.Mono; import reactor.core.publisher.SynchronousSink; @SuppressWarnings("unused") -public class DatabaseSingleMapped implements DatabaseStageEntry { +public class DatabaseSingleMapped extends ResourceSupport, DatabaseSingleMapped> + implements DatabaseStageEntry { - private final DatabaseStageEntry serializedSingle; private final Mapper mapper; - public DatabaseSingleMapped(DatabaseStageEntry serializedSingle, Mapper mapper) { + private DatabaseStageEntry serializedSingle; + + public DatabaseSingleMapped(DatabaseStageEntry serializedSingle, Mapper mapper, + Drop> drop) { + super(new CloseOnDrop<>(drop)); this.serializedSingle = serializedSingle; this.mapper = mapper; } + private DatabaseSingleMapped(Send> serializedSingle, Mapper mapper, + Drop> drop) { + super(new CloseOnDrop<>(drop)); + this.mapper = mapper; + + this.serializedSingle = (DatabaseStageEntry) serializedSingle.receive(); + } + private void deserializeSink(B value, SynchronousSink sink) { try { sink.next(this.unMap(value)); @@ -107,11 +123,6 @@ public class DatabaseSingleMapped implements DatabaseStageEntry { return serializedSingle.clearAndGetStatus(); } - @Override - public Mono close() { - return serializedSingle.close(); - } - @Override public Mono leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) { return serializedSingle.leavesCount(snapshot, fast); @@ -132,11 +143,6 @@ public class DatabaseSingleMapped implements DatabaseStageEntry { return this.serializedSingle.badBlocks(); } - @Override - public void release() { - serializedSingle.release(); - } - //todo: temporary wrapper. convert the whole class to buffers private A unMap(B bytes) throws SerializationException { return mapper.unmap(bytes); @@ -146,4 +152,37 @@ public class DatabaseSingleMapped implements DatabaseStageEntry { private B map(A bytes) throws SerializationException { return mapper.map(bytes); } + + @Override + protected RuntimeException createResourceClosedException() { + throw new IllegalStateException("Closed"); + } + + @Override + protected Owned> prepareSend() { + var serializedSingle = this.serializedSingle.send(); + return drop -> new DatabaseSingleMapped<>(serializedSingle, mapper, drop); + } + + @Override + protected void makeInaccessible() { + this.serializedSingle = null; + } + + private static class CloseOnDrop implements Drop> { + + private final Drop> delegate; + + public CloseOnDrop(Drop> drop) { + this.delegate = drop; + } + + @Override + public void drop(DatabaseSingleMapped obj) { + if (obj.serializedSingle != null) { + obj.serializedSingle.close(); + } + delegate.drop(obj); + } + } } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStage.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStage.java index acd160b..444773b 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStage.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStage.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database.collections; +import io.net5.buffer.api.Resource; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.Delta; @@ -12,7 +13,7 @@ import org.jetbrains.annotations.Nullable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public interface DatabaseStage extends DatabaseStageWithEntry { +public interface DatabaseStage extends DatabaseStageWithEntry, Resource> { default Mono get(@Nullable CompositeSnapshot snapshot) { return get(snapshot, false); @@ -74,12 +75,6 @@ public interface DatabaseStage extends DatabaseStageWithEntry { return clearAndGetPrevious().map(Objects::nonNull).defaultIfEmpty(false); } - void release(); - - default Mono close() { - return Mono.empty(); - } - /** * Count all the elements. * If it's a nested collection the count will include all the children recursively diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageEntry.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageEntry.java index 0e09d66..c5df267 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageEntry.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageEntry.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database.collections; +import io.net5.buffer.api.Resource; import it.cavallium.dbengine.client.BadBlock; import reactor.core.publisher.Flux; diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java index fdc4c2e..6588646 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageMap.java @@ -34,11 +34,8 @@ public interface DatabaseStageMap> extends Dat Mono at(@Nullable CompositeSnapshot snapshot, T key); default Mono getValue(@Nullable CompositeSnapshot snapshot, T key, boolean existsAlmostCertainly) { - return Mono.usingWhen( - this.at(snapshot, key), - stage -> stage.get(snapshot, existsAlmostCertainly), - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(this.at(snapshot, key), + stage -> stage.get(snapshot, existsAlmostCertainly), true); } default Mono getValue(@Nullable CompositeSnapshot snapshot, T key) { @@ -50,11 +47,8 @@ public interface DatabaseStageMap> extends Dat } default Mono putValue(T key, U value) { - return Mono.usingWhen( - at(null, key).single(), - stage -> stage.set(value), - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(at(null, key).single(), + stage -> stage.set(value), true); } Mono getUpdateMode(); @@ -63,11 +57,8 @@ public interface DatabaseStageMap> extends Dat UpdateReturnMode updateReturnMode, boolean existsAlmostCertainly, SerializationFunction<@Nullable U, @Nullable U> updater) { - return Mono.usingWhen( - this.at(null, key).single(), - stage -> stage.update(updater, updateReturnMode, existsAlmostCertainly), - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(this.at(null, key).single(), + stage -> stage.update(updater, updateReturnMode, existsAlmostCertainly), true); } default Flux> updateMulti(Flux> entries, @@ -94,11 +85,8 @@ public interface DatabaseStageMap> extends Dat default Mono> updateValueAndGetDelta(T key, boolean existsAlmostCertainly, SerializationFunction<@Nullable U, @Nullable U> updater) { - return Mono.usingWhen( - this.at(null, key).single(), - stage -> stage.updateAndGetDelta(updater, existsAlmostCertainly), - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(this.at(null, key).single(), + stage -> stage.updateAndGetDelta(updater, existsAlmostCertainly), true); } default Mono> updateValueAndGetDelta(T key, SerializationFunction<@Nullable U, @Nullable U> updater) { @@ -106,22 +94,14 @@ public interface DatabaseStageMap> extends Dat } default Mono putValueAndGetPrevious(T key, U value) { - return Mono.usingWhen( - at(null, key).single(), - stage -> stage.setAndGetPrevious(value), - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(at(null, key).single(), stage -> stage.setAndGetPrevious(value), true); } /** * @return true if the key was associated with any value, false if the key didn't exist. */ default Mono putValueAndGetChanged(T key, U value) { - return Mono.usingWhen( - at(null, key).single(), - stage -> stage.setAndGetChanged(value), - stage -> Mono.fromRunnable(stage::release) - ).single(); + return LLUtils.usingResource(at(null, key).single(), stage -> stage.setAndGetChanged(value), true).single(); } default Mono remove(T key) { @@ -129,11 +109,7 @@ public interface DatabaseStageMap> extends Dat } default Mono removeAndGetPrevious(T key) { - return Mono.usingWhen( - at(null, key), - DatabaseStage::clearAndGetPrevious, - stage -> Mono.fromRunnable(stage::release) - ); + return LLUtils.usingResource(at(null, key), DatabaseStage::clearAndGetPrevious, true); } default Mono removeAndGetStatus(T key) { @@ -175,11 +151,11 @@ public interface DatabaseStageMap> extends Dat default Flux> getAllValues(@Nullable CompositeSnapshot snapshot) { return this .getAllStages(snapshot) - .flatMapSequential(entry -> entry + .flatMapSequential(stage -> stage .getValue() .get(snapshot, true) - .map(value -> Map.entry(entry.getKey(), value)) - .doAfterTerminate(() -> entry.getValue().release()) + .map(value -> Map.entry(stage.getKey(), value)) + .doFinally(s -> stage.getValue().close()) ); } @@ -193,7 +169,8 @@ public interface DatabaseStageMap> extends Dat return setAllValues(Flux.empty()); } - default Mono replaceAllValues(boolean canKeysChange, Function, Mono>> entriesReplacer) { + default Mono replaceAllValues(boolean canKeysChange, Function, + Mono>> entriesReplacer) { if (canKeysChange) { return this.setAllValues(this.getAllValues(null).flatMap(entriesReplacer)).then(); } else { @@ -202,7 +179,11 @@ public interface DatabaseStageMap> extends Dat .flatMap(entriesReplacer) .flatMap(replacedEntry -> this .at(null, replacedEntry.getKey()) - .flatMap(v -> v.set(replacedEntry.getValue()).doAfterTerminate(v::release))) + .flatMap(stage -> stage + .set(replacedEntry.getValue()) + .doFinally(s -> stage.close()) + ) + ) .then(); } } @@ -210,9 +191,8 @@ public interface DatabaseStageMap> extends Dat default Mono replaceAll(Function, Mono> entriesReplacer) { return this .getAllStages(null) - .flatMap(stage -> Mono - .defer(() -> entriesReplacer.apply(stage)) - .doAfterTerminate(() -> stage.getValue().release()) + .flatMap(stage -> entriesReplacer.apply(stage) + .doFinally(s -> stage.getValue().close()) ) .then(); } @@ -221,14 +201,15 @@ public interface DatabaseStageMap> extends Dat default Mono> setAndGetPrevious(Map value) { return this .setAllValuesAndGetPrevious(Flux.fromIterable(Map.copyOf(value).entrySet())) - .collectMap(Entry::getKey, Entry::getValue, HashMap::new); + .collectMap(Entry::getKey, Entry::getValue, HashMap::new) + .filter(map -> !map.isEmpty()); } @Override default Mono setAndGetChanged(Map value) { return this .setAndGetPrevious(value) - .map(oldValue -> !Objects.equals(oldValue, value)) + .map(oldValue -> !Objects.equals(oldValue, value.isEmpty() ? null : value)) .switchIfEmpty(Mono.fromSupplier(() -> !value.isEmpty())); } @@ -286,18 +267,17 @@ public interface DatabaseStageMap> extends Dat @Override default Mono> get(@Nullable CompositeSnapshot snapshot, boolean existsAlmostCertainly) { - return getAllValues(snapshot) + return this + .getAllValues(snapshot) .collectMap(Entry::getKey, Entry::getValue, HashMap::new) .filter(map -> !map.isEmpty()); } @Override default Mono leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) { - return getAllStages(snapshot) - .flatMap(stage -> Mono - .fromRunnable(() -> stage.getValue().release()) - .thenReturn(true) - ) + return this + .getAllStages(snapshot) + .doOnNext(stage -> stage.getValue().close()) .count(); } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageWithEntry.java b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageWithEntry.java index 5e48507..3d7106c 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageWithEntry.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/DatabaseStageWithEntry.java @@ -1,5 +1,6 @@ package it.cavallium.dbengine.database.collections; +import io.net5.buffer.api.Resource; import it.cavallium.dbengine.client.BadBlock; import reactor.core.publisher.Mono; diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashMap.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashMap.java index 48e6add..38da00b 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashMap.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashMap.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.util.Map; @@ -34,20 +35,9 @@ public class SubStageGetterHashMap implements public Mono> subStage(LLDictionary dictionary, @Nullable CompositeSnapshot snapshot, Mono> prefixKeyMono) { - return Mono.usingWhen( - prefixKeyMono, - prefixKey -> Mono - .fromSupplier(() -> DatabaseMapDictionaryHashed - .tail(dictionary, - prefixKey, - keySerializer, - valueSerializer, - keyHashFunction, - keyHashSerializer - ) - ), - prefixKey -> Mono.fromRunnable(prefixKey::close) - ); + return LLUtils.usingSend(prefixKeyMono, prefixKey -> Mono.just(DatabaseMapDictionaryHashed + .tail(dictionary, prefixKey, keySerializer, valueSerializer, keyHashFunction, + keyHashSerializer, d -> {})), true); } public int getKeyHashBinaryLength() { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashSet.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashSet.java index 101b8a6..39c3279 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashSet.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterHashSet.java @@ -34,13 +34,8 @@ public class SubStageGetterHashSet implements Mono> prefixKeyMono) { return Mono.usingWhen(prefixKeyMono, prefixKey -> Mono - .fromSupplier(() -> DatabaseSetDictionaryHashed - .tail(dictionary, - prefixKey, - keySerializer, - keyHashFunction, - keyHashSerializer - ) + .fromSupplier(() -> DatabaseSetDictionaryHashed.tail(dictionary, prefixKey, keySerializer, + keyHashFunction, keyHashSerializer, d -> {}) ), prefixKey -> Mono.fromRunnable(prefixKey::close) ); diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMap.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMap.java index dee4cb6..772f417 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMap.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMap.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.serialization.Serializer; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.util.Map; @@ -25,11 +26,9 @@ public class SubStageGetterMap implements SubStageGetter, Databa public Mono> subStage(LLDictionary dictionary, @Nullable CompositeSnapshot snapshot, Mono> prefixKeyMono) { - return Mono.usingWhen(prefixKeyMono, + return LLUtils.usingSend(prefixKeyMono, prefixKey -> Mono.fromSupplier(() -> DatabaseMapDictionary - .tail(dictionary, prefixKey, keySerializer, valueSerializer)), - prefixKey -> Mono.fromRunnable(prefixKey::close) - ); + .tail(dictionary, prefixKey, keySerializer, valueSerializer, d -> {})), true); } public int getKeyBinaryLength() { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMapDeep.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMapDeep.java index 5794c85..34e25c5 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMapDeep.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterMapDeep.java @@ -4,6 +4,7 @@ import io.net5.buffer.api.Buffer; import io.net5.buffer.api.Send; import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.database.LLDictionary; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength; import java.util.Map; import org.jetbrains.annotations.Nullable; @@ -40,11 +41,9 @@ public class SubStageGetterMapDeep> implements public Mono> subStage(LLDictionary dictionary, @Nullable CompositeSnapshot snapshot, Mono> prefixKeyMono) { - return Mono.usingWhen(prefixKeyMono, - prefixKey -> Mono.fromSupplier(() -> DatabaseMapDictionaryDeep - .deepIntermediate(dictionary, prefixKey, keySerializer, subStageGetter, keyExtLength)), - prefixKey -> Mono.fromRunnable(prefixKey::close) - ); + return LLUtils.usingSend(prefixKeyMono, prefixKey -> Mono.just(DatabaseMapDictionaryDeep + .deepIntermediate(dictionary, prefixKey, keySerializer, subStageGetter, keyExtLength, + d -> {})), true); } public int getKeyBinaryLength() { diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSet.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSet.java index 0e26fea..6b6260c 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSet.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSet.java @@ -24,7 +24,7 @@ public class SubStageGetterSet implements SubStageGetter, Dat Mono> prefixKeyMono) { return Mono.usingWhen(prefixKeyMono, prefixKey -> Mono - .fromSupplier(() -> DatabaseSetDictionary.tail(dictionary, prefixKey, keySerializer)), + .fromSupplier(() -> DatabaseSetDictionary.tail(dictionary, prefixKey, keySerializer, d -> {})), prefixKey -> Mono.fromRunnable(prefixKey::close) ); } diff --git a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSingle.java b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSingle.java index eb71e26..533b4a6 100644 --- a/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSingle.java +++ b/src/main/java/it/cavallium/dbengine/database/collections/SubStageGetterSingle.java @@ -20,12 +20,7 @@ public class SubStageGetterSingle implements SubStageGetter> subStage(LLDictionary dictionary, @Nullable CompositeSnapshot snapshot, Mono> keyPrefixMono) { - return Mono.usingWhen( - keyPrefixMono, - keyPrefix -> Mono - .>fromSupplier(() -> new DatabaseSingle<>(dictionary, keyPrefix, serializer)), - keyPrefix -> Mono.fromRunnable(keyPrefix::close) - ); + return keyPrefixMono.map(keyPrefix -> new DatabaseSingle<>(dictionary, keyPrefix, serializer, d -> {})); } } diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java index f5b6e5e..19ee489 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearcher.java @@ -3,6 +3,7 @@ package it.cavallium.dbengine.database.disk; import io.net5.buffer.api.Drop; import io.net5.buffer.api.Owned; import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.database.LiveResourceSupport; import java.io.IOException; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.IndexSearcher; @@ -11,7 +12,7 @@ import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class LLIndexSearcher extends ResourceSupport { +public class LLIndexSearcher extends LiveResourceSupport { private IndexSearcher indexSearcher; diff --git a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java index 04203dd..a6a0411 100644 --- a/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java +++ b/src/main/java/it/cavallium/dbengine/database/disk/LLIndexSearchers.java @@ -5,6 +5,7 @@ import io.net5.buffer.api.Owned; import io.net5.buffer.api.Resource; import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import java.io.IOException; import java.io.UncheckedIOException; @@ -37,7 +38,7 @@ public interface LLIndexSearchers extends Resource { IndexReader allShards(); - class UnshardedIndexSearchers extends ResourceSupport + class UnshardedIndexSearchers extends LiveResourceSupport implements LLIndexSearchers { private LLIndexSearcher indexSearcher; @@ -103,7 +104,7 @@ public interface LLIndexSearchers extends Resource { } } - class ShardedIndexSearchers extends ResourceSupport + class ShardedIndexSearchers extends LiveResourceSupport implements LLIndexSearchers { private List indexSearchers; diff --git a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java index cff94bf..98079ff 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java +++ b/src/main/java/it/cavallium/dbengine/lucene/LuceneUtils.java @@ -9,6 +9,7 @@ import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.EnglishItalianStopFilter; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLSnapshot; +import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.collections.DatabaseMapDictionary; import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep; import it.cavallium.dbengine.database.collections.ValueGetter; @@ -231,9 +232,8 @@ public class LuceneUtils { public static ValueGetter, V> getAsyncDbValueGetterDeep( CompositeSnapshot snapshot, DatabaseMapDictionaryDeep, DatabaseMapDictionary> dictionaryDeep) { - return entry -> dictionaryDeep - .at(snapshot, entry.getKey()) - .flatMap(sub -> sub.getValue(snapshot, entry.getValue()).doAfterTerminate(sub::release)); + return entry -> LLUtils.usingResource(dictionaryDeep + .at(snapshot, entry.getKey()), sub -> sub.getValue(snapshot, entry.getValue()), true); } public static PerFieldAnalyzerWrapper toPerFieldAnalyzerWrapper(IndicizerAnalyzers indicizerAnalyzers) { diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java index 9a52926..f92d27f 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/LuceneSearchResult.java @@ -6,6 +6,7 @@ import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.query.current.data.TotalHitsCount; import it.cavallium.dbengine.database.LLKeyScore; import it.cavallium.dbengine.database.LLSearchResultShard; +import it.cavallium.dbengine.database.LiveResourceSupport; import it.cavallium.dbengine.database.disk.LLLocalKeyValueDatabase; import java.io.IOException; import java.util.Objects; @@ -14,7 +15,7 @@ import org.warp.commonutils.log.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class LuceneSearchResult extends ResourceSupport { +public final class LuceneSearchResult extends LiveResourceSupport { private static final Logger logger = LoggerFactory.getLogger(LuceneSearchResult.class); diff --git a/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java index 43f4a25..038f65e 100644 --- a/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java +++ b/src/main/java/it/cavallium/dbengine/netty/NullableBuffer.java @@ -6,9 +6,10 @@ import io.net5.buffer.api.Owned; import io.net5.buffer.api.Send; import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.client.SearchResult; +import it.cavallium.dbengine.database.LiveResourceSupport; import org.jetbrains.annotations.Nullable; -public class NullableBuffer extends ResourceSupport { +public class NullableBuffer extends LiveResourceSupport { @Nullable private Buffer buffer; diff --git a/src/test/java/it/cavallium/dbengine/DbTestUtils.java b/src/test/java/it/cavallium/dbengine/DbTestUtils.java index c16b139..3a06cfb 100644 --- a/src/test/java/it/cavallium/dbengine/DbTestUtils.java +++ b/src/test/java/it/cavallium/dbengine/DbTestUtils.java @@ -179,7 +179,8 @@ public class DbTestUtils { if (mapType == MapType.MAP) { return DatabaseMapDictionary.simple(dictionary, SerializerFixedBinaryLength.utf8(dictionary.getAllocator(), keyBytes), - Serializer.utf8(dictionary.getAllocator()) + Serializer.utf8(dictionary.getAllocator()), + d -> {} ); } else { return DatabaseMapDictionaryHashed.simple(dictionary, @@ -209,7 +210,8 @@ public class DbTestUtils { return out.send(); } } - } + }, + d -> {} ); } } @@ -224,7 +226,8 @@ public class DbTestUtils { key2Bytes, new SubStageGetterMap<>(SerializerFixedBinaryLength.utf8(dictionary.getAllocator(), key2Bytes), Serializer.utf8(dictionary.getAllocator()) - ) + ), + d -> {} ); } @@ -239,7 +242,8 @@ public class DbTestUtils { Serializer.utf8(dictionary.getAllocator()), String::hashCode, SerializerFixedBinaryLength.intSerializer(dictionary.getAllocator()) - ) + ), + d -> {} ); } @@ -249,7 +253,8 @@ public class DbTestUtils { Serializer.utf8(dictionary.getAllocator()), Serializer.utf8(dictionary.getAllocator()), String::hashCode, - SerializerFixedBinaryLength.intSerializer(dictionary.getAllocator()) + SerializerFixedBinaryLength.intSerializer(dictionary.getAllocator()), + d -> {} ); } } diff --git a/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java b/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java index f56ce1d..cc1e3f6 100644 --- a/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java +++ b/src/test/java/it/cavallium/dbengine/TestDictionaryMap.java @@ -115,8 +115,7 @@ public abstract class TestDictionaryMap { var resultingMap = run(map.get(null)); Assertions.assertEquals(shouldFail ? null : Map.of(key, value), resultingMap); - runVoid(map.close()); - map.release(); + map.close(); //if (shouldFail) this.checkLeaks = false; @@ -129,10 +128,10 @@ public abstract class TestDictionaryMap { var stpVer = StepVerifier .create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode) .map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5)) - .flatMap(map -> map - .at(null, key).flatMap(v -> v.set(value).doAfterTerminate(v::release)) - .then(map.at(null, key).flatMap(v -> v.get(null).doAfterTerminate(v::release))) - .doAfterTerminate(map::release) + .flatMap(map -> LLUtils + .usingResource(map.at(null, key), v -> v.set(value), true) + .then(LLUtils.usingResource(map.at(null, key), v -> v.get(null), true)) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -155,7 +154,7 @@ public abstract class TestDictionaryMap { map.putValueAndGetPrevious(key, value), map.putValueAndGetPrevious(key, value) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -178,7 +177,7 @@ public abstract class TestDictionaryMap { map.putValue(key, value).then(map.removeAndGetPrevious(key)), map.removeAndGetPrevious(key) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -201,7 +200,7 @@ public abstract class TestDictionaryMap { map.putValue(key, value).then(map.removeAndGetStatus(key)), map.removeAndGetStatus(key) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -244,7 +243,7 @@ public abstract class TestDictionaryMap { return value; }) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -311,7 +310,7 @@ public abstract class TestDictionaryMap { .doOnSuccess(s -> log.debug("5. Getting value: {}", key)) .then(map.getValue(null, key)) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -336,7 +335,7 @@ public abstract class TestDictionaryMap { map.remove(key), map.putValueAndGetChanged(key, "error?").single() ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -395,7 +394,7 @@ public abstract class TestDictionaryMap { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.getMulti(null, Flux.fromIterable(entries.keySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -423,7 +422,7 @@ public abstract class TestDictionaryMap { .flatMapMany(map -> map .setAllValues(Flux.fromIterable(entries.entrySet())) .thenMany(map.getMulti(null, Flux.fromIterable(entries.keySet()))) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -453,7 +452,7 @@ public abstract class TestDictionaryMap { map.setAllValuesAndGetPrevious(Flux.fromIterable(entries.entrySet())), map.setAllValuesAndGetPrevious(Flux.fromIterable(entries.entrySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -481,7 +480,7 @@ public abstract class TestDictionaryMap { map.set(entries).then(Mono.empty()), map.getMulti(null, Flux.fromIterable(entries.keySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -522,7 +521,7 @@ public abstract class TestDictionaryMap { removalMono.then(Mono.empty()), map.setAndGetChanged(entries).single() ) - .doAfterTerminate(map::release); + .doFinally(s -> map.close()); }) )); if (shouldFail) { @@ -544,7 +543,7 @@ public abstract class TestDictionaryMap { .concat(map.setAndGetPrevious(entries), map.setAndGetPrevious(entries)) .map(Map::entrySet) .concatMapIterable(list -> list) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -570,7 +569,7 @@ public abstract class TestDictionaryMap { .concat(map.set(entries).then(Mono.empty()), map.clearAndGetPrevious(), map.get(null)) .map(Map::entrySet) .concatMapIterable(list -> list) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -597,7 +596,7 @@ public abstract class TestDictionaryMap { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.getAllValues(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -627,7 +626,7 @@ public abstract class TestDictionaryMap { .map(Map::entrySet) .flatMapIterable(list -> list) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -659,10 +658,10 @@ public abstract class TestDictionaryMap { .getValue() .get(null) .map(val -> Map.entry(stage.getKey(), val)) - .doAfterTerminate(() -> stage.getValue().release()) + .doFinally(s -> stage.getValue().close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -691,7 +690,7 @@ public abstract class TestDictionaryMap { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.isEmpty(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .flatMap(val -> shouldFail ? Mono.empty() : Mono.just(val)) .transform(LLUtils::handleDiscard) @@ -718,7 +717,7 @@ public abstract class TestDictionaryMap { map.clear().then(Mono.empty()), map.isEmpty(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .flatMap(val -> shouldFail ? Mono.empty() : Mono.just(val)) .transform(LLUtils::handleDiscard) diff --git a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java index d7073c7..dd79298 100644 --- a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java +++ b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeep.java @@ -12,6 +12,7 @@ import static it.cavallium.dbengine.DbTestUtils.tempDatabaseMapDictionaryMap; import static it.cavallium.dbengine.DbTestUtils.tempDb; import static it.cavallium.dbengine.DbTestUtils.tempDictionary; +import io.net5.buffer.api.internal.ResourceSupport; import it.cavallium.dbengine.DbTestUtils.TestAllocator; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.UpdateMode; @@ -196,8 +197,7 @@ public abstract class TestDictionaryMapDeep { var resultingMap = run(map.get(null)); Assertions.assertEquals(shouldFail ? null : Map.of(key, value), resultingMap); - runVoid(map.close()); - map.release(); + map.close(); //if (shouldFail) this.checkLeaks = false; @@ -220,8 +220,7 @@ public abstract class TestDictionaryMapDeep { Assertions.assertEquals(shouldFail ? null : value, returnedValue); - runVoid(map.close()); - map.release(); + map.close(); //if (shouldFail) this.checkLeaks = false; @@ -240,7 +239,7 @@ public abstract class TestDictionaryMapDeep { .flatMapMany(map -> map .putValue(key, value) .thenMany(map.getAllValues(null)) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -265,14 +264,14 @@ public abstract class TestDictionaryMapDeep { .flatMap(v_ -> Mono.using( () -> v_, v -> v.set(value), - DatabaseMapDictionaryDeep::release + ResourceSupport::close )) .then(map .at(null, "capra") .flatMap(v_ -> Mono.using( () -> v_, v -> v.set(Map.of("normal", "123", "ormaln", "456")), - DatabaseMapDictionaryDeep::release + ResourceSupport::close )) ) .thenMany(map @@ -280,10 +279,10 @@ public abstract class TestDictionaryMapDeep { .flatMap(v -> v.getValue() .getAllValues(null) .map(result -> Tuples.of(v.getKey(), result.getKey(), result.getValue())) - .doAfterTerminate(() -> v.getValue().release()) + .doFinally(s -> v.getValue().close()) ) ), - DatabaseMapDictionaryDeep::release + ResourceSupport::close )) )); if (shouldFail) { @@ -308,9 +307,9 @@ public abstract class TestDictionaryMapDeep { .create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode) .map(dict -> tempDatabaseMapDictionaryDeepMap(dict, 5, 6)) .flatMap(map -> map - .at(null, key1).flatMap(v -> v.putValue(key2, value).doAfterTerminate(v::release)) - .then(map.at(null, key1).flatMap(v -> v.getValue(null, key2).doAfterTerminate(v::release))) - .doAfterTerminate(map::release) + .at(null, key1).flatMap(v -> v.putValue(key2, value).doFinally(s -> v.close())) + .then(map.at(null, key1).flatMap(v -> v.getValue(null, key2).doFinally(s -> v.close()))) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -335,7 +334,7 @@ public abstract class TestDictionaryMapDeep { map.putValueAndGetPrevious(key, value), map.putValueAndGetPrevious(key, value) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -358,22 +357,22 @@ public abstract class TestDictionaryMapDeep { .at(null, key1) .flatMap(v -> v .putValueAndGetPrevious(key2, "error?") - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .putValueAndGetPrevious(key2, value) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .putValueAndGetPrevious(key2, value) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -396,7 +395,7 @@ public abstract class TestDictionaryMapDeep { map.putValue(key, value).then(map.removeAndGetPrevious(key)), map.removeAndGetPrevious(key) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -420,22 +419,22 @@ public abstract class TestDictionaryMapDeep { .flatMap(v -> v .putValue(key2, "error?") .then(v.removeAndGetPrevious(key2)) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .putValue(key2, value) .then(v.removeAndGetPrevious(key2)) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v.removeAndGetPrevious(key2) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -458,7 +457,7 @@ public abstract class TestDictionaryMapDeep { map.putValue(key, value).then(map.removeAndGetStatus(key)), map.removeAndGetStatus(key) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -482,22 +481,22 @@ public abstract class TestDictionaryMapDeep { .flatMap(v -> v .putValue(key2, "error?") .then(v.removeAndGetStatus(key2)) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .putValue(key2, value) .then(v.removeAndGetStatus(key2)) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v.removeAndGetStatus(key2) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -540,7 +539,7 @@ public abstract class TestDictionaryMapDeep { return value; }) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (updateMode != UpdateMode.ALLOW_UNSAFE || shouldFail) { @@ -565,28 +564,28 @@ public abstract class TestDictionaryMapDeep { .at(null, key1) .flatMap(v -> v .updateValue(key2, prev -> prev) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .updateValue(key2, prev -> value) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .updateValue(key2, prev -> value) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) .flatMap(v -> v .updateValue(key2, prev -> null) - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) .transform(LLUtils::handleDiscard) ) )); @@ -626,7 +625,7 @@ public abstract class TestDictionaryMapDeep { assert Objects.equals(old, value); return value; }).then(map.getValue(null, key)) - ).doAfterTerminate(map::release)) + ).doFinally(s -> map.close())) )); if (updateMode != UpdateMode.ALLOW_UNSAFE || shouldFail) { stpVer.verifyError(); @@ -652,7 +651,7 @@ public abstract class TestDictionaryMapDeep { .updateValue(key2, prev -> prev) .then(v.getValue(null, key2)) .defaultIfEmpty("empty") - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) @@ -660,7 +659,7 @@ public abstract class TestDictionaryMapDeep { .updateValue(key2, prev -> value) .then(v.getValue(null, key2)) .defaultIfEmpty("empty") - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) @@ -668,7 +667,7 @@ public abstract class TestDictionaryMapDeep { .updateValue(key2, prev -> value) .then(v.getValue(null, key2)) .defaultIfEmpty("empty") - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ), map .at(null, key1) @@ -676,10 +675,10 @@ public abstract class TestDictionaryMapDeep { .updateValue(key2, prev -> null) .then(v.getValue(null, key2)) .defaultIfEmpty("empty") - .doAfterTerminate(v::release) + .doFinally(s -> v.close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) .transform(LLUtils::handleDiscard) ) )); @@ -704,7 +703,7 @@ public abstract class TestDictionaryMapDeep { map.remove(key), map.putValueAndGetChanged(key, Map.of("error?", "error.")).single() ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -758,7 +757,7 @@ public abstract class TestDictionaryMapDeep { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.getMulti(null, Flux.fromIterable(entries.keySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -786,7 +785,7 @@ public abstract class TestDictionaryMapDeep { .flatMapMany(map -> map .setAllValues(Flux.fromIterable(entries.entrySet())) .thenMany(map.getMulti(null, Flux.fromIterable(entries.keySet()))) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -815,7 +814,7 @@ public abstract class TestDictionaryMapDeep { map.setAllValuesAndGetPrevious(Flux.fromIterable(entries.entrySet())), map.setAllValuesAndGetPrevious(Flux.fromIterable(entries.entrySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) .transform(LLUtils::handleDiscard) ) )); @@ -843,7 +842,7 @@ public abstract class TestDictionaryMapDeep { map.set(entries).then(Mono.empty()), map.getMulti(null, Flux.fromIterable(entries.keySet())) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .filter(k -> k.getValue().isPresent()) .map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow())) @@ -881,7 +880,7 @@ public abstract class TestDictionaryMapDeep { removalMono.then(Mono.empty()), map.setAndGetChanged(entries).single() ) - .doAfterTerminate(map::release); + .doFinally(s -> map.close()); }) .transform(LLUtils::handleDiscard) )); @@ -907,7 +906,7 @@ public abstract class TestDictionaryMapDeep { ) .map(Map::entrySet) .concatMapIterable(list -> list) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -933,7 +932,7 @@ public abstract class TestDictionaryMapDeep { .concat(map.set(entries).then(Mono.empty()), map.clearAndGetPrevious(), map.get(null)) .map(Map::entrySet) .concatMapIterable(list -> list) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -961,7 +960,7 @@ public abstract class TestDictionaryMapDeep { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.getAllValues(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -990,7 +989,7 @@ public abstract class TestDictionaryMapDeep { .map(Map::entrySet) .flatMapIterable(list -> list) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -1021,10 +1020,10 @@ public abstract class TestDictionaryMapDeep { .getValue() .get(null) .map(val -> Map.entry(stage.getKey(), val)) - .doAfterTerminate(() -> stage.getValue().release()) + .doFinally(s -> stage.getValue().close()) ) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { @@ -1051,7 +1050,7 @@ public abstract class TestDictionaryMapDeep { map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()), map.isEmpty(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) .transform(LLUtils::handleDiscard) )); @@ -1077,7 +1076,7 @@ public abstract class TestDictionaryMapDeep { map.clear().then(Mono.empty()), map.isEmpty(null) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { diff --git a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeepHashMap.java b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeepHashMap.java index 8486a82..9a81610 100644 --- a/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeepHashMap.java +++ b/src/test/java/it/cavallium/dbengine/TestDictionaryMapDeepHashMap.java @@ -121,14 +121,14 @@ public abstract class TestDictionaryMapDeepHashMap { .create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode) .map(dict -> tempDatabaseMapDictionaryDeepMapHashMap(dict, 5)) .flatMapMany(map -> map - .at(null, key1).flatMap(v -> v.putValue(key2, value).doAfterTerminate(v::release)) + .at(null, key1).flatMap(v -> v.putValue(key2, value).doFinally(s -> v.close())) .thenMany(map .getAllValues(null) .map(Entry::getValue) .flatMap(maps -> Flux.fromIterable(maps.entrySet())) .map(Entry::getValue) ) - .doAfterTerminate(map::release) + .doFinally(s -> map.close()) ) )); if (shouldFail) { From 1a73a5a33f37667d7cbfef5b542f6f3ba0288280 Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 22:04:01 +0200 Subject: [PATCH 22/23] Bugfixes --- .../dbengine/database/LiveResourceSupport.java | 7 +++---- .../SimpleUnsortedUnscoredLuceneMultiSearcher.java | 10 ++++++---- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java b/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java index 88b1c36..cb61d33 100644 --- a/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java +++ b/src/main/java/it/cavallium/dbengine/database/LiveResourceSupport.java @@ -19,13 +19,12 @@ public abstract class LiveResourceSupport, T extends LiveR @Override protected void finalize() throws Throwable { if (this.isAccessible()) { + var ise = new IllegalStateException("Resource not released"); + ise.setStackTrace(new StackTraceElement[0]); + logger.error("Resource not released: {}", this, attachTrace(ise)); try { this.close(); } catch (Throwable ignored) { - } finally { - var ise = new IllegalStateException("Resource not released"); - ise.setStackTrace(new StackTraceElement[0]); - logger.error("Resource not released: {}", this, attachTrace(ise)); } } super.finalize(); diff --git a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java index fd88002..ad2b99c 100644 --- a/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java +++ b/src/main/java/it/cavallium/dbengine/lucene/searcher/SimpleUnsortedUnscoredLuceneMultiSearcher.java @@ -24,7 +24,7 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea LocalQueryParams queryParams, String keyFieldName, LLSearchTransformer transformer) { - var indexSearchersResource = Mono + var indexSearchersSendResource = Mono .fromRunnable(() -> { LLUtils.ensureBlocking(); if (queryParams.isSorted() && queryParams.limit() > 0) { @@ -36,11 +36,12 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea + " by SimpleUnsortedUnscoredLuceneMultiSearcher"); } }) - .then(indexSearchersMono.map(Send::receive)); + .then(indexSearchersMono); var localQueryParams = getLocalQueryParams(queryParams); - return LLUtils.usingResource(indexSearchersResource, - indexSearchers -> Flux.fromIterable(indexSearchers.shards()) + return LLUtils.usingSendResource(indexSearchersSendResource, + indexSearchers -> Flux + .fromIterable(indexSearchers.shards()) .flatMap(searcher -> { var llSearcher = Mono.fromCallable(() -> new LLIndexSearcher(searcher, d -> {}).send()); return localSearcher.collect(llSearcher, localQueryParams, keyFieldName, transformer); @@ -69,6 +70,7 @@ public class SimpleUnsortedUnscoredLuceneMultiSearcher implements LuceneMultiSea for (LuceneSearchResult luceneSearchResult : resultsToDrop) { luceneSearchResult.close(); } + indexSearchers.close(); }).send(); }), false From be1ca997a75b14ab5893ec5d71a6fddc0b49e8bf Mon Sep 17 00:00:00 2001 From: Andrea Cavalli Date: Thu, 23 Sep 2021 23:45:41 +0200 Subject: [PATCH 23/23] Bugfix --- .../cavallium/dbengine/database/LLUtils.java | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/src/main/java/it/cavallium/dbengine/database/LLUtils.java b/src/main/java/it/cavallium/dbengine/database/LLUtils.java index 16f38cb..8140892 100644 --- a/src/main/java/it/cavallium/dbengine/database/LLUtils.java +++ b/src/main/java/it/cavallium/dbengine/database/LLUtils.java @@ -619,7 +619,17 @@ public class LLUtils { public static Buffer compositeBuffer(BufferAllocator alloc, @NotNull Send buffer1, @NotNull Send buffer2) { - return CompositeBuffer.compose(alloc, buffer1, buffer2); + var b1 = buffer1.receive(); + try (var b2 = buffer2.receive()) { + if (b1.writerOffset() < b1.capacity() || b2.writerOffset() < b2.capacity()) { + b1.ensureWritable(b2.readableBytes(), b2.readableBytes(), true); + b2.copyInto(b2.readerOffset(), b1, b1.writerOffset(), b2.readableBytes()); + b1.writerOffset(b1.writerOffset() + b2.readableBytes()); + return b1; + } else { + return CompositeBuffer.compose(alloc, b1.send(), b2.send()); + } + } } @NotNull @@ -627,7 +637,25 @@ public class LLUtils { @NotNull Send buffer1, @NotNull Send buffer2, @NotNull Send buffer3) { - return CompositeBuffer.compose(alloc, buffer1, buffer2, buffer3); + var b1 = buffer1.receive(); + try (var b2 = buffer2.receive()) { + try (var b3 = buffer3.receive()) { + if (b1.writerOffset() < b1.capacity() + || b2.writerOffset() < b2.capacity() + || b3.writerOffset() < b3.capacity()) { + b1.ensureWritable(b2.readableBytes(), b2.readableBytes(), true); + b2.copyInto(b2.readerOffset(), b1, b1.writerOffset(), b2.readableBytes()); + b1.writerOffset(b1.writerOffset() + b2.readableBytes()); + + b1.ensureWritable(b3.readableBytes(), b3.readableBytes(), true); + b3.copyInto(b3.readerOffset(), b1, b1.writerOffset(), b3.readableBytes()); + b1.writerOffset(b1.writerOffset() + b3.readableBytes()); + return b1; + } else { + return CompositeBuffer.compose(alloc, b1.send(), b2.send(), b3.send()); + } + } + } } public static Mono resolveDelta(Mono> prev, UpdateReturnMode updateReturnMode) {