Use global hooks

This commit is contained in:
Andrea Cavalli 2022-01-26 14:22:54 +01:00
parent f837a1d1b2
commit cf53eb4f5a
25 changed files with 178 additions and 474 deletions

View File

@ -77,7 +77,7 @@ public interface LLLuceneIndex extends LLSnapshottable {
}
})
.defaultIfEmpty(TotalHitsCount.of(0, true))
).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
);
}
boolean isLowMemoryMode();

View File

@ -20,10 +20,15 @@ import io.net5.buffer.api.unsafe.UnsafeMemoryManager;
import io.net5.util.IllegalReferenceCountException;
import io.net5.util.internal.PlatformDependent;
import it.cavallium.dbengine.database.collections.DatabaseStage;
import it.cavallium.dbengine.database.disk.LLLocalKeyValueDatabase;
import it.cavallium.dbengine.database.disk.MemorySegmentUtils;
import it.cavallium.dbengine.database.disk.UpdateAtomicResultCurrent;
import it.cavallium.dbengine.database.disk.UpdateAtomicResultDelta;
import it.cavallium.dbengine.database.disk.UpdateAtomicResultPrevious;
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.collector.LMDBFullFieldDocCollector;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.ArrayList;
@ -34,6 +39,7 @@ import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
@ -62,6 +68,7 @@ import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.RocksDB;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.util.function.Tuple2;
@ -84,6 +91,7 @@ public class LLUtils {
private static final byte[] RESPONSE_TRUE_BUF = new byte[]{1};
private static final byte[] RESPONSE_FALSE_BUF = new byte[]{0};
public static final byte[][] LEXICONOGRAPHIC_ITERATION_SEEKS = new byte[256][1];
public static final AtomicBoolean hookRegistered = new AtomicBoolean();
static {
MemoryManager unsafeMemoryManager;
@ -97,6 +105,13 @@ public class LLUtils {
var b = LEXICONOGRAPHIC_ITERATION_SEEKS[i1];
b[0] = (byte) i1;
}
initHooks();
}
public static void initHooks() {
if (hookRegistered.compareAndSet(false, true)) {
Hooks.onNextDropped(LLUtils::onNextDropped);
}
}
public static boolean responseToBoolean(byte[] response) {
@ -492,9 +507,7 @@ public class LLUtils {
} else {
return Mono.empty();
}
}, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close()))
.doOnDiscard(Send.class, send -> send.close())
.doOnDiscard(Resource.class, Resource::close);
}, (r, ex) -> Mono.fromRunnable(() -> r.close()), r -> Mono.fromRunnable(() -> r.close()));
}
// todo: remove this ugly method
@ -523,13 +536,7 @@ public class LLUtils {
if (r.isAccessible()) {
r.close();
}
}))
.doOnDiscard(Resource.class, resource -> {
if (resource.isAccessible()) {
resource.close();
}
})
.doOnDiscard(Send.class, send -> send.close());
}));
}
// todo: remove this ugly method
@ -558,13 +565,7 @@ public class LLUtils {
if (r.isAccessible()) {
r.close();
}
}))
.doOnDiscard(Resource.class, resource -> {
if (resource.isAccessible()) {
resource.close();
}
})
.doOnDiscard(Send.class, send -> send.close());
}));
}
// todo: remove this ugly method
@ -594,13 +595,7 @@ public class LLUtils {
if (r.isAccessible()) {
r.close();
}
})))
.doOnDiscard(Resource.class, resource -> {
if (resource.isAccessible()) {
resource.close();
}
})
.doOnDiscard(Send.class, send -> send.close());
})));
}
// todo: remove this ugly method
@ -625,13 +620,7 @@ public class LLUtils {
if (r.isAccessible()) {
r.close();
}
}))
.doOnDiscard(Resource.class, resource -> {
if (resource.isAccessible()) {
resource.close();
}
})
.doOnDiscard(Send.class, send -> send.close());
}));
}
// todo: remove this ugly method
@ -660,13 +649,7 @@ public class LLUtils {
if (r.isAccessible()) {
r.close();
}
}))
.doOnDiscard(Resource.class, resource -> {
if (resource.isAccessible()) {
resource.close();
}
})
.doOnDiscard(Send.class, send -> send.close());
}));
}
public static boolean isSet(ScoreDoc[] scoreDocs) {
@ -958,184 +941,6 @@ public class LLUtils {
return Mono.fromCallable(rangeCallable);
}
public static <T> Mono<T> handleDiscard(Mono<T> mono) {
return mono.doOnDiscard(Object.class, obj -> {
if (obj instanceof SafeCloseable o) {
discardRefCounted(o);
} else if (obj instanceof Entry o) {
discardEntry(o);
} else if (obj instanceof Collection o) {
discardCollection(o);
} else if (obj instanceof Tuple3 o) {
discardTuple3(o);
} else if (obj instanceof Tuple2 o) {
discardTuple2(o);
} else if (obj instanceof LLEntry o) {
discardLLEntry(o);
} else if (obj instanceof LLRange o) {
discardLLRange(o);
} else if (obj instanceof LLDelta o) {
discardLLDelta(o);
} else if (obj instanceof Delta o) {
discardDelta(o);
} else if (obj instanceof Send o) {
discardSend(o);
} else if (obj instanceof Map o) {
discardMap(o);
} else if (obj instanceof DatabaseStage o) {
discardStage(o);
}
});
}
public static <T> Flux<T> handleDiscard(Flux<T> mono) {
return mono.doOnDiscard(Object.class, obj -> {
if (obj instanceof SafeCloseable o) {
discardRefCounted(o);
} else if (obj instanceof Entry o) {
discardEntry(o);
} else if (obj instanceof Collection o) {
discardCollection(o);
} else if (obj instanceof Tuple3 o) {
discardTuple3(o);
} else if (obj instanceof Tuple2 o) {
discardTuple2(o);
} else if (obj instanceof LLEntry o) {
discardLLEntry(o);
} else if (obj instanceof LLRange o) {
discardLLRange(o);
} else if (obj instanceof Delta o) {
discardDelta(o);
} else if (obj instanceof LLDelta o) {
discardLLDelta(o);
} else if (obj instanceof Send o) {
discardSend(o);
} else if (obj instanceof Resource o) {
discardResource(o);
} else if (obj instanceof Map o) {
discardMap(o);
} else if (obj instanceof DatabaseStage o) {
discardStage(o);
}
});
}
private static void discardLLEntry(LLEntry entry) {
logger.trace(MARKER_ROCKSDB, "Releasing discarded Buffer");
entry.close();
}
private static void discardLLRange(LLRange range) {
logger.trace(MARKER_ROCKSDB, "Releasing discarded LLRange");
range.close();
}
private static void discardLLDelta(LLDelta delta) {
logger.trace(MARKER_ROCKSDB, "Releasing discarded LLDelta");
delta.close();
}
private static void discardEntry(Map.Entry<?, ?> e) {
if (e.getKey() instanceof Buffer bb) {
bb.close();
}
if (e.getValue() instanceof Buffer bb) {
bb.close();
}
}
private static void discardTuple2(Tuple2<?, ?> e) {
if (e.getT1() instanceof Buffer bb) {
bb.close();
}
if (e.getT2() instanceof Buffer bb) {
bb.close();
}
}
private static void discardTuple3(Tuple3<?, ?, ?> e) {
if (e.getT1() instanceof Buffer bb) {
bb.close();
} else if (e.getT1() instanceof Optional opt) {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
}
if (e.getT2() instanceof Buffer bb) {
bb.close();
} else if (e.getT1() instanceof Optional opt) {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
}
if (e.getT3() instanceof Buffer bb) {
bb.close();
} else if (e.getT1() instanceof Optional opt) {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
}
}
private static void discardRefCounted(SafeCloseable safeCloseable) {
safeCloseable.close();
}
private static void discardCollection(Collection<?> collection) {
for (Object o : collection) {
if (o instanceof SafeCloseable safeCloseable) {
safeCloseable.close();
} else if (o instanceof Map.Entry entry) {
if (entry.getKey() instanceof SafeCloseable bb) {
bb.close();
}
if (entry.getValue() instanceof SafeCloseable bb) {
bb.close();
}
} else {
break;
}
}
}
private static void discardDelta(Delta<?> delta) {
if (delta.previous() instanceof Buffer bb) {
bb.close();
}
if (delta.current() instanceof Buffer bb) {
bb.close();
}
}
private static void discardSend(Send<?> send) {
send.close();
}
private static void discardResource(Resource<?> res) {
res.close();
}
private static void discardMap(Map<?, ?> map) {
for (Entry<?, ?> entry : map.entrySet()) {
boolean hasByteBuf = false;
if (entry.getKey() instanceof Buffer bb) {
bb.close();
hasByteBuf = true;
}
if (entry.getValue() instanceof Buffer bb) {
bb.close();
hasByteBuf = true;
}
if (!hasByteBuf) {
break;
}
}
}
private static void discardStage(DatabaseStage<?> stage) {
// do nothing for now, to avoid double-free problems
}
public static boolean isDirect(Buffer key) {
var readableComponents = key.countReadableComponents();
if (readableComponents == 0) {
@ -1164,4 +969,51 @@ public class LLUtils {
public static int utf8MaxBytes(String deserialized) {
return deserialized.length() * 3;
}
private static void onNextDropped(Object next) {
if (next instanceof Send<?> send) {
send.close();
} else if (next instanceof Resource<?> resource) {
resource.close();
} else if (next instanceof Iterable<?> iterable) {
iterable.forEach(LLUtils::onNextDropped);
} else if (next instanceof SafeCloseable safeCloseable) {
safeCloseable.close();
} else if (next instanceof UpdateAtomicResultDelta delta) {
delta.delta().close();
} else if (next instanceof UpdateAtomicResultCurrent cur) {
cur.current().close();
} else if (next instanceof UpdateAtomicResultPrevious cur) {
cur.previous().close();
} else if (next instanceof Optional<?> optional) {
optional.ifPresent(LLUtils::onNextDropped);
} else if (next instanceof Map.Entry<?, ?> entry) {
var key = entry.getKey();
if (key != null) {
onNextDropped(key);
}
var value = entry.getValue();
if (value != null) {
onNextDropped(value);
}
} else if (next instanceof Delta<?> delta) {
var previous = delta.previous();
if (previous != null) {
onNextDropped(previous);
}
var current = delta.current();
if (current != null) {
onNextDropped(current);
}
} else if (next instanceof Map<?, ?> map) {
map.forEach((key, value) -> {
if (key != null) {
onNextDropped(key);
}
if (value != null) {
onNextDropped(value);
}
});
}
}
}

View File

@ -165,8 +165,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
.fromIterable(Collections.unmodifiableMap(value).entrySet())
.handle(this::serializeEntrySink)
).then(Mono.empty()))
.singleOrEmpty()
.transform(LLUtils::handleDiscard);
.singleOrEmpty();
}
@Override
@ -359,8 +358,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
} finally {
valueBufOpt.ifPresent(Resource::close);
}
})
.transform(LLUtils::handleDiscard);
});
}
private LLEntry serializeEntry(T keySuffix, U value) throws SerializationException {
@ -391,47 +389,22 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
} catch (Throwable e) {
sink.error(e);
}
})
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
return dictionary
.putMulti(serializedEntries, false)
.then()
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close)
.doOnDiscard(LLEntry.class, ResourceSupport::close)
.doOnDiscard(List.class, list -> {
for (Object o : list) {
if (o instanceof Send send) {
send.close();
} else if (o instanceof Buffer buf) {
buf.close();
}
}
});
return dictionary.putMulti(serializedEntries, false).then();
}
@Override
public Flux<Boolean> updateMulti(Flux<T> keys,
KVSerializationFunction<T, @Nullable U, @Nullable U> updater) {
var sharedKeys = keys.publish().refCount(2);
var serializedKeys = sharedKeys
.<Send<Buffer>>handle((key, sink) -> {
try {
Send<Buffer> serializedKey = serializeKeySuffixToKey(key).send();
sink.next(serializedKey);
} catch (Throwable ex) {
sink.error(ex);
}
})
.doOnDiscard(Tuple2.class, uncastedEntry -> {
if (uncastedEntry.getT1() instanceof Buffer byteBuf) {
byteBuf.close();
}
if (uncastedEntry.getT2() instanceof Buffer byteBuf) {
byteBuf.close();
}
});
var serializedKeys = sharedKeys.<Send<Buffer>>handle((key, sink) -> {
try {
Send<Buffer> serializedKey = serializeKeySuffixToKey(key).send();
sink.next(serializedKey);
} catch (Throwable ex) {
sink.error(ex);
}
});
var serializedUpdater = getSerializedUpdater(updater);
return dictionary.updateMulti(sharedKeys, serializedKeys, serializedUpdater);
}
@ -484,14 +457,6 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
} catch (Throwable e) {
sink.error(e);
}
})
.doOnDiscard(Entry.class, uncastedEntry -> {
if (uncastedEntry.getKey() instanceof Buffer byteBuf) {
byteBuf.close();
}
if (uncastedEntry.getValue() instanceof Buffer byteBuf) {
byteBuf.close();
}
});
}

View File

@ -301,9 +301,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
}
});
return this.subStageGetter
.subStage(dictionary, snapshot, suffixKeyWithoutExt)
.transform(LLUtils::handleDiscard)
.doOnDiscard(DatabaseStage.class, DatabaseStage::close);
.subStage(dictionary, snapshot, suffixKeyWithoutExt);
}
@Override
@ -334,8 +332,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
}
}),
Resource::close
))
.transform(LLUtils::handleDiscard);
));
}
/**

View File

@ -197,15 +197,13 @@ public class DatabaseMapDictionaryHashed<T, U, TH> extends
public Mono<DatabaseStageEntry<U>> at(@Nullable CompositeSnapshot snapshot, T key) {
return this
.atPrivate(snapshot, key, keySuffixHashFunction.apply(key))
.map(cast -> (DatabaseStageEntry<U>) cast)
.doOnDiscard(Resource.class, Resource::close);
.map(cast -> (DatabaseStageEntry<U>) cast);
}
private Mono<DatabaseSingleBucket<T, U, TH>> atPrivate(@Nullable CompositeSnapshot snapshot, T key, TH hash) {
return subDictionary
.at(snapshot, hash)
.map(entry -> new DatabaseSingleBucket<T, U, TH>(entry, key, null))
.doOnDiscard(Resource.class, Resource::close);
.map(entry -> new DatabaseSingleBucket<T, U, TH>(entry, key, null));
}
@Override

View File

@ -119,14 +119,7 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends
return keys
.flatMapSequential(key -> this.getValue(snapshot, key, existsAlmostCertainly))
.map(Optional::of)
.defaultIfEmpty(Optional.empty())
.doOnDiscard(Entry.class, unknownEntry -> {
if (unknownEntry.getValue() instanceof Optional optionalBuffer
&& optionalBuffer.isPresent()
&& optionalBuffer.get() instanceof Buffer buffer) {
buffer.close();
}
});
.defaultIfEmpty(Optional.empty());
}
/**

View File

@ -47,7 +47,7 @@ public class SubStageGetterHashMap<T, U, TH> implements
keyHashSerializer,
null
);
}).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
});
}
public int getKeyHashBinaryLength() {

View File

@ -44,7 +44,7 @@ public class SubStageGetterHashSet<T, TH> implements
keyHashSerializer,
null
);
}).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
});
}
public int getKeyHashBinaryLength() {

View File

@ -32,7 +32,7 @@ public class SubStageGetterMap<T, U> implements
return prefixKeyMono.map(prefixKeyToReceive -> {
var prefixKey = prefixKeyToReceive.receive();
return DatabaseMapDictionary.tail(dictionary, prefixKey, keySerializer, valueSerializer, null);
}).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
});
}
public int getKeyBinaryLength() {

View File

@ -52,7 +52,7 @@ public class SubStageGetterMapDeep<T, U, US extends DatabaseStage<U>> implements
keyExtLength,
null
);
}).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
});
}
public int getKeyBinaryLength() {

View File

@ -28,7 +28,7 @@ public class SubStageGetterSet<T> implements
return prefixKeyMono.map(prefixKeyToReceive -> {
var prefixKey = prefixKeyToReceive.receive();
return DatabaseSetDictionary.tail(dictionary, prefixKey, keySerializer, null);
}).doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
});
}
public int getKeyBinaryLength() {

View File

@ -156,9 +156,7 @@ public class CachedIndexSearcherManager implements IndexSearcherManager {
indexSearcher.setSimilarity(similarity);
assert indexSearcher.getIndexReader().getRefCount() > 0;
return new LLIndexSearcher(indexSearcher, decRef, this::dropCachedIndexSearcher).send();
})
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
});
}
private void dropCachedIndexSearcher() {

View File

@ -65,6 +65,7 @@ import org.rocksdb.Snapshot;
import org.rocksdb.WriteBatch;
import org.rocksdb.WriteOptions;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
@ -261,7 +262,7 @@ public class LLLocalDictionary implements LLDictionary {
Mono<Send<Buffer>> keyMono,
boolean existsAlmostCertainly) {
return keyMono
.publishOn(Schedulers.boundedElastic())
.publishOn(dbScheduler)
.<Send<Buffer>>handle((keySend, sink) -> {
try (var key = keySend.receive()) {
try {
@ -290,15 +291,15 @@ public class LLLocalDictionary implements LLDictionary {
@Override
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
return rangeMono
.publishOn(Schedulers.boundedElastic())
.<Boolean>handle((rangeSend, sink) -> {
.publishOn(dbScheduler)
.handle((rangeSend, sink) -> {
try (var range = rangeSend.receive()) {
sink.next(containsRange(snapshot, range));
boolean rangeEmpty = !containsRange(snapshot, range);
sink.next(rangeEmpty);
} catch (Throwable ex) {
sink.error(ex);
}
})
.map(isContained -> !isContained);
});
}
public boolean containsRange(@Nullable LLSnapshot snapshot, LLRange range) throws RocksDBException {
@ -367,7 +368,7 @@ public class LLLocalDictionary implements LLDictionary {
private Mono<Boolean> containsKey(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
return keyMono
.publishOn(Schedulers.boundedElastic())
.publishOn(dbScheduler)
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
sink.next(containsKey(snapshot, key));
@ -404,7 +405,7 @@ public class LLLocalDictionary implements LLDictionary {
var previousDataMono = this.getPreviousData(keyMono, resultType, false);
// Write the new entry to the database
var putMono = entryMono
.publishOn(Schedulers.boundedElastic())
.publishOn(dbScheduler)
.<Void>handle((entry, sink) -> {
try (var key = entry.getKey().receive()) {
try (var value = entry.getValue().receive()) {
@ -423,16 +424,6 @@ public class LLLocalDictionary implements LLDictionary {
return Flux
.concat(previousDataMono, putMono.then(Mono.empty()))
.singleOrEmpty()
// Clean discarded elements
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Entry.class, entry -> {
if (entry.getKey() instanceof SafeCloseable safeCloseable) {
safeCloseable.close();
}
if (entry.getValue() instanceof SafeCloseable safeCloseable) {
safeCloseable.close();
}
})
.onErrorMap(cause -> new IOException("Failed to write", cause))
.elapsed()
.map(tuple -> {
@ -513,15 +504,7 @@ public class LLLocalDictionary implements LLDictionary {
assert result != null;
return ((UpdateAtomicResultDelta) result).delta();
}).onErrorMap(cause -> new IOException("Failed to read or write", cause)),
keySend -> Mono.fromRunnable(keySend::close)).doOnDiscard(UpdateAtomicResult.class, uar -> {
if (uar instanceof UpdateAtomicResultDelta delta) {
delta.delta().close();
} else if (uar instanceof UpdateAtomicResultCurrent cur) {
cur.current().close();
} else if (uar instanceof UpdateAtomicResultPrevious cur) {
cur.previous().close();
}
});
keySend -> Mono.fromRunnable(keySend::close));
}
@Override
@ -590,19 +573,6 @@ public class LLLocalDictionary implements LLDictionary {
boolean existsAlmostCertainly) {
return keys
.buffer(MULTI_GET_WINDOW)
.doOnDiscard(Tuple2.class, discardedEntry -> {
if (discardedEntry.getT2() instanceof Resource<?> resource) {
resource.close();
}
})
.doOnDiscard(Tuple3.class, discardedEntry -> {
if (discardedEntry.getT2() instanceof Resource<?> resource) {
resource.close();
}
if (discardedEntry.getT3() instanceof Resource<?> resource) {
resource.close();
}
})
.flatMapSequential(keysWindow -> runOnDb(() -> {
List<Buffer> keyBufsWindow = new ArrayList<>(keysWindow.size());
for (Send<Buffer> bufferSend : keysWindow) {
@ -637,13 +607,7 @@ public class LLLocalDictionary implements LLDictionary {
})
.flatMapIterable(list -> list)
.onErrorMap(cause -> new IOException("Failed to read keys", cause))
.doAfterTerminate(() -> keysWindow.forEach(Send::close)), 2) // Max concurrency is 2 to read data while preparing the next segment
.doOnDiscard(LLEntry.class, ResourceSupport::close)
.doOnDiscard(Optional.class, opt -> {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
});
.doAfterTerminate(() -> keysWindow.forEach(Send::close)), 2); // Max concurrency is 2 to read data while preparing the next segment;
}
@Override
@ -711,8 +675,7 @@ public class LLLocalDictionary implements LLDictionary {
}
}
}).subscribeOn(dbScheduler), 2) // Max concurrency is 2 to read data while preparing the next segment
.flatMapIterable(oldValuesList -> oldValuesList)
.transform(LLUtils::handleDiscard);
.flatMapIterable(oldValuesList -> oldValuesList);
}
@Override
@ -820,15 +783,7 @@ public class LLLocalDictionary implements LLDictionary {
tuple.getT2().close();
}
}
}).flatMapIterable(list -> list), /* Max concurrency is 2 to update data while preparing the next segment */ 2)
.doOnDiscard(Tuple2.class, entry -> {
if (entry.getT1() instanceof Buffer bb) {
bb.close();
}
if (entry.getT2() instanceof Buffer bb) {
bb.close();
}
});
}).flatMapIterable(list -> list), /* Max concurrency is 2 to update data while preparing the next segment */ 2);
}
@Override
@ -875,8 +830,7 @@ public class LLLocalDictionary implements LLDictionary {
return Mono
.zip(keyMono, this.get(snapshot, keyMono, existsAlmostCertainly))
.map(result -> LLEntry.of(result.getT1(), result.getT2()).send())
.flux()
.transform(LLUtils::handleDiscard);
.flux();
}
private Flux<Send<LLEntry>> getRangeMulti(LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
@ -886,7 +840,7 @@ public class LLLocalDictionary implements LLDictionary {
nettyDirect, resolveSnapshot(snapshot)),
iterator -> iterator.flux().subscribeOn(dbScheduler, false),
LLLocalReactiveRocksIterator::close
).transform(LLUtils::handleDiscard),
),
rangeSend -> Mono.fromRunnable(rangeSend::close)
);
}
@ -899,7 +853,7 @@ public class LLLocalDictionary implements LLDictionary {
nettyDirect, resolveSnapshot(snapshot)),
iterator -> iterator.flux().subscribeOn(dbScheduler, false),
LLLocalGroupedReactiveRocksIterator::close
).transform(LLUtils::handleDiscard),
),
rangeSend -> Mono.fromRunnable(rangeSend::close)
);
}
@ -930,7 +884,7 @@ public class LLLocalDictionary implements LLDictionary {
nettyDirect, resolveSnapshot(snapshot)),
iterator -> iterator.flux().subscribeOn(dbScheduler, false),
LLLocalGroupedReactiveRocksIterator::close
).transform(LLUtils::handleDiscard),
),
rangeSend -> Mono.fromRunnable(rangeSend::close)
);
}
@ -1015,8 +969,7 @@ public class LLLocalDictionary implements LLDictionary {
sink.complete();
}
})
.flux()
.doOnDiscard(Buffer.class, Buffer::close),
.flux(),
keySend -> Mono.fromRunnable(keySend::close)
);
}
@ -1029,7 +982,7 @@ public class LLLocalDictionary implements LLDictionary {
),
iterator -> iterator.flux().subscribeOn(dbScheduler, false),
LLLocalReactiveRocksIterator::close
).transform(LLUtils::handleDiscard),
),
rangeSend -> Mono.fromRunnable(rangeSend::close)
);
}

View File

@ -8,13 +8,18 @@ import io.micrometer.core.instrument.MeterRegistry;
import io.micrometer.core.instrument.Tag;
import io.micrometer.core.instrument.Timer;
import io.net5.buffer.api.BufferAllocator;
import io.net5.buffer.api.Resource;
import io.net5.buffer.api.Send;
import io.net5.util.internal.PlatformDependent;
import it.cavallium.dbengine.client.DatabaseOptions;
import it.cavallium.dbengine.client.DatabaseVolume;
import it.cavallium.dbengine.client.MemoryStats;
import it.cavallium.dbengine.database.Column;
import it.cavallium.dbengine.database.LLEntry;
import it.cavallium.dbengine.database.LLKeyValueDatabase;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.SafeCloseable;
import it.cavallium.dbengine.database.UpdateMode;
import java.io.File;
import java.io.IOException;
@ -32,6 +37,7 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
@ -66,14 +72,22 @@ import org.rocksdb.TxnDBWritePolicy;
import org.rocksdb.WALRecoveryMode;
import org.rocksdb.WriteBufferManager;
import org.rocksdb.util.SizeUnit;
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
import reactor.util.function.Tuple2;
import reactor.util.function.Tuple3;
import reactor.util.function.Tuple4;
import reactor.util.function.Tuple5;
import reactor.util.function.Tuple6;
import reactor.util.function.Tuple7;
public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
static {
RocksDB.loadLibrary();
LLUtils.initHooks();
}
protected static final Logger logger = LogManager.getLogger(LLLocalKeyValueDatabase.class);

View File

@ -85,8 +85,7 @@ import reactor.util.function.Tuple2;
public class LLLocalLuceneIndex implements LLLuceneIndex {
protected static final Logger logger = LogManager.getLogger(LLLocalLuceneIndex.class);
private final LocalSearcher localSearcher;
private final DecimalBucketMultiSearcher decimalBucketMultiSearcher = new DecimalBucketMultiSearcher();
/**
* Global lucene index scheduler.
* There is only a single thread globally to not overwhelm the disk with
@ -95,6 +94,13 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
private static final ReentrantLock shutdownLock = new ReentrantLock();
private static final Scheduler luceneHeavyTasksScheduler = uninterruptibleScheduler(Schedulers.single(Schedulers.boundedElastic()));
static {
LLUtils.initHooks();
}
private final LocalSearcher localSearcher;
private final DecimalBucketMultiSearcher decimalBucketMultiSearcher = new DecimalBucketMultiSearcher();
private final Counter startedDocIndexings;
private final Counter endeddDocIndexings;
private final Timer docIndexingTime;
@ -443,9 +449,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return localSearcher
.collect(searcher, localQueryParams, keyFieldName, transformer)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close));
}
@Override
@ -456,9 +460,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
return localSearcher
.collect(searcher, localQueryParams, keyFieldName, NO_TRANSFORMATION)
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close));
}
@Override
@ -475,17 +477,11 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
.retrieveSearcher(snapshot)
.map(indexSearcher -> LLIndexSearchers.unsharded(indexSearcher).send());
return decimalBucketMultiSearcher
.collectMulti(searchers, bucketParams, localQueries, localNormalizationQuery)
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
return decimalBucketMultiSearcher.collectMulti(searchers, bucketParams, localQueries, localNormalizationQuery);
}
public Mono<Send<LLIndexSearcher>> retrieveSearcher(@Nullable LLSnapshot snapshot) {
return searcherManager
.retrieveSearcher(snapshot)
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
return searcherManager.retrieveSearcher(snapshot);
}
@Override

View File

@ -18,6 +18,7 @@ import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLSearchResultShard;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLTerm;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.lucene.LuceneHacks;
import it.cavallium.dbengine.lucene.LuceneUtils;
import it.cavallium.dbengine.lucene.collector.Buckets;
@ -52,6 +53,10 @@ import reactor.util.function.Tuple2;
public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
static {
LLUtils.initHooks();
}
private final MeterRegistry meterRegistry;
private final ConcurrentHashMap<Long, LLSnapshot[]> registeredSnapshots = new ConcurrentHashMap<>();
private final AtomicLong nextSnapshotNumber = new AtomicLong(1);
@ -237,9 +242,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
return multiSearcher
.collectMulti(searchers, localQueryParams, keyFieldName, transformer)
// Transform the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close));
}
@Override
@ -253,8 +256,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
return multiSearcher
.collectMulti(searchers, localQueryParams, keyFieldName, LLSearchTransformer.NO_TRANSFORMATION)
// Transform the result type
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close))
.doOnDiscard(Send.class, Send::close).doOnDiscard(Resource.class, Resource::close);
.map(result -> new LLSearchResultShard(result.results(), result.totalHitsCount(), result::close));
}
@Override
@ -270,10 +272,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
var searchers = getIndexSearchers(snapshot);
// Collect all the shards results into a single global result
return decimalBucketMultiSearcher
.collectMulti(searchers, bucketParams, localQueries, localNormalizationQuery)
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
return decimalBucketMultiSearcher.collectMulti(searchers, bucketParams, localQueries, localNormalizationQuery);
}
@Override

View File

@ -538,18 +538,14 @@ public class LLMemoryDictionary implements LLDictionary {
public Mono<Send<LLEntry>> getOne(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
return getRange(snapshot, rangeMono)
.take(1, true)
.singleOrEmpty()
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.singleOrEmpty();
}
@Override
public Mono<Send<Buffer>> getOneKey(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
return getRangeKeys(snapshot, rangeMono)
.take(1, true)
.singleOrEmpty()
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.singleOrEmpty();
}
@Override

View File

@ -8,6 +8,7 @@ import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLKeyValueDatabase;
import it.cavallium.dbengine.database.LLSingleton;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateMode;
import it.unimi.dsi.fastutil.bytes.ByteList;
import java.nio.charset.StandardCharsets;
@ -19,6 +20,10 @@ import reactor.core.publisher.Mono;
public class LLMemoryKeyValueDatabase implements LLKeyValueDatabase {
static {
LLUtils.initHooks();
}
private final BufferAllocator allocator;
private final MeterRegistry meterRegistry;
private final String name;

View File

@ -134,9 +134,7 @@ public class CountMultiSearcher implements MultiSearcher {
},
is -> Mono.empty()
)
.map(count -> new LuceneSearchResult(TotalHitsCount.of(count, true), Flux.empty(), null))
.doOnDiscard(Send.class, Send::close)
.doOnDiscard(Resource.class, Resource::close);
.map(count -> new LuceneSearchResult(TotalHitsCount.of(count, true), Flux.empty(), null));
}
@Override

View File

@ -99,27 +99,7 @@ public class SortedByScoreFullMultiSearcher implements MultiSearcher {
throw ex;
}
}))
)
.doOnDiscard(List.class, list -> {
try {
for (Object o : list) {
if (o instanceof FullDocsCollector<?,?,?> fullDocsCollector) {
fullDocsCollector.close();
}
}
} catch (Exception ex) {
logger.error("Failed to discard collector", ex);
}
})
.doOnDiscard(FullDocsCollector.class, fullDocsCollector -> {
try {
fullDocsCollector.close();
} catch (Exception ex) {
logger.error("Failed to discard collector", ex);
}
})
.doOnDiscard(FullDocs.class, SafeCloseable::close);
);
}
/**

View File

@ -94,27 +94,7 @@ public class SortedScoredFullMultiSearcher implements MultiSearcher {
throw ex;
}
}))
)
.doOnDiscard(List.class, list -> {
try {
for (Object o : list) {
if (o instanceof LMDBFullFieldDocCollector fullDocsCollector) {
fullDocsCollector.close();
}
}
} catch (Exception ex) {
logger.error("Failed to discard collector", ex);
}
})
.doOnDiscard(LMDBFullFieldDocCollector.class, fullDocsCollector -> {
try {
fullDocsCollector.close();
} catch (Exception ex) {
logger.error("Failed to discard collector", ex);
}
})
.doOnDiscard(FullDocs.class, SafeCloseable::close);
);
}
/**

View File

@ -54,10 +54,9 @@ public abstract class TestDictionaryMap {
} else {
badKeys = List.of();
}
List<Tuple2<String, Boolean>> keys = Stream.concat(
goodKeys.stream().map(s -> Tuples.of(s, false)),
badKeys.stream().map(s -> Tuples.of(s, true))
).collect(Collectors.toList());
List<Tuple2<String, Boolean>> keys = Stream
.concat(goodKeys.stream().map(s -> Tuples.of(s, false)), badKeys.stream().map(s -> Tuples.of(s, true)))
.toList();
var values = isCIMode() ? List.of("val") : List.of("", "\0", BIG_STRING);
return keys
@ -249,7 +248,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (updateMode == UpdateMode.DISALLOW || shouldFail) {
stpVer.verifyError();
@ -316,7 +314,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (updateMode == UpdateMode.DISALLOW || shouldFail) {
stpVer.verifyError();
@ -405,7 +402,6 @@ public abstract class TestDictionaryMap {
.filter(entry -> entry.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -436,7 +432,6 @@ public abstract class TestDictionaryMap {
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -464,7 +459,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -500,7 +494,6 @@ public abstract class TestDictionaryMap {
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -620,7 +613,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -650,7 +642,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -685,7 +676,6 @@ public abstract class TestDictionaryMap {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -715,7 +705,6 @@ public abstract class TestDictionaryMap {
.doFinally(s -> map.close())
)
.flatMap(val -> shouldFail ? Mono.empty() : Mono.just(val))
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -743,7 +732,6 @@ public abstract class TestDictionaryMap {
.doFinally(s -> map.close())
)
.flatMap(val -> shouldFail ? Mono.empty() : Mono.just(val))
.transform(LLUtils::handleDiscard)
.collectList()
).singleOrEmpty());
} catch (Exception ex) {

View File

@ -627,7 +627,6 @@ public abstract class TestDictionaryMapDeep {
)
)
.doFinally(s -> map.close())
.transform(LLUtils::handleDiscard)
)
));
if (updateMode == UpdateMode.DISALLOW || shouldFail) {
@ -730,7 +729,6 @@ public abstract class TestDictionaryMapDeep {
)
)
.doFinally(s -> map.close())
.transform(LLUtils::handleDiscard)
)
));
if (updateMode == UpdateMode.DISALLOW || shouldFail) {
@ -817,7 +815,6 @@ public abstract class TestDictionaryMapDeep {
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
);
if (shouldFail) {
this.checkLeaks = false;
@ -874,7 +871,6 @@ public abstract class TestDictionaryMapDeep {
map.setAllValuesAndGetPrevious(Flux.fromIterable(entries.entrySet()))
)
.doFinally(s -> map.close())
.transform(LLUtils::handleDiscard)
)
));
if (shouldFail) {
@ -908,7 +904,6 @@ public abstract class TestDictionaryMapDeep {
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -945,7 +940,6 @@ public abstract class TestDictionaryMapDeep {
)
.doFinally(s -> map.close());
})
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -999,7 +993,6 @@ public abstract class TestDictionaryMapDeep {
.concatMapIterable(list -> list)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;
@ -1120,7 +1113,6 @@ public abstract class TestDictionaryMapDeep {
)
.doFinally(s -> map.close())
)
.transform(LLUtils::handleDiscard)
));
if (shouldFail) {
this.checkLeaks = false;

View File

@ -156,12 +156,12 @@ public abstract class TestLLDictionary {
var dict = getDict(updateMode);
var keyEx = Mono.fromCallable(() -> fromString("test-key-1").send());
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent").send());
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)));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx).map(this::toString)));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx, true).map(this::toString)));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx, false).map(this::toString)));
Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx).map(this::toString)));
Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx, true).map(this::toString)));
Assertions.assertEquals((String) null, run(dict.get(null, keyNonEx, false).map(this::toString)));
}
@ParameterizedTest
@ -211,17 +211,17 @@ public abstract class TestLLDictionary {
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)
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, true).then()
);
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)
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, false).then()
);
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)
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).then()
);
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
assertEquals(0, afterSize - beforeSize);
@ -236,17 +236,17 @@ public abstract class TestLLDictionary {
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)
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, true).then()
);
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)
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, false).then()
);
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)
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).then()
);
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
assertEquals(expected, afterSize - beforeSize);

View File

@ -133,9 +133,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));
runVoid(dict.get(null, key).then());
runVoid(dict.get(null, key, true).then());
runVoid(dict.get(null, key, false).then());
}
@ParameterizedTest
@ -160,13 +160,13 @@ public abstract class TestLLDictionaryLeaks {
var dict = getDict(updateMode);
var key = Mono.fromCallable(() -> fromString("test-key"));
runVoid(updateMode == UpdateMode.DISALLOW,
dict.update(key, this::pass, updateReturnMode, true).then().transform(LLUtils::handleDiscard)
dict.update(key, this::pass, updateReturnMode, true).then()
);
runVoid(updateMode == UpdateMode.DISALLOW,
dict.update(key, this::pass, updateReturnMode, false).then().transform(LLUtils::handleDiscard)
dict.update(key, this::pass, updateReturnMode, false).then()
);
runVoid(updateMode == UpdateMode.DISALLOW,
dict.update(key, this::pass, updateReturnMode).then().transform(LLUtils::handleDiscard)
dict.update(key, this::pass, updateReturnMode).then()
);
}
@ -180,13 +180,13 @@ public abstract class TestLLDictionaryLeaks {
var dict = getDict(updateMode);
var key = Mono.fromCallable(() -> fromString("test-key"));
runVoid(updateMode == UpdateMode.DISALLOW,
dict.updateAndGetDelta(key, this::pass, true).then().transform(LLUtils::handleDiscard)
dict.updateAndGetDelta(key, this::pass, true).then()
);
runVoid(updateMode == UpdateMode.DISALLOW,
dict.updateAndGetDelta(key, this::pass, false).then().transform(LLUtils::handleDiscard)
dict.updateAndGetDelta(key, this::pass, false).then()
);
runVoid(updateMode == UpdateMode.DISALLOW,
dict.updateAndGetDelta(key, this::pass).then().transform(LLUtils::handleDiscard)
dict.updateAndGetDelta(key, this::pass).then()
);
}