Use merge operator when possible

This commit is contained in:
Andrea Cavalli 2021-11-08 10:49:59 +01:00
parent b48ab0b095
commit 63cd178988
14 changed files with 215 additions and 259 deletions

View File

@ -76,15 +76,23 @@ public final class Hits<T> extends ResourceSupport<Hits<T>, Hits<T>> {
ValueTransformer<T, U> valueTransformer) {
return resultToReceive -> {
var result = resultToReceive.receive();
var hitsToTransform = result.results().map(hit -> Tuples.of(hit.score(), hit.key()));
var transformed = valueTransformer
.transform(hitsToTransform)
.filter(tuple3 -> tuple3.getT3().isPresent())
.map(tuple3 -> new LazyHitEntry<>(Mono.just(tuple3.getT2()),
Mono.just(tuple3.getT3().orElseThrow()),
tuple3.getT1()
));
return new Hits<>(transformed, result.totalHitsCount(), result::close).send();
var sharedHitsFlux = result.results().publish().refCount(3);
var scoresFlux = sharedHitsFlux.map(HitKey::score);
var keysFlux = sharedHitsFlux.map(HitKey::key);
var valuesFlux = valueTransformer.transform(keysFlux);
var transformedFlux = Flux.zip((Object[] data) -> {
//noinspection unchecked
var keyMono = Mono.just((T) data[0]);
//noinspection unchecked
var valMono = Mono.just((U) data[1]);
var score = (Float) data[2];
return new LazyHitEntry<>(keyMono, valMono, score);
}, keysFlux, valuesFlux, scoresFlux);
return new Hits<>(transformedFlux, result.totalHitsCount(), result::close).send();
};
}

View File

@ -1,3 +0,0 @@
package it.cavallium.dbengine.database;
public record ExtraKeyOperationResult<T, X>(T key, X extra, boolean changed) {}

View File

@ -1,3 +0,0 @@
package it.cavallium.dbengine.database;
public record KeyOperationResult<T>(T key, boolean changed) {}

View File

@ -4,7 +4,7 @@ import io.net5.buffer.api.Buffer;
import io.net5.buffer.api.BufferAllocator;
import io.net5.buffer.api.Send;
import it.cavallium.dbengine.client.BadBlock;
import it.cavallium.dbengine.database.serialization.BiSerializationFunction;
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import java.util.List;
import java.util.Optional;
@ -13,8 +13,6 @@ import org.jetbrains.annotations.Nullable;
import org.warp.commonutils.concurrency.atomicity.NotAtomic;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.util.function.Tuple2;
import reactor.util.function.Tuple3;
@SuppressWarnings("unused")
@NotAtomic
@ -62,19 +60,19 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
Mono<Send<Buffer>> remove(Mono<Send<Buffer>> key, LLDictionaryResultType resultType);
<K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Tuple2<K, Send<Buffer>>> keys,
Flux<Optional<Send<Buffer>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Send<Buffer>> keys,
boolean existsAlmostCertainly);
default <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Tuple2<K, Send<Buffer>>> keys) {
default Flux<Optional<Send<Buffer>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Send<Buffer>> keys) {
return getMulti(snapshot, keys, false);
}
Flux<Send<LLEntry>> putMulti(Flux<Send<LLEntry>> entries, boolean getOldValues);
<X> Flux<ExtraKeyOperationResult<Send<Buffer>, X>> updateMulti(Flux<Tuple2<Send<Buffer>, X>> entries,
BiSerializationFunction<Send<Buffer>, X, Send<Buffer>> updateFunction);
<K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Send<Buffer>> updateFunction);
Flux<Send<LLEntry>> getRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean existsAlmostCertainly);

View File

@ -20,6 +20,11 @@ public interface LLLuceneIndex extends LLSnapshottable {
Mono<Void> addDocument(LLTerm id, LLUpdateDocument doc);
/**
* WARNING! This operation is atomic!
* Please don't send infinite or huge documents fluxes, because they will
* be kept in ram all at once.
*/
Mono<Void> addDocuments(Flux<Entry<LLTerm, LLUpdateDocument>> documents);
Mono<Void> deleteDocument(LLTerm id);

View File

@ -5,14 +5,13 @@ import io.net5.buffer.api.Send;
import io.net5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.Delta;
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
import it.cavallium.dbengine.database.LLEntry;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.UpdateReturnMode;
import it.cavallium.dbengine.database.serialization.BiSerializationFunction;
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
import it.cavallium.dbengine.database.serialization.SerializationException;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import it.cavallium.dbengine.database.serialization.Serializer;
@ -244,16 +243,16 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
};
}
public <X> BiSerializationFunction<@Nullable Send<Buffer>, X, @Nullable Send<Buffer>> getSerializedUpdater(
BiSerializationFunction<@Nullable U, X, @Nullable U> updater) {
return (oldSerialized, extra) -> {
public KVSerializationFunction<@NotNull T, @Nullable Send<Buffer>, @Nullable Send<Buffer>> getSerializedUpdater(
KVSerializationFunction<@NotNull T, @Nullable U, @Nullable U> updater) {
return (key, oldSerialized) -> {
try (oldSerialized) {
U result;
if (oldSerialized == null) {
result = updater.apply(null, extra);
result = updater.apply(key, null);
} else {
try (var oldSerializedReceived = oldSerialized.receive()) {
result = updater.apply(valueSerializer.deserialize(oldSerializedReceived), extra);
result = updater.apply(key, valueSerializer.deserialize(oldSerializedReceived));
}
}
if (result == null) {
@ -307,34 +306,33 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
@Override
public Flux<Entry<T, Optional<U>>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
public Flux<Optional<U>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
var mappedKeys = keys
.<Tuple2<T, Send<Buffer>>>handle((keySuffix, sink) -> {
.<Send<Buffer>>handle((keySuffix, sink) -> {
try {
Tuple2<T, Send<Buffer>> tuple = Tuples.of(keySuffix, serializeKeySuffixToKey(keySuffix));
sink.next(tuple);
var buf = serializeKeySuffixToKey(keySuffix);
sink.next(buf);
} catch (Throwable ex) {
sink.error(ex);
}
});
return dictionary
.getMulti(resolveSnapshot(snapshot), mappedKeys, existsAlmostCertainly)
.<Entry<T, Optional<U>>>handle((entry, sink) -> {
.<Optional<U>>handle((valueBufOpt, sink) -> {
try {
Optional<U> valueOpt;
if (entry.getT3().isPresent()) {
try (var buf = entry.getT3().get().receive()) {
if (valueBufOpt.isPresent()) {
try (var buf = valueBufOpt.get().receive()) {
valueOpt = Optional.of(valueSerializer.deserialize(buf));
}
} else {
valueOpt = Optional.empty();
}
sink.next(Map.entry(entry.getT1(), valueOpt));
sink.next(valueOpt);
} catch (Throwable ex) {
sink.error(ex);
} finally {
entry.getT2().close();
entry.getT3().ifPresent(Send::close);
valueBufOpt.ifPresent(Send::close);
}
})
.transform(LLUtils::handleDiscard);
@ -384,13 +382,14 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
@Override
public <X> Flux<ExtraKeyOperationResult<T, X>> updateMulti(Flux<Tuple2<T, X>> entries,
BiSerializationFunction<@Nullable U, X, @Nullable U> updater) {
var serializedEntries = entries
.<Tuple2<Send<Buffer>, X>>handle((entry, sink) -> {
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(entry.getT1());
sink.next(Tuples.of(serializedKey, entry.getT2()));
Send<Buffer> serializedKey = serializeKeySuffixToKey(key);
sink.next(serializedKey);
} catch (Throwable ex) {
sink.error(ex);
}
@ -404,17 +403,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
});
var serializedUpdater = getSerializedUpdater(updater);
return dictionary.updateMulti(serializedEntries, serializedUpdater).handle((result, sink) -> {
try {
T keySuffix;
try (var keySuffixBuf = result.key().receive()) {
keySuffix = deserializeSuffix(keySuffixBuf);
}
sink.next(new ExtraKeyOperationResult<>(keySuffix, result.extra(), result.changed()));
} catch (Throwable ex) {
sink.error(ex);
}
});
return dictionary.updateMulti(sharedKeys, serializedKeys, serializedUpdater);
}
@Override

View File

@ -3,12 +3,10 @@ package it.cavallium.dbengine.database.collections;
import io.net5.buffer.api.Buffer;
import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.Delta;
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
import it.cavallium.dbengine.database.KeyOperationResult;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.UpdateReturnMode;
import it.cavallium.dbengine.database.serialization.BiSerializationFunction;
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
import it.cavallium.dbengine.database.serialization.SerializationException;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import java.util.HashMap;
@ -16,16 +14,11 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.function.BiFunction;
import java.util.function.Function;
import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.util.function.Tuple2;
import reactor.util.function.Tuple3;
import reactor.util.function.Tuples;
@SuppressWarnings("unused")
@ -61,13 +54,8 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends Dat
stage -> stage.update(updater, updateReturnMode, existsAlmostCertainly), true);
}
default <X> Flux<ExtraKeyOperationResult<T, X>> updateMulti(Flux<Tuple2<T, X>> entries,
BiSerializationFunction<@Nullable U, X, @Nullable U> updater) {
return entries
.flatMapSequential(entry -> this
.updateValue(entry.getT1(), prevValue -> updater.apply(prevValue, entry.getT2()))
.map(changed -> new ExtraKeyOperationResult<>(entry.getT1(), entry.getT2(), changed))
);
default Flux<Boolean> updateMulti(Flux<T> keys, KVSerializationFunction<T, @Nullable U, @Nullable U> updater) {
return keys.flatMapSequential(key -> this.updateValue(key, prevValue -> updater.apply(key, prevValue)));
}
default Mono<U> updateValue(T key, UpdateReturnMode updateReturnMode, SerializationFunction<@Nullable U, @Nullable U> updater) {
@ -119,13 +107,11 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends Dat
/**
* GetMulti must return the elements in sequence!
*/
default Flux<Entry<T, Optional<U>>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
default Flux<Optional<U>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
return keys
.flatMapSequential(key -> this
.getValue(snapshot, key, existsAlmostCertainly)
.map(value -> Map.entry(key, Optional.of(value)))
.switchIfEmpty(Mono.fromSupplier(() -> Map.entry(key, Optional.empty())))
)
.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()
@ -138,7 +124,7 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends Dat
/**
* GetMulti must return the elements in sequence!
*/
default Flux<Entry<T, Optional<U>>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys) {
default Flux<Optional<U>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys) {
return getMulti(snapshot, keys, false);
}
@ -293,20 +279,10 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends Dat
}
default ValueTransformer<T, U> getAsyncDbValueTransformer(@Nullable CompositeSnapshot snapshot) {
return new ValueTransformer<>() {
@Override
public <X> Flux<Tuple3<X, T, Optional<U>>> transform(Flux<Tuple2<X, T>> keys) {
return Flux.defer(() -> {
ConcurrentLinkedQueue<X> extraValues = new ConcurrentLinkedQueue<>();
return getMulti(snapshot, keys.map(key -> {
extraValues.add(key.getT1());
return key.getT2();
})).map(result -> {
var extraValue = extraValues.remove();
return Tuples.of(extraValue, result.getKey(), result.getValue());
});
});
}
return keys -> {
var sharedKeys = keys.publish().refCount(2);
var values = getMulti(snapshot, sharedKeys);
return Flux.zip(sharedKeys, values, Map::entry);
};
}
}

View File

@ -12,5 +12,5 @@ public interface ValueTransformer<KEY, VALUE> {
/**
* Can return Flux error IOException
*/
<X> Flux<Tuple3<X, KEY, Optional<VALUE>>> transform(Flux<Tuple2<X, KEY>> keys);
Flux<Entry<KEY, Optional<VALUE>>> transform(Flux<KEY> keys);
}

View File

@ -9,16 +9,13 @@ import static java.util.Objects.requireNonNullElse;
import io.net5.buffer.api.Buffer;
import io.net5.buffer.api.BufferAllocator;
import io.net5.buffer.api.MemoryManager;
import io.net5.buffer.api.Resource;
import io.net5.buffer.api.Send;
import io.net5.buffer.api.StandardAllocationTypes;
import io.net5.buffer.api.internal.ResourceSupport;
import io.net5.util.internal.PlatformDependent;
import it.cavallium.dbengine.client.BadBlock;
import it.cavallium.dbengine.client.DatabaseOptions;
import it.cavallium.dbengine.database.Column;
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
import it.cavallium.dbengine.database.LLDelta;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
@ -29,7 +26,7 @@ import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.SafeCloseable;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.UpdateReturnMode;
import it.cavallium.dbengine.database.serialization.BiSerializationFunction;
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -57,8 +54,6 @@ import org.rocksdb.RocksDBException;
import org.rocksdb.RocksIterator;
import org.rocksdb.Slice;
import org.rocksdb.Snapshot;
import org.rocksdb.TransactionDB;
import org.rocksdb.TransactionOptions;
import org.rocksdb.WriteBatch;
import org.rocksdb.WriteOptions;
import org.warp.commonutils.concurrency.atomicity.NotAtomic;
@ -491,8 +486,8 @@ public class LLLocalDictionary implements LLDictionary {
}
@Override
public <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Tuple2<K, Send<Buffer>>> keys,
public Flux<Optional<Send<Buffer>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Send<Buffer>> keys,
boolean existsAlmostCertainly) {
return keys
.buffer(MULTI_GET_WINDOW)
@ -509,58 +504,46 @@ public class LLLocalDictionary implements LLDictionary {
resource.close();
}
})
.flatMapSequential(keysWindow -> {
List<Send<Buffer>> keyBufsWindowSend = new ArrayList<>(keysWindow.size());
for (Tuple2<K, Send<Buffer>> objects : keysWindow) {
keyBufsWindowSend.add(objects.getT2());
.flatMapSequential(keysWindow -> runOnDb(() -> {
List<Buffer> keyBufsWindow = new ArrayList<>(keysWindow.size());
for (Send<Buffer> bufferSend : keysWindow) {
keyBufsWindow.add(bufferSend.receive());
}
return runOnDb(() -> {
List<Buffer> keyBufsWindow = new ArrayList<>(keyBufsWindowSend.size());
for (Send<Buffer> bufferSend : keyBufsWindowSend) {
keyBufsWindow.add(bufferSend.receive());
try {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called getMulti in a nonblocking thread");
}
try {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called getMulti in a nonblocking thread");
}
var readOptions = Objects.requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
List<byte[]> results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
var mappedResults = new ArrayList<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>>(results.size());
for (int i = 0; i < results.size(); i++) {
byte[] val = results.get(i);
Optional<Buffer> valueOpt;
if (val != null) {
results.set(i, null);
valueOpt = Optional.of(LLUtils.fromByteArray(alloc, val));
} else {
valueOpt = Optional.empty();
}
mappedResults.add(Tuples.of(keysWindow.get(i).getT1(),
keyBufsWindow.get(i).send(),
valueOpt.map(Resource::send)
));
}
return mappedResults;
} finally {
for (Buffer buffer : keyBufsWindow) {
buffer.close();
var readOptions = Objects.requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
List<byte[]> results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
var mappedResults = new ArrayList<Optional<Send<Buffer>>>(results.size());
for (int i = 0; i < results.size(); i++) {
byte[] val = results.get(i);
Optional<Buffer> valueOpt;
if (val != null) {
// free memory
results.set(i, null);
valueOpt = Optional.of(LLUtils.fromByteArray(alloc, val));
} else {
valueOpt = Optional.empty();
}
mappedResults.add(valueOpt.map(Resource::send));
}
})
.flatMapIterable(list -> list)
.onErrorMap(cause -> new IOException("Failed to read keys", cause))
.doAfterTerminate(() -> keyBufsWindowSend.forEach(Send::close));
}, 2) // Max concurrency is 2 to read data while preparing the next segment
return mappedResults;
} finally {
for (Buffer buffer : keyBufsWindow) {
buffer.close();
}
}
})
.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(Tuple3.class, discardedEntry -> {
if (discardedEntry.getT2() instanceof Buffer bb) {
.doOnDiscard(Optional.class, opt -> {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
if (discardedEntry.getT2() instanceof Optional opt) {
if (opt.isPresent() && opt.get() instanceof Buffer bb) {
bb.close();
}
}
});
}
@ -634,24 +617,24 @@ public class LLLocalDictionary implements LLDictionary {
}
@Override
public <X> Flux<ExtraKeyOperationResult<Send<Buffer>, X>> updateMulti(Flux<Tuple2<Send<Buffer>, X>> entries,
BiSerializationFunction<Send<Buffer>, X, Send<Buffer>> updateFunction) {
return entries
public <K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Send<Buffer>> updateFunction) {
return Flux.zip(keys, serializedKeys)
.buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
.flatMapSequential(ew -> this.<List<ExtraKeyOperationResult<Send<Buffer>, X>>>runOnDb(() -> {
List<Tuple2<Buffer, X>> entriesWindow = new ArrayList<>(ew.size());
for (Tuple2<Send<Buffer>, X> tuple : ew) {
entriesWindow.add(tuple.mapT1(Send::receive));
.flatMapSequential(ew -> this.<List<Boolean>>runOnDb(() -> {
List<Tuple2<K, Buffer>> entriesWindow = new ArrayList<>(ew.size());
for (Tuple2<K, Send<Buffer>> tuple : ew) {
entriesWindow.add(tuple.mapT2(Send::receive));
}
try {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called updateMulti in a nonblocking thread");
}
List<Buffer> keyBufsWindow = new ArrayList<>(entriesWindow.size());
for (Tuple2<Buffer, X> objects : entriesWindow) {
keyBufsWindow.add(objects.getT1());
for (Tuple2<K, Buffer> objects : entriesWindow) {
keyBufsWindow.add(objects.getT2());
}
ArrayList<Tuple3<Send<Buffer>, X, Optional<Send<Buffer>>>> mappedInputs;
ArrayList<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> mappedInputs;
{
var readOptions = Objects.requireNonNullElse(resolveSnapshot(null), EMPTY_READ_OPTIONS);
var inputs = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
@ -661,30 +644,38 @@ public class LLLocalDictionary implements LLDictionary {
if (val != null) {
inputs.set(i, null);
mappedInputs.add(Tuples.of(
entriesWindow.get(i).getT1(),
keyBufsWindow.get(i).send(),
entriesWindow.get(i).getT2(),
Optional.of(fromByteArray(alloc, val).send())
));
} else {
mappedInputs.add(Tuples.of(
entriesWindow.get(i).getT1(),
keyBufsWindow.get(i).send(),
entriesWindow.get(i).getT2(),
Optional.empty()
));
}
}
}
var updatedValuesToWrite = new ArrayList<Send<Buffer>>(mappedInputs.size());
var valueChangedResult = new ArrayList<ExtraKeyOperationResult<Send<Buffer>, X>>(mappedInputs.size());
var valueChangedResult = new ArrayList<Boolean>(mappedInputs.size());
try {
for (var mappedInput : mappedInputs) {
try (var updatedValue = updateFunction
.apply(mappedInput.getT1(), mappedInput.getT2()).receive()) {
try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) {
valueChangedResult.add(new ExtraKeyOperationResult<>(mappedInput.getT1(),
mappedInput.getT2(), !LLUtils.equals(t3, updatedValue)));
try (var updatedValueToReceive = updateFunction
.apply(mappedInput.getT1(), mappedInput.getT2())) {
if (updatedValueToReceive != null) {
try (var updatedValue = updatedValueToReceive.receive()) {
try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) {
valueChangedResult.add(!LLUtils.equals(t3, updatedValue));
}
updatedValuesToWrite.add(updatedValue.send());
}
} else {
try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) {
valueChangedResult.add(!LLUtils.equals(t3, null));
}
updatedValuesToWrite.add(null);
}
updatedValuesToWrite.add(updatedValue.send());
}
}
} finally {
@ -702,12 +693,12 @@ public class LLLocalDictionary implements LLDictionary {
BATCH_WRITE_OPTIONS
);
int i = 0;
for (Tuple2<Buffer, X> entry : entriesWindow) {
for (Tuple2<K, Buffer> entry : entriesWindow) {
var valueToWrite = updatedValuesToWrite.get(i);
if (valueToWrite == null) {
batch.delete(cfh, entry.getT1().send());
batch.delete(cfh, entry.getT2().send());
} else {
batch.put(cfh, entry.getT1().send(), valueToWrite);
batch.put(cfh, entry.getT2().send(), valueToWrite);
}
i++;
}
@ -715,15 +706,15 @@ public class LLLocalDictionary implements LLDictionary {
batch.close();
} else {
int i = 0;
for (Tuple2<Buffer, X> entry : entriesWindow) {
db.put(EMPTY_WRITE_OPTIONS, entry.getT1().send(), updatedValuesToWrite.get(i));
for (Tuple2<K, Buffer> entry : entriesWindow) {
db.put(EMPTY_WRITE_OPTIONS, entry.getT2().send(), updatedValuesToWrite.get(i));
i++;
}
}
return valueChangedResult;
} finally {
for (Tuple2<Buffer, X> tuple : entriesWindow) {
tuple.getT1().close();
for (Tuple2<K, Buffer> tuple : entriesWindow) {
tuple.getT2().close();
}
}
}).flatMapIterable(list -> list), /* Max concurrency is 2 to update data while preparing the next segment */ 2)
@ -734,28 +725,6 @@ public class LLLocalDictionary implements LLDictionary {
if (entry.getT2() instanceof Buffer bb) {
bb.close();
}
})
.doOnDiscard(ExtraKeyOperationResult.class, entry -> {
if (entry.key() instanceof Buffer bb) {
bb.close();
}
if (entry.extra() instanceof Buffer bb) {
bb.close();
}
})
.doOnDiscard(List.class, obj -> {
if (!obj.isEmpty() && obj.get(0) instanceof ExtraKeyOperationResult<?, ?>) {
//noinspection unchecked
var castedEntries = (List<ExtraKeyOperationResult<?, ?>>) obj;
for (ExtraKeyOperationResult<?, ?> entry : castedEntries) {
if (entry.key() instanceof Resource<?> bb) {
bb.close();
}
if (entry.extra() instanceof Resource<?> bb) {
bb.close();
}
}
}
});
}

View File

@ -4,7 +4,6 @@ import io.net5.buffer.api.Buffer;
import io.net5.buffer.api.BufferAllocator;
import io.net5.buffer.api.Send;
import it.cavallium.dbengine.client.BadBlock;
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
import it.cavallium.dbengine.database.LLDelta;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
@ -13,7 +12,7 @@ import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.serialization.BiSerializationFunction;
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
import it.cavallium.dbengine.database.serialization.SerializationException;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import it.unimi.dsi.fastutil.bytes.ByteList;
@ -254,20 +253,18 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
Flux<Tuple2<K, Send<Buffer>>> keys,
public Flux<Optional<Send<Buffer>>> getMulti(@Nullable LLSnapshot snapshot, Flux<Send<Buffer>> keys,
boolean existsAlmostCertainly) {
return keys
.map(key -> {
try (var t2 = key.getT2().receive()) {
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(t2.copy().send()));
if (v != null) {
return Tuples.of(key.getT1(), t2.send(), Optional.of(kk(v)));
} else {
return Tuples.of(key.getT1(), t2.send(), Optional.empty());
}
}
});
return keys.map(key -> {
try (var t2 = key.receive()) {
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(t2.copy().send()));
if (v != null) {
return Optional.of(kk(v));
} else {
return Optional.empty();
}
}
});
}
@Override
@ -287,8 +284,9 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public <X> Flux<ExtraKeyOperationResult<Send<Buffer>, X>> updateMulti(Flux<Tuple2<Send<Buffer>, X>> entries,
BiSerializationFunction<Send<Buffer>, X, Send<Buffer>> updateFunction) {
public <K> Flux<Boolean> updateMulti(Flux<K> keys,
Flux<Send<Buffer>> serializedKeys,
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Send<Buffer>> updateFunction) {
return Flux.error(new UnsupportedOperationException("Not implemented"));
}

View File

@ -1,7 +0,0 @@
package it.cavallium.dbengine.database.serialization;
@FunctionalInterface
public interface BiSerializationFunction<T1, T2, U> {
U apply(T1 argument1, T2 argument2) throws SerializationException;
}

View File

@ -0,0 +1,7 @@
package it.cavallium.dbengine.database.serialization;
@FunctionalInterface
public interface KVSerializationFunction<K, V, UV> {
UV apply(K key, V value) throws SerializationException;
}

View File

@ -390,14 +390,18 @@ public abstract class TestDictionaryMap {
Step<Entry<String, String>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5))
.flatMapMany(map -> Flux
.concat(
map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
)
.doFinally(s -> map.close())
)
.filter(k -> k.getValue().isPresent())
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = Flux
.concat(
map.putMulti(entriesFlux).then(Mono.empty()),
map.getMulti(null, keysFlux)
);
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(entry -> entry.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
));
@ -420,11 +424,14 @@ public abstract class TestDictionaryMap {
Step<Entry<String, String>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5))
.flatMapMany(map -> map
.setAllValues(Flux.fromIterable(entries.entrySet()))
.thenMany(map.getMulti(null, Flux.fromIterable(entries.keySet())))
.doFinally(s -> map.close())
)
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = map
.setAllValues(entriesFlux)
.thenMany(map.getMulti(null, keysFlux));
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
@ -476,13 +483,16 @@ public abstract class TestDictionaryMap {
Step<Entry<String, String>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryMap(dict, mapType, 5))
.flatMapMany(map -> Flux
.concat(
map.set(entries).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
)
.doFinally(s -> map.close())
)
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = Flux
.concat(
map.set(entries).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
);
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)

View File

@ -769,13 +769,16 @@ public abstract class TestDictionaryMapDeep {
Step<Entry<String, Map<String, String>>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryDeepMap(dict, 5, 6))
.flatMapMany(map -> Flux
.concat(
map.putMulti(Flux.fromIterable(entries.entrySet())).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
)
.doFinally(s -> map.close())
)
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = Flux
.concat(
map.putMulti(entriesFlux).then(Mono.empty()),
map.getMulti(null, keysFlux)
);
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)
@ -800,11 +803,14 @@ public abstract class TestDictionaryMapDeep {
Step<Entry<String, Map<String, String>>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryDeepMap(dict, 5, 6))
.flatMapMany(map -> map
.setAllValues(Flux.fromIterable(entries.entrySet()))
.thenMany(map.getMulti(null, Flux.fromIterable(entries.keySet())))
.doFinally(s -> map.close())
)
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = map
.setAllValues(Flux.fromIterable(entries.entrySet()))
.thenMany(map.getMulti(null, Flux.fromIterable(entries.keySet())));
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
));
@ -856,13 +862,16 @@ public abstract class TestDictionaryMapDeep {
Step<Entry<String, Map<String, String>>> stpVer = StepVerifier
.create(tempDb(getTempDbGenerator(), allocator, db -> tempDictionary(db, updateMode)
.map(dict -> tempDatabaseMapDictionaryDeepMap(dict, 5, 6))
.flatMapMany(map -> Flux
.concat(
map.set(entries).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
)
.doFinally(s -> map.close())
)
.flatMapMany(map -> {
var entriesFlux = Flux.fromIterable(entries.entrySet());
var keysFlux = entriesFlux.map(Entry::getKey);
var resultsFlux = Flux
.concat(
map.set(entries).then(Mono.empty()),
map.getMulti(null, Flux.fromIterable(entries.keySet()))
);
return Flux.zip(keysFlux, resultsFlux, Map::entry).doFinally(s -> map.close());
})
.filter(k -> k.getValue().isPresent())
.map(k -> Map.entry(k.getKey(), k.getValue().orElseThrow()))
.transform(LLUtils::handleDiscard)