2021-01-30 20:16:14 +01:00
|
|
|
package it.cavallium.dbengine.database.collections;
|
|
|
|
|
2022-05-20 10:44:00 +02:00
|
|
|
import static java.util.Objects.requireNonNullElseGet;
|
|
|
|
|
2022-03-16 13:47:56 +01:00
|
|
|
import io.netty5.buffer.api.Buffer;
|
|
|
|
import io.netty5.buffer.api.Resource;
|
2022-05-20 10:20:00 +02:00
|
|
|
import io.netty5.buffer.api.internal.ResourceSupport;
|
2021-01-30 20:16:14 +01:00
|
|
|
import it.cavallium.dbengine.client.CompositeSnapshot;
|
2022-05-20 10:20:00 +02:00
|
|
|
import it.cavallium.dbengine.database.BufSupplier;
|
2021-05-08 03:09:00 +02:00
|
|
|
import it.cavallium.dbengine.database.Delta;
|
2021-01-30 20:16:14 +01:00
|
|
|
import it.cavallium.dbengine.database.LLDictionary;
|
2021-01-31 21:23:43 +01:00
|
|
|
import it.cavallium.dbengine.database.LLDictionaryResultType;
|
2021-08-28 22:42:51 +02:00
|
|
|
import it.cavallium.dbengine.database.LLEntry;
|
2022-01-22 23:21:40 +01:00
|
|
|
import it.cavallium.dbengine.database.LLRange;
|
2021-01-31 21:23:43 +01:00
|
|
|
import it.cavallium.dbengine.database.LLUtils;
|
2022-05-20 23:59:56 +02:00
|
|
|
import it.cavallium.dbengine.database.SubStageEntry;
|
2021-05-02 19:18:15 +02:00
|
|
|
import it.cavallium.dbengine.database.UpdateMode;
|
2021-05-08 03:09:00 +02:00
|
|
|
import it.cavallium.dbengine.database.UpdateReturnMode;
|
2022-04-01 01:30:56 +02:00
|
|
|
import it.cavallium.dbengine.database.disk.BinarySerializationFunction;
|
2021-11-08 10:49:59 +01:00
|
|
|
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
|
2021-08-22 21:23:22 +02:00
|
|
|
import it.cavallium.dbengine.database.serialization.SerializationException;
|
|
|
|
import it.cavallium.dbengine.database.serialization.SerializationFunction;
|
2021-02-02 19:40:37 +01:00
|
|
|
import it.cavallium.dbengine.database.serialization.Serializer;
|
|
|
|
import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength;
|
2021-12-18 18:16:56 +01:00
|
|
|
import it.unimi.dsi.fastutil.objects.Object2ObjectLinkedOpenHashMap;
|
|
|
|
import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMap;
|
|
|
|
import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMaps;
|
2021-06-06 02:23:51 +02:00
|
|
|
import java.util.Collections;
|
2021-01-30 20:16:14 +01:00
|
|
|
import java.util.Map;
|
|
|
|
import java.util.Map.Entry;
|
2021-05-02 19:18:15 +02:00
|
|
|
import java.util.Objects;
|
2021-07-23 15:20:33 +02:00
|
|
|
import java.util.Optional;
|
2022-03-11 17:59:46 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
import org.apache.logging.log4j.LogManager;
|
|
|
|
import org.apache.logging.log4j.Logger;
|
2021-09-23 02:15:58 +02:00
|
|
|
import org.jetbrains.annotations.NotNull;
|
2021-01-30 20:16:14 +01:00
|
|
|
import org.jetbrains.annotations.Nullable;
|
|
|
|
import reactor.core.publisher.Flux;
|
|
|
|
import reactor.core.publisher.Mono;
|
2021-08-22 21:23:22 +02:00
|
|
|
import reactor.core.publisher.SynchronousSink;
|
2021-01-30 20:16:14 +01:00
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
/**
|
|
|
|
* Optimized implementation of "DatabaseMapDictionary with SubStageGetterSingle"
|
|
|
|
*/
|
|
|
|
public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U, DatabaseStageEntry<U>> {
|
2021-01-30 20:16:14 +01:00
|
|
|
|
2022-03-11 17:59:46 +01:00
|
|
|
private static final Logger LOG = LogManager.getLogger(DatabaseMapDictionary.class);
|
|
|
|
|
|
|
|
private final AtomicLong totalZeroBytesErrors = new AtomicLong();
|
2021-09-02 17:15:40 +02:00
|
|
|
private final Serializer<U> valueSerializer;
|
2021-01-30 20:16:14 +01:00
|
|
|
|
2021-02-01 11:00:27 +01:00
|
|
|
protected DatabaseMapDictionary(LLDictionary dictionary,
|
2021-11-08 16:33:41 +01:00
|
|
|
@Nullable Buffer prefixKey,
|
2021-09-02 17:15:40 +02:00
|
|
|
SerializerFixedBinaryLength<T> keySuffixSerializer,
|
2021-09-23 20:57:28 +02:00
|
|
|
Serializer<U> valueSerializer,
|
2021-10-01 19:17:33 +02:00
|
|
|
Runnable onClose) {
|
2021-04-30 19:15:04 +02:00
|
|
|
// Do not retain or release or use the prefixKey here
|
2021-10-01 19:17:33 +02:00
|
|
|
super(dictionary, prefixKey, keySuffixSerializer, new SubStageGetterSingle<>(valueSerializer), 0, onClose);
|
2021-01-31 21:23:43 +01:00
|
|
|
this.valueSerializer = valueSerializer;
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
public static <T, U> DatabaseMapDictionary<T, U> simple(LLDictionary dictionary,
|
2021-09-02 17:15:40 +02:00
|
|
|
SerializerFixedBinaryLength<T> keySerializer,
|
2021-09-23 20:57:28 +02:00
|
|
|
Serializer<U> valueSerializer,
|
2021-10-01 19:17:33 +02:00
|
|
|
Runnable onClose) {
|
2022-05-20 18:31:05 +02:00
|
|
|
return new DatabaseMapDictionary<>(dictionary, null, keySerializer, valueSerializer, onClose);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
public static <T, U> DatabaseMapDictionary<T, U> tail(LLDictionary dictionary,
|
2021-11-08 16:33:41 +01:00
|
|
|
@Nullable Buffer prefixKey,
|
2021-09-02 17:15:40 +02:00
|
|
|
SerializerFixedBinaryLength<T> keySuffixSerializer,
|
2021-09-23 20:57:28 +02:00
|
|
|
Serializer<U> valueSerializer,
|
2021-10-01 19:17:33 +02:00
|
|
|
Runnable onClose) {
|
|
|
|
return new DatabaseMapDictionary<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer, onClose);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2022-03-20 14:33:27 +01:00
|
|
|
public static <K, V> Flux<Entry<K, V>> getLeavesFrom(DatabaseMapDictionary<K, V> databaseMapDictionary,
|
|
|
|
CompositeSnapshot snapshot,
|
2022-03-25 00:27:44 +01:00
|
|
|
Mono<K> keyMin,
|
|
|
|
Mono<K> keyMax,
|
2022-03-24 23:56:23 +01:00
|
|
|
boolean reverse, boolean smallRange) {
|
2022-03-25 00:27:44 +01:00
|
|
|
Mono<Optional<K>> keyMinOptMono = keyMin.map(Optional::of).defaultIfEmpty(Optional.empty());
|
|
|
|
Mono<Optional<K>> keyMaxOptMono = keyMax.map(Optional::of).defaultIfEmpty(Optional.empty());
|
|
|
|
|
|
|
|
return Mono.zip(keyMinOptMono, keyMaxOptMono).flatMapMany(entry -> {
|
|
|
|
var keyMinOpt = entry.getT1();
|
|
|
|
var keyMaxOpt = entry.getT2();
|
|
|
|
if (keyMinOpt.isPresent() || keyMaxOpt.isPresent()) {
|
|
|
|
return databaseMapDictionary.getAllValues(snapshot,
|
|
|
|
keyMinOpt.orElse(null),
|
|
|
|
keyMaxOpt.orElse(null),
|
|
|
|
reverse,
|
|
|
|
smallRange
|
|
|
|
);
|
2022-03-20 14:33:27 +01:00
|
|
|
} else {
|
2022-03-24 23:56:23 +01:00
|
|
|
return databaseMapDictionary.getAllValues(snapshot, smallRange);
|
2022-03-20 14:33:27 +01:00
|
|
|
}
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2022-03-24 21:14:17 +01:00
|
|
|
public static <K> Flux<K> getKeyLeavesFrom(DatabaseMapDictionary<K, ?> databaseMapDictionary,
|
|
|
|
CompositeSnapshot snapshot,
|
2022-03-25 00:27:44 +01:00
|
|
|
Mono<K> keyMin,
|
|
|
|
Mono<K> keyMax,
|
2022-03-24 23:56:23 +01:00
|
|
|
boolean reverse, boolean smallRange) {
|
2022-03-25 00:27:44 +01:00
|
|
|
Mono<Optional<K>> keyMinOptMono = keyMin.map(Optional::of).defaultIfEmpty(Optional.empty());
|
|
|
|
Mono<Optional<K>> keyMaxOptMono = keyMax.map(Optional::of).defaultIfEmpty(Optional.empty());
|
2022-03-24 21:14:17 +01:00
|
|
|
|
2022-03-25 00:27:44 +01:00
|
|
|
return Mono.zip(keyMinOptMono, keyMaxOptMono).flatMapMany(keys -> {
|
|
|
|
var keyMinOpt = keys.getT1();
|
|
|
|
var keyMaxOpt = keys.getT2();
|
2022-03-24 21:14:17 +01:00
|
|
|
Flux<? extends Entry<K, ? extends DatabaseStageEntry<?>>> stagesFlux;
|
2022-03-25 00:27:44 +01:00
|
|
|
if (keyMinOpt.isPresent() || keyMaxOpt.isPresent()) {
|
2022-03-24 21:14:17 +01:00
|
|
|
stagesFlux = databaseMapDictionary
|
2022-03-25 00:27:44 +01:00
|
|
|
.getAllStages(snapshot, keyMinOpt.orElse(null), keyMaxOpt.orElse(null), reverse, smallRange);
|
2022-03-24 21:14:17 +01:00
|
|
|
} else {
|
2022-03-24 23:56:23 +01:00
|
|
|
stagesFlux = databaseMapDictionary.getAllStages(snapshot, smallRange);
|
2022-03-24 21:14:17 +01:00
|
|
|
}
|
|
|
|
return stagesFlux.doOnNext(e -> e.getValue().close())
|
|
|
|
.doOnDiscard(Entry.class, e -> {
|
|
|
|
if (e.getValue() instanceof DatabaseStageEntry<?> resource) {
|
|
|
|
resource.close();
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.map(Entry::getKey);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2022-05-20 23:59:56 +02:00
|
|
|
private @Nullable U deserializeValue(T keySuffix, Buffer value) {
|
2022-05-20 18:31:05 +02:00
|
|
|
try {
|
2022-05-20 23:59:56 +02:00
|
|
|
return valueSerializer.deserialize(value);
|
2022-05-20 18:31:05 +02:00
|
|
|
} catch (IndexOutOfBoundsException ex) {
|
|
|
|
var exMessage = ex.getMessage();
|
|
|
|
if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) {
|
|
|
|
var totalZeroBytesErrors = this.totalZeroBytesErrors.incrementAndGet();
|
|
|
|
if (totalZeroBytesErrors < 512 || totalZeroBytesErrors % 10000 == 0) {
|
|
|
|
try (var keySuffixBytes = serializeKeySuffixToKey(keySuffix)) {
|
2022-05-20 23:59:56 +02:00
|
|
|
LOG.error(
|
|
|
|
"Unexpected zero-bytes value at "
|
|
|
|
+ dictionary.getDatabaseName() + ":" + dictionary.getColumnName()
|
|
|
|
+ ":" + LLUtils.toStringSafe(this.keyPrefix) + ":" + keySuffix
|
|
|
|
+ "(" + LLUtils.toStringSafe(keySuffixBytes) + ") total=" + totalZeroBytesErrors);
|
2022-05-20 18:31:05 +02:00
|
|
|
} catch (SerializationException e) {
|
2022-05-20 23:59:56 +02:00
|
|
|
LOG.error(
|
|
|
|
"Unexpected zero-bytes value at " + dictionary.getDatabaseName() + ":" + dictionary.getColumnName()
|
|
|
|
+ ":" + LLUtils.toStringSafe(this.keyPrefix) + ":" + keySuffix + "(?) total="
|
|
|
|
+ totalZeroBytesErrors);
|
2022-03-18 15:33:54 +01:00
|
|
|
}
|
2022-03-11 17:59:46 +01:00
|
|
|
}
|
2022-05-20 23:59:56 +02:00
|
|
|
return null;
|
2022-05-20 18:31:05 +02:00
|
|
|
} else {
|
2022-05-20 23:59:56 +02:00
|
|
|
throw ex;
|
2022-03-11 17:59:46 +01:00
|
|
|
}
|
2021-10-19 00:22:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-08 16:33:41 +01:00
|
|
|
private Buffer serializeValue(U value) throws SerializationException {
|
2021-10-19 00:22:05 +02:00
|
|
|
var valSizeHint = valueSerializer.getSerializedSizeHint();
|
|
|
|
if (valSizeHint == -1) valSizeHint = 128;
|
2021-11-08 16:33:41 +01:00
|
|
|
var valBuf = dictionary.getAllocator().allocate(valSizeHint);
|
|
|
|
try {
|
2021-10-19 00:22:05 +02:00
|
|
|
valueSerializer.serialize(value, valBuf);
|
2021-11-08 16:33:41 +01:00
|
|
|
return valBuf;
|
|
|
|
} catch (Throwable t) {
|
|
|
|
valBuf.close();
|
|
|
|
throw t;
|
2021-10-19 00:22:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-08 16:33:41 +01:00
|
|
|
private Buffer serializeKeySuffixToKey(T keySuffix) throws SerializationException {
|
|
|
|
Buffer keyBuf;
|
|
|
|
if (keyPrefix != null) {
|
|
|
|
keyBuf = keyPrefix.copy();
|
|
|
|
} else {
|
|
|
|
keyBuf = this.dictionary.getAllocator().allocate(keyPrefixLength + keySuffixLength + keyExtLength);
|
|
|
|
}
|
|
|
|
try {
|
2021-10-19 00:22:05 +02:00
|
|
|
assert keyBuf.readableBytes() == keyPrefixLength;
|
|
|
|
keyBuf.ensureWritable(keySuffixLength + keyExtLength);
|
|
|
|
serializeSuffix(keySuffix, keyBuf);
|
|
|
|
assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
|
2021-11-08 16:33:41 +01:00
|
|
|
return keyBuf;
|
|
|
|
} catch (Throwable t) {
|
|
|
|
keyBuf.close();
|
|
|
|
throw t;
|
2021-10-19 00:22:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-08 16:33:41 +01:00
|
|
|
private Buffer toKey(Buffer suffixKey) {
|
|
|
|
assert suffixKeyLengthConsistency(suffixKey.readableBytes());
|
|
|
|
if (keyPrefix != null && keyPrefix.readableBytes() > 0) {
|
|
|
|
var result = LLUtils.compositeBuffer(dictionary.getAllocator(),
|
|
|
|
LLUtils.copy(dictionary.getAllocator(), keyPrefix),
|
|
|
|
suffixKey.send()
|
|
|
|
);
|
|
|
|
try {
|
|
|
|
assert result.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
|
|
|
|
return result;
|
|
|
|
} catch (Throwable t) {
|
|
|
|
result.close();
|
|
|
|
throw t;
|
2021-09-23 02:15:58 +02:00
|
|
|
}
|
2021-11-08 16:33:41 +01:00
|
|
|
} else {
|
|
|
|
assert suffixKey.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
|
|
|
|
return suffixKey;
|
2021-04-30 19:15:04 +02:00
|
|
|
}
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-12-18 18:16:56 +01:00
|
|
|
public Mono<Object2ObjectSortedMap<T, U>> get(@Nullable CompositeSnapshot snapshot, boolean existsAlmostCertainly) {
|
2021-08-22 18:20:05 +02:00
|
|
|
return dictionary
|
2022-03-24 23:56:23 +01:00
|
|
|
.getRange(resolveSnapshot(snapshot), rangeMono, false, true)
|
2022-05-20 10:20:00 +02:00
|
|
|
.<Entry<T, U>>handle((entry, sink) -> {
|
2021-10-19 00:22:05 +02:00
|
|
|
Entry<T, U> deserializedEntry;
|
|
|
|
try {
|
2022-05-20 10:20:00 +02:00
|
|
|
try (entry) {
|
2021-10-19 00:22:05 +02:00
|
|
|
T key;
|
2022-05-20 10:20:00 +02:00
|
|
|
var serializedKey = entry.getKeyUnsafe();
|
|
|
|
var serializedValue = entry.getValueUnsafe();
|
|
|
|
splitPrefix(serializedKey).close();
|
|
|
|
suffixKeyLengthConsistency(serializedKey.readableBytes());
|
|
|
|
key = deserializeSuffix(serializedKey);
|
|
|
|
U value = valueSerializer.deserialize(serializedValue);
|
2021-10-19 00:22:05 +02:00
|
|
|
deserializedEntry = Map.entry(key, value);
|
2021-09-23 02:15:58 +02:00
|
|
|
}
|
2021-10-19 00:22:05 +02:00
|
|
|
sink.next(deserializedEntry);
|
|
|
|
} catch (Throwable ex) {
|
2021-08-22 21:23:22 +02:00
|
|
|
sink.error(ex);
|
|
|
|
}
|
|
|
|
})
|
2021-12-18 18:16:56 +01:00
|
|
|
.collectMap(Entry::getKey, Entry::getValue, Object2ObjectLinkedOpenHashMap::new)
|
|
|
|
.map(map -> (Object2ObjectSortedMap<T, U>) map)
|
2021-08-22 18:20:05 +02:00
|
|
|
.filter(map -> !map.isEmpty());
|
2021-01-31 19:52:47 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-12-18 18:16:56 +01:00
|
|
|
public Mono<Object2ObjectSortedMap<T, U>> setAndGetPrevious(Object2ObjectSortedMap<T, U> value) {
|
2021-08-22 21:23:22 +02:00
|
|
|
return this
|
|
|
|
.get(null, false)
|
|
|
|
.concatWith(dictionary.setRange(rangeMono, Flux
|
2021-08-22 19:54:23 +02:00
|
|
|
.fromIterable(Collections.unmodifiableMap(value).entrySet())
|
2022-03-24 23:56:23 +01:00
|
|
|
.handle(this::serializeEntrySink), true).then(Mono.empty()))
|
2022-01-26 14:22:54 +01:00
|
|
|
.singleOrEmpty();
|
2021-01-31 19:52:47 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-12-18 18:16:56 +01:00
|
|
|
public Mono<Object2ObjectSortedMap<T, U>> clearAndGetPrevious() {
|
2021-05-02 19:18:15 +02:00
|
|
|
return this
|
2021-12-18 18:16:56 +01:00
|
|
|
.setAndGetPrevious(Object2ObjectSortedMaps.emptyMap());
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-02-24 16:43:07 +01:00
|
|
|
public Mono<Long> leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) {
|
2021-08-22 18:20:05 +02:00
|
|
|
return dictionary.sizeRange(resolveSnapshot(snapshot), rangeMono, fast);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-03-14 03:13:19 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
2022-01-26 16:06:15 +01:00
|
|
|
return dictionary.isRangeEmpty(resolveSnapshot(snapshot), rangeMono, false);
|
2021-03-14 03:13:19 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<DatabaseStageEntry<U>> at(@Nullable CompositeSnapshot snapshot, T keySuffix) {
|
2021-09-02 17:15:40 +02:00
|
|
|
return Mono.fromCallable(() ->
|
2022-05-20 10:20:00 +02:00
|
|
|
new DatabaseMapSingle<>(dictionary, BufSupplier.ofOwned(serializeKeySuffixToKey(keySuffix)), valueSerializer, null));
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2022-01-22 23:21:40 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Boolean> containsKey(@Nullable CompositeSnapshot snapshot, T keySuffix) {
|
|
|
|
return dictionary
|
|
|
|
.isRangeEmpty(resolveSnapshot(snapshot),
|
2022-05-20 10:20:00 +02:00
|
|
|
Mono.fromCallable(() -> LLRange.singleUnsafe(serializeKeySuffixToKey(keySuffix))),
|
2022-01-26 16:06:15 +01:00
|
|
|
true
|
2022-01-22 23:21:40 +01:00
|
|
|
)
|
|
|
|
.map(empty -> !empty);
|
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-03-18 16:19:41 +01:00
|
|
|
public Mono<U> getValue(@Nullable CompositeSnapshot snapshot, T keySuffix, boolean existsAlmostCertainly) {
|
2022-05-20 23:59:56 +02:00
|
|
|
return Mono.usingWhen(dictionary
|
|
|
|
.get(resolveSnapshot(snapshot), Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix))),
|
|
|
|
value -> Mono.fromCallable(() -> deserializeValue(keySuffix, value)),
|
|
|
|
value -> Mono.fromRunnable(value::close));
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> putValue(T keySuffix, U value) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix)).single();
|
|
|
|
var valueMono = Mono.fromCallable(() -> serializeValue(value)).single();
|
2021-08-31 09:14:46 +02:00
|
|
|
return dictionary
|
|
|
|
.put(keyMono, valueMono, LLDictionaryResultType.VOID)
|
2022-05-20 10:20:00 +02:00
|
|
|
.doOnNext(Resource::close)
|
2021-08-31 09:14:46 +02:00
|
|
|
.then();
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-05-02 19:18:15 +02:00
|
|
|
@Override
|
|
|
|
public Mono<UpdateMode> getUpdateMode() {
|
|
|
|
return dictionary.getUpdateMode();
|
|
|
|
}
|
|
|
|
|
2021-02-06 19:21:31 +01:00
|
|
|
@Override
|
2022-05-20 18:31:05 +02:00
|
|
|
public Mono<U> updateValue(T keySuffix, UpdateReturnMode updateReturnMode,
|
2021-08-22 21:23:22 +02:00
|
|
|
SerializationFunction<@Nullable U, @Nullable U> updater) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
2022-05-20 23:59:56 +02:00
|
|
|
return Mono.usingWhen(dictionary.update(keyMono, getSerializedUpdater(updater), updateReturnMode),
|
|
|
|
result -> Mono.fromCallable(() -> deserializeValue(keySuffix, result)),
|
|
|
|
result -> Mono.fromRunnable(result::close)
|
|
|
|
);
|
2021-05-08 03:09:00 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2022-05-20 18:31:05 +02:00
|
|
|
public Mono<Delta<U>> updateValueAndGetDelta(T keySuffix, SerializationFunction<@Nullable U, @Nullable U> updater) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
2022-05-20 18:31:05 +02:00
|
|
|
return dictionary
|
2022-03-02 18:33:58 +01:00
|
|
|
.updateAndGetDelta(keyMono, getSerializedUpdater(updater))
|
2022-05-20 23:59:56 +02:00
|
|
|
.transform(mono -> LLUtils.mapLLDelta(mono, valueSerializer::deserialize));
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
|
2022-05-20 18:31:05 +02:00
|
|
|
public BinarySerializationFunction getSerializedUpdater(SerializationFunction<@Nullable U, @Nullable U> updater) {
|
2021-07-17 11:52:08 +02:00
|
|
|
return oldSerialized -> {
|
2022-05-20 18:31:05 +02:00
|
|
|
U result;
|
|
|
|
if (oldSerialized == null) {
|
|
|
|
result = updater.apply(null);
|
|
|
|
} else {
|
|
|
|
result = updater.apply(valueSerializer.deserialize(oldSerialized));
|
|
|
|
}
|
|
|
|
if (result == null) {
|
|
|
|
return null;
|
|
|
|
} else {
|
|
|
|
return serializeValue(result);
|
2021-07-17 11:52:08 +02:00
|
|
|
}
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2022-05-20 10:20:00 +02:00
|
|
|
public KVSerializationFunction<@NotNull T, @Nullable Buffer, @Nullable Buffer> getSerializedUpdater(
|
2021-11-08 10:49:59 +01:00
|
|
|
KVSerializationFunction<@NotNull T, @Nullable U, @Nullable U> updater) {
|
|
|
|
return (key, oldSerialized) -> {
|
2021-08-31 09:14:46 +02:00
|
|
|
try (oldSerialized) {
|
2021-08-22 21:23:22 +02:00
|
|
|
U result;
|
|
|
|
if (oldSerialized == null) {
|
2021-11-08 10:49:59 +01:00
|
|
|
result = updater.apply(key, null);
|
2021-08-22 21:23:22 +02:00
|
|
|
} else {
|
2022-05-20 10:20:00 +02:00
|
|
|
try (oldSerialized) {
|
|
|
|
result = updater.apply(key, valueSerializer.deserialize(oldSerialized));
|
2021-10-19 00:22:05 +02:00
|
|
|
}
|
2021-08-22 21:23:22 +02:00
|
|
|
}
|
2021-07-17 11:52:08 +02:00
|
|
|
if (result == null) {
|
|
|
|
return null;
|
|
|
|
} else {
|
2021-10-19 00:22:05 +02:00
|
|
|
return serializeValue(result);
|
2021-07-17 11:52:08 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<U> putValueAndGetPrevious(T keySuffix, U value) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
|
|
|
var valueMono = Mono.fromCallable(() -> serializeValue(value));
|
2022-05-20 23:59:56 +02:00
|
|
|
return Mono.usingWhen(dictionary.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE),
|
|
|
|
valueBuf -> Mono.fromCallable(() -> deserializeValue(keySuffix, valueBuf)),
|
|
|
|
valueBuf -> Mono.fromRunnable(valueBuf::close)
|
|
|
|
);
|
2021-01-31 00:36:21 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
2021-05-02 19:18:15 +02:00
|
|
|
public Mono<Boolean> putValueAndGetChanged(T keySuffix, U value) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
|
|
|
var valueMono = Mono.fromCallable(() -> serializeValue(value));
|
2022-05-20 23:59:56 +02:00
|
|
|
return Mono
|
|
|
|
.usingWhen(dictionary.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE),
|
|
|
|
valueBuf -> Mono.fromCallable(() -> deserializeValue(keySuffix, valueBuf)),
|
|
|
|
valueBuf -> Mono.fromRunnable(valueBuf::close)
|
|
|
|
)
|
2021-08-31 09:14:46 +02:00
|
|
|
.map(oldValue -> !Objects.equals(oldValue, value))
|
|
|
|
.defaultIfEmpty(value != null);
|
2021-01-31 00:36:21 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> remove(T keySuffix) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
2021-08-31 09:14:46 +02:00
|
|
|
return dictionary
|
|
|
|
.remove(keyMono, LLDictionaryResultType.VOID)
|
2022-05-20 10:20:00 +02:00
|
|
|
.doOnNext(Resource::close)
|
2021-08-31 09:14:46 +02:00
|
|
|
.then();
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<U> removeAndGetPrevious(T keySuffix) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
2022-05-20 23:59:56 +02:00
|
|
|
return Mono.usingWhen(dictionary.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE),
|
|
|
|
valueBuf -> Mono.fromCallable(() -> deserializeValue(keySuffix, valueBuf)),
|
|
|
|
valueBuf -> Mono.fromRunnable(valueBuf::close)
|
|
|
|
);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 21:23:43 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Boolean> removeAndGetStatus(T keySuffix) {
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
|
2021-08-31 09:14:46 +02:00
|
|
|
return dictionary
|
|
|
|
.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE)
|
|
|
|
.map(LLUtils::responseToBoolean);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-11-08 10:49:59 +01:00
|
|
|
public Flux<Optional<U>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
|
2021-08-31 09:14:46 +02:00
|
|
|
var mappedKeys = keys
|
2022-05-20 10:20:00 +02:00
|
|
|
.<Buffer>handle((keySuffix, sink) -> {
|
2021-08-28 22:42:51 +02:00
|
|
|
try {
|
2022-05-20 10:20:00 +02:00
|
|
|
sink.next(serializeKeySuffixToKey(keySuffix));
|
2021-10-19 00:22:05 +02:00
|
|
|
} catch (Throwable ex) {
|
2021-08-31 09:14:46 +02:00
|
|
|
sink.error(ex);
|
2021-08-22 21:23:22 +02:00
|
|
|
}
|
2021-08-31 09:14:46 +02:00
|
|
|
});
|
|
|
|
return dictionary
|
2022-03-24 21:14:17 +01:00
|
|
|
.getMulti(resolveSnapshot(snapshot), mappedKeys)
|
2022-05-20 23:59:56 +02:00
|
|
|
.handle((valueBufOpt, sink) -> {
|
2021-08-31 09:14:46 +02:00
|
|
|
try {
|
2022-05-20 23:59:56 +02:00
|
|
|
sink.next(valueBufOpt.map(valueSerializer::deserialize));
|
2021-10-19 00:22:05 +02:00
|
|
|
} catch (Throwable ex) {
|
2021-08-31 09:14:46 +02:00
|
|
|
sink.error(ex);
|
2021-09-08 00:22:39 +02:00
|
|
|
} finally {
|
2021-11-08 16:33:41 +01:00
|
|
|
valueBufOpt.ifPresent(Resource::close);
|
2021-08-31 09:14:46 +02:00
|
|
|
}
|
2022-01-26 14:22:54 +01:00
|
|
|
});
|
2021-05-02 19:18:15 +02:00
|
|
|
}
|
|
|
|
|
2021-11-08 16:33:41 +01:00
|
|
|
private LLEntry serializeEntry(T keySuffix, U value) throws SerializationException {
|
|
|
|
var key = serializeKeySuffixToKey(keySuffix);
|
|
|
|
try {
|
|
|
|
var serializedValue = serializeValue(value);
|
|
|
|
return LLEntry.of(key, serializedValue);
|
|
|
|
} catch (Throwable t) {
|
|
|
|
key.close();
|
|
|
|
throw t;
|
2021-05-02 19:18:15 +02:00
|
|
|
}
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2022-05-20 10:20:00 +02:00
|
|
|
private void serializeEntrySink(Entry<T,U> entry, SynchronousSink<LLEntry> sink) {
|
2021-10-19 00:22:05 +02:00
|
|
|
try {
|
2022-05-20 10:20:00 +02:00
|
|
|
sink.next(serializeEntry(entry.getKey(), entry.getValue()));
|
2021-10-19 00:22:05 +02:00
|
|
|
} catch (Throwable e) {
|
|
|
|
sink.error(e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-01 02:21:53 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> putMulti(Flux<Entry<T, U>> entries) {
|
2021-05-02 19:18:15 +02:00
|
|
|
var serializedEntries = entries
|
2022-05-20 10:20:00 +02:00
|
|
|
.<LLEntry>handle((entry, sink) -> {
|
2021-08-28 22:42:51 +02:00
|
|
|
try {
|
2022-05-20 10:20:00 +02:00
|
|
|
sink.next(serializeEntry(entry.getKey(), entry.getValue()));
|
2021-10-19 00:22:05 +02:00
|
|
|
} catch (Throwable e) {
|
2021-08-28 22:42:51 +02:00
|
|
|
sink.error(e);
|
|
|
|
}
|
2021-09-08 00:22:39 +02:00
|
|
|
});
|
2022-01-26 19:03:51 +01:00
|
|
|
return dictionary.putMulti(serializedEntries);
|
2021-02-01 02:21:53 +01:00
|
|
|
}
|
|
|
|
|
2021-07-17 11:52:08 +02:00
|
|
|
@Override
|
2021-11-08 10:49:59 +01:00
|
|
|
public Flux<Boolean> updateMulti(Flux<T> keys,
|
|
|
|
KVSerializationFunction<T, @Nullable U, @Nullable U> updater) {
|
|
|
|
var sharedKeys = keys.publish().refCount(2);
|
2022-05-20 10:20:00 +02:00
|
|
|
var serializedKeys = sharedKeys.<Buffer>handle((key, sink) -> {
|
2022-01-26 14:22:54 +01:00
|
|
|
try {
|
2022-05-20 10:20:00 +02:00
|
|
|
Buffer serializedKey = serializeKeySuffixToKey(key);
|
2022-01-26 14:22:54 +01:00
|
|
|
sink.next(serializedKey);
|
|
|
|
} catch (Throwable ex) {
|
|
|
|
sink.error(ex);
|
|
|
|
}
|
|
|
|
});
|
2021-07-17 11:52:08 +02:00
|
|
|
var serializedUpdater = getSerializedUpdater(updater);
|
2021-11-08 10:49:59 +01:00
|
|
|
return dictionary.updateMulti(sharedKeys, serializedKeys, serializedUpdater);
|
2021-07-17 11:52:08 +02:00
|
|
|
}
|
|
|
|
|
2021-01-30 20:16:14 +01:00
|
|
|
@Override
|
2022-05-20 23:59:56 +02:00
|
|
|
public Flux<SubStageEntry<T, DatabaseStageEntry<U>>> getAllStages(@Nullable CompositeSnapshot snapshot, boolean smallRange) {
|
2022-03-24 23:56:23 +01:00
|
|
|
return getAllStages(snapshot, rangeMono, false, smallRange);
|
2022-03-24 21:14:17 +01:00
|
|
|
}
|
|
|
|
|
2022-05-20 10:20:00 +02:00
|
|
|
private LLRange getPatchedRange(@NotNull LLRange range, @Nullable T keyMin, @Nullable T keyMax)
|
2022-03-25 00:27:44 +01:00
|
|
|
throws SerializationException {
|
2022-05-20 13:16:26 +02:00
|
|
|
Buffer keyMinBuf = requireNonNullElseGet(serializeSuffixForRange(keyMin), range::getMinCopy);
|
|
|
|
Buffer keyMaxBuf = requireNonNullElseGet(serializeSuffixForRange(keyMax), range::getMaxCopy);
|
|
|
|
return LLRange.ofUnsafe(keyMinBuf, keyMaxBuf);
|
2022-03-25 00:27:44 +01:00
|
|
|
}
|
|
|
|
|
2022-05-20 10:20:00 +02:00
|
|
|
private Buffer serializeSuffixForRange(@Nullable T key) throws SerializationException {
|
2022-03-25 00:27:44 +01:00
|
|
|
if (key == null) {
|
|
|
|
return null;
|
|
|
|
}
|
2022-05-20 10:20:00 +02:00
|
|
|
var keyWithoutExtBuf = keyPrefix == null ? alloc.allocate(keySuffixLength + keyExtLength)
|
2022-03-25 00:27:44 +01:00
|
|
|
// todo: use a read-only copy
|
2022-05-20 10:20:00 +02:00
|
|
|
: keyPrefix.copy();
|
|
|
|
try {
|
2022-03-25 00:27:44 +01:00
|
|
|
keyWithoutExtBuf.ensureWritable(keySuffixLength + keyExtLength);
|
|
|
|
serializeSuffix(key, keyWithoutExtBuf);
|
2022-05-20 10:20:00 +02:00
|
|
|
return keyWithoutExtBuf;
|
|
|
|
} catch (Throwable ex) {
|
|
|
|
keyWithoutExtBuf.close();
|
|
|
|
throw ex;
|
2022-03-25 00:27:44 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-03-24 21:14:17 +01:00
|
|
|
/**
|
|
|
|
* Get all stages
|
|
|
|
* @param reverse if true, the results will go backwards from the specified key (inclusive)
|
|
|
|
*/
|
2022-05-20 23:59:56 +02:00
|
|
|
public Flux<SubStageEntry<T, DatabaseStageEntry<U>>> getAllStages(@Nullable CompositeSnapshot snapshot,
|
2022-03-25 00:27:44 +01:00
|
|
|
@Nullable T keyMin,
|
|
|
|
@Nullable T keyMax,
|
2022-03-24 23:56:23 +01:00
|
|
|
boolean reverse,
|
|
|
|
boolean smallRange) {
|
2022-03-25 00:27:44 +01:00
|
|
|
if (keyMin == null && keyMax == null) {
|
2022-03-24 23:56:23 +01:00
|
|
|
return getAllStages(snapshot, smallRange);
|
2022-03-24 21:14:17 +01:00
|
|
|
} else {
|
2022-05-20 23:59:56 +02:00
|
|
|
Mono<LLRange> boundedRangeMono = Mono.usingWhen(rangeMono,
|
|
|
|
range -> Mono.fromCallable(() -> getPatchedRange(range, keyMin, keyMax)),
|
|
|
|
range -> Mono.fromRunnable(range::close)
|
|
|
|
);
|
2022-03-24 23:56:23 +01:00
|
|
|
return getAllStages(snapshot, boundedRangeMono, reverse, smallRange);
|
2022-03-24 21:14:17 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-20 23:59:56 +02:00
|
|
|
private Flux<SubStageEntry<T, DatabaseStageEntry<U>>> getAllStages(@Nullable CompositeSnapshot snapshot,
|
2022-05-20 10:20:00 +02:00
|
|
|
Mono<LLRange> sliceRangeMono, boolean reverse, boolean smallRange) {
|
2021-08-22 18:20:05 +02:00
|
|
|
return dictionary
|
2022-03-24 23:56:23 +01:00
|
|
|
.getRangeKeys(resolveSnapshot(snapshot), sliceRangeMono, reverse, smallRange)
|
2022-05-20 23:59:56 +02:00
|
|
|
.flatMapSequential(keyBuf -> Mono
|
|
|
|
.<SubStageEntry<T, DatabaseStageEntry<U>>>fromCallable(() -> {
|
|
|
|
assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
|
|
|
|
// Remove prefix. Keep only the suffix and the ext
|
|
|
|
splitPrefix(keyBuf).close();
|
|
|
|
suffixKeyLengthConsistency(keyBuf.readableBytes());
|
|
|
|
T keySuffix;
|
|
|
|
try (var keyBufCopy = keyBuf.copy()) {
|
|
|
|
keySuffix = deserializeSuffix(keyBufCopy);
|
|
|
|
}
|
|
|
|
var bufSupplier = BufSupplier.ofOwned(toKey(keyBuf));
|
|
|
|
var subStage = new DatabaseMapSingle<>(dictionary, bufSupplier, valueSerializer, null);
|
|
|
|
return new SubStageEntry<>(keySuffix, subStage);
|
|
|
|
}).doOnCancel(() -> {
|
|
|
|
if (keyBuf.isAccessible()) {
|
|
|
|
keyBuf.close();
|
|
|
|
}
|
|
|
|
}).doOnError(ex -> {
|
|
|
|
if (keyBuf.isAccessible()) {
|
|
|
|
keyBuf.close();
|
|
|
|
}
|
|
|
|
})
|
|
|
|
);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
2021-01-31 12:02:02 +01:00
|
|
|
|
2021-03-22 20:02:19 +01:00
|
|
|
@Override
|
2022-03-24 23:56:23 +01:00
|
|
|
public Flux<Entry<T, U>> getAllValues(@Nullable CompositeSnapshot snapshot, boolean smallRange) {
|
|
|
|
return getAllValues(snapshot, rangeMono, false, smallRange);
|
2022-03-20 14:33:27 +01:00
|
|
|
}
|
|
|
|
|
2022-03-24 21:14:17 +01:00
|
|
|
/**
|
|
|
|
* Get all values
|
|
|
|
* @param reverse if true, the results will go backwards from the specified key (inclusive)
|
|
|
|
*/
|
2022-03-24 23:56:23 +01:00
|
|
|
public Flux<Entry<T, U>> getAllValues(@Nullable CompositeSnapshot snapshot,
|
2022-03-25 00:27:44 +01:00
|
|
|
@Nullable T keyMin,
|
|
|
|
@Nullable T keyMax,
|
2022-03-24 23:56:23 +01:00
|
|
|
boolean reverse,
|
|
|
|
boolean smallRange) {
|
2022-03-25 00:27:44 +01:00
|
|
|
if (keyMin == null && keyMax == null) {
|
2022-03-24 23:56:23 +01:00
|
|
|
return getAllValues(snapshot, smallRange);
|
2022-03-20 14:33:27 +01:00
|
|
|
} else {
|
2022-05-20 23:59:56 +02:00
|
|
|
Mono<LLRange> boundedRangeMono = Mono.usingWhen(rangeMono,
|
|
|
|
range -> Mono.fromCallable(() -> getPatchedRange(range, keyMin, keyMax)),
|
|
|
|
range -> Mono.fromRunnable(range::close));
|
2022-03-24 23:56:23 +01:00
|
|
|
return getAllValues(snapshot, boundedRangeMono, reverse, smallRange);
|
2022-03-20 14:33:27 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-03-24 21:14:17 +01:00
|
|
|
private Flux<Entry<T, U>> getAllValues(@Nullable CompositeSnapshot snapshot,
|
2022-05-20 10:20:00 +02:00
|
|
|
Mono<LLRange> sliceRangeMono,
|
2022-03-24 23:56:23 +01:00
|
|
|
boolean reverse, boolean smallRange) {
|
2021-08-22 18:20:05 +02:00
|
|
|
return dictionary
|
2022-03-24 23:56:23 +01:00
|
|
|
.getRange(resolveSnapshot(snapshot), sliceRangeMono, reverse, smallRange)
|
2022-05-20 23:59:56 +02:00
|
|
|
.handle((serializedEntry, sink) -> {
|
2021-10-19 00:22:05 +02:00
|
|
|
try {
|
|
|
|
Entry<T, U> entry;
|
2022-05-20 10:20:00 +02:00
|
|
|
try (serializedEntry) {
|
2021-11-08 16:33:41 +01:00
|
|
|
var keyBuf = serializedEntry.getKeyUnsafe();
|
|
|
|
assert keyBuf != null;
|
|
|
|
assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
|
|
|
|
// Remove prefix. Keep only the suffix and the ext
|
|
|
|
splitPrefix(keyBuf).close();
|
2022-03-11 17:59:46 +01:00
|
|
|
assert suffixKeyLengthConsistency(keyBuf.readableBytes());
|
2021-11-08 16:33:41 +01:00
|
|
|
T keySuffix = deserializeSuffix(keyBuf);
|
|
|
|
|
|
|
|
assert serializedEntry.getValueUnsafe() != null;
|
|
|
|
U value = valueSerializer.deserialize(serializedEntry.getValueUnsafe());
|
|
|
|
entry = Map.entry(keySuffix, value);
|
2021-09-23 02:15:58 +02:00
|
|
|
}
|
2021-10-19 00:22:05 +02:00
|
|
|
sink.next(entry);
|
|
|
|
} catch (Throwable e) {
|
2021-08-22 21:23:22 +02:00
|
|
|
sink.error(e);
|
|
|
|
}
|
2021-08-22 18:20:05 +02:00
|
|
|
});
|
2021-03-22 20:02:19 +01:00
|
|
|
}
|
|
|
|
|
2021-01-31 15:47:48 +01:00
|
|
|
@Override
|
|
|
|
public Flux<Entry<T, U>> setAllValuesAndGetPrevious(Flux<Entry<T, U>> entries) {
|
2021-08-29 01:15:51 +02:00
|
|
|
return Flux.concat(
|
2022-03-24 23:56:23 +01:00
|
|
|
this.getAllValues(null, false),
|
|
|
|
dictionary.setRange(rangeMono, entries.handle(this::serializeEntrySink), false).then(Mono.empty())
|
2021-08-22 21:23:22 +02:00
|
|
|
);
|
2021-01-31 15:47:48 +01:00
|
|
|
}
|
|
|
|
|
2021-03-14 03:13:19 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Void> clear() {
|
2022-05-20 10:20:00 +02:00
|
|
|
return Mono.using(rangeSupplier::get, range -> {
|
|
|
|
if (range.isAll()) {
|
|
|
|
return dictionary.clear();
|
|
|
|
} else if (range.isSingle()) {
|
|
|
|
return dictionary
|
|
|
|
.remove(Mono.fromCallable(range::getSingleUnsafe), LLDictionaryResultType.VOID)
|
|
|
|
.doOnNext(Resource::close)
|
|
|
|
.then();
|
|
|
|
} else {
|
|
|
|
return dictionary.setRange(rangeMono, Flux.empty(), false);
|
|
|
|
}
|
|
|
|
}, ResourceSupport::close);
|
2021-01-31 12:02:02 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|