Remove RocksObj, add BufSupplier, remove sends

This commit is contained in:
Andrea Cavalli 2022-05-20 10:20:00 +02:00
parent 1c36ab040b
commit 0e7df84c38
52 changed files with 1241 additions and 1249 deletions

View File

@ -0,0 +1,62 @@
package it.cavallium.dbengine.database;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import java.util.function.Supplier;
public abstract class BufSupplier implements SafeCloseable, Supplier<Buffer> {
public static BufSupplier of(Supplier<Buffer> supplier) {
return new SimpleBufSupplier(supplier);
}
public static BufSupplier of(Send<Buffer> supplier) {
return new CopyBufSupplier(supplier.receive());
}
public static BufSupplier ofOwned(Buffer supplier) {
return new CopyBufSupplier(supplier);
}
public static BufSupplier ofShared(Buffer supplier) {
return new SimpleBufSupplier(supplier::copy);
}
private static final class SimpleBufSupplier extends BufSupplier {
private final Supplier<Buffer> supplier;
public SimpleBufSupplier(Supplier<Buffer> supplier) {
this.supplier = supplier;
}
@Override
public Buffer get() {
return supplier.get();
}
@Override
public void close() {
}
}
private static final class CopyBufSupplier extends BufSupplier {
private final Buffer supplier;
public CopyBufSupplier(Buffer supplier) {
this.supplier = supplier;
}
@Override
public Buffer get() {
return supplier.copy();
}
@Override
public void close() {
supplier.close();
}
}
}

View File

@ -18,14 +18,14 @@ public class LLDelta extends ResourceSupport<LLDelta, LLDelta> {
@Override
public void drop(LLDelta obj) {
try {
if (obj.previous != null) {
if (obj.previous != null && obj.previous.isAccessible()) {
obj.previous.close();
}
} catch (Throwable ex) {
logger.error("Failed to close previous", ex);
}
try {
if (obj.current != null) {
if (obj.current != null && obj.current.isAccessible()) {
obj.current.close();
}
} catch (Throwable ex) {
@ -58,11 +58,11 @@ public class LLDelta extends ResourceSupport<LLDelta, LLDelta> {
@Nullable
private Runnable onClose;
private LLDelta(@Nullable Send<Buffer> previous, @Nullable Send<Buffer> current, @Nullable Runnable onClose) {
private LLDelta(@Nullable Buffer previous, @Nullable Buffer current, @Nullable Runnable onClose) {
super(DROP);
assert isAllAccessible();
this.previous = previous != null ? previous.receive().makeReadOnly() : null;
this.current = current != null ? current.receive().makeReadOnly() : null;
this.previous = previous != null ? previous.makeReadOnly() : null;
this.current = current != null ? current.makeReadOnly() : null;
this.onClose = onClose;
}
@ -74,7 +74,7 @@ public class LLDelta extends ResourceSupport<LLDelta, LLDelta> {
return true;
}
public static LLDelta of(Send<Buffer> previous, Send<Buffer> current) {
public static LLDelta of(Buffer previous, Buffer current) {
assert (previous == null && current == null) || (previous != current);
return new LLDelta(previous, current, null);
}
@ -89,6 +89,16 @@ public class LLDelta extends ResourceSupport<LLDelta, LLDelta> {
return current != null ? current.copy().send() : null;
}
public Buffer currentUnsafe() {
ensureOwned();
return current;
}
public Buffer previousUnsafe() {
ensureOwned();
return previous;
}
public boolean isModified() {
return !LLUtils.equals(previous, current);
}
@ -149,7 +159,11 @@ public class LLDelta extends ResourceSupport<LLDelta, LLDelta> {
Send<Buffer> maxSend = this.current != null ? this.current.send() : null;
Runnable onClose = this.onClose;
return drop -> {
var instance = new LLDelta(minSend, maxSend, onClose);
var instance = new LLDelta(
minSend != null ? minSend.receive() : null,
maxSend != null ? maxSend.receive() : null,
onClose
);
drop.attach(instance);
return instance;
};

View File

@ -21,13 +21,13 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
BufferAllocator getAllocator();
Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> key);
Mono<Buffer> get(@Nullable LLSnapshot snapshot, Mono<Buffer> key);
Mono<Send<Buffer>> put(Mono<Send<Buffer>> key, Mono<Send<Buffer>> value, LLDictionaryResultType resultType);
Mono<Buffer> put(Mono<Buffer> key, Mono<Buffer> value, LLDictionaryResultType resultType);
Mono<UpdateMode> getUpdateMode();
default Mono<Send<Buffer>> update(Mono<Send<Buffer>> key,
default Mono<Buffer> update(Mono<Buffer> key,
BinarySerializationFunction updater,
UpdateReturnMode updateReturnMode) {
return this
@ -35,51 +35,51 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
.transform(prev -> LLUtils.resolveLLDelta(prev, updateReturnMode));
}
Mono<Send<LLDelta>> updateAndGetDelta(Mono<Send<Buffer>> key, BinarySerializationFunction updater);
Mono<LLDelta> updateAndGetDelta(Mono<Buffer> key, BinarySerializationFunction updater);
Mono<Void> clear();
Mono<Send<Buffer>> remove(Mono<Send<Buffer>> key, LLDictionaryResultType resultType);
Mono<Buffer> remove(Mono<Buffer> key, LLDictionaryResultType resultType);
Flux<Optional<Buffer>> getMulti(@Nullable LLSnapshot snapshot, Flux<Send<Buffer>> keys);
Flux<Optional<Buffer>> getMulti(@Nullable LLSnapshot snapshot, Flux<Buffer> keys);
Mono<Void> putMulti(Flux<Send<LLEntry>> entries);
Mono<Void> putMulti(Flux<LLEntry> entries);
<K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Buffer> updateFunction);
<K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Buffer> serializedKeys,
KVSerializationFunction<K, @Nullable Buffer, @Nullable Buffer> updateFunction);
Flux<Send<LLEntry>> getRange(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> range,
Flux<LLEntry> getRange(@Nullable LLSnapshot snapshot,
Mono<LLRange> range,
boolean reverse,
boolean smallRange);
Flux<List<Send<LLEntry>>> getRangeGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> range,
Flux<List<LLEntry>> getRangeGrouped(@Nullable LLSnapshot snapshot,
Mono<LLRange> range,
int prefixLength,
boolean smallRange);
Flux<Send<Buffer>> getRangeKeys(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> range,
Flux<Buffer> getRangeKeys(@Nullable LLSnapshot snapshot,
Mono<LLRange> range,
boolean reverse,
boolean smallRange);
Flux<List<Send<Buffer>>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> range,
Flux<List<Buffer>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
Mono<LLRange> range,
int prefixLength,
boolean smallRange);
Flux<Send<Buffer>> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> range,
Flux<Buffer> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot,
Mono<LLRange> range,
int prefixLength,
boolean smallRange);
Flux<BadBlock> badBlocks(Mono<Send<LLRange>> range);
Flux<BadBlock> badBlocks(Mono<LLRange> range);
Mono<Void> setRange(Mono<Send<LLRange>> range, Flux<Send<LLEntry>> entries, boolean smallRange);
Mono<Void> setRange(Mono<LLRange> range, Flux<LLEntry> entries, boolean smallRange);
default Mono<Void> replaceRange(Mono<Send<LLRange>> range,
default Mono<Void> replaceRange(Mono<LLRange> range,
boolean canKeysChange,
Function<Send<LLEntry>, Mono<Send<LLEntry>>> entriesReplacer,
Function<LLEntry, Mono<LLEntry>> entriesReplacer,
boolean smallRange) {
return Mono.defer(() -> {
if (canKeysChange) {
@ -93,13 +93,13 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
});
}
Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean fillCache);
Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<LLRange> range, boolean fillCache);
Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean fast);
Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<LLRange> range, boolean fast);
Mono<Send<LLEntry>> getOne(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range);
Mono<LLEntry> getOne(@Nullable LLSnapshot snapshot, Mono<LLRange> range);
Mono<Send<Buffer>> getOneKey(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range);
Mono<Buffer> getOneKey(@Nullable LLSnapshot snapshot, Mono<LLRange> range);
Mono<Send<LLEntry>> removeOne(Mono<Send<LLRange>> range);
Mono<LLEntry> removeOne(Mono<LLRange> range);
}

View File

@ -13,70 +13,29 @@ import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
public class LLEntry extends ResourceSupport<LLEntry, LLEntry> {
public class LLEntry implements SafeCloseable {
private static final Logger logger = LogManager.getLogger(LLEntry.class);
private static final Drop<LLEntry> DROP = new Drop<>() {
@Override
public void drop(LLEntry obj) {
try {
if (obj.key != null) {
obj.key.close();
}
} catch (Throwable ex) {
logger.error("Failed to close key", ex);
}
try {
if (obj.value != null) {
obj.value.close();
}
} catch (Throwable ex) {
logger.error("Failed to close value", ex);
}
}
@Override
public Drop<LLEntry> fork() {
return this;
}
@Override
public void attach(LLEntry obj) {
}
};
@Nullable
private Buffer key;
@Nullable
private Buffer value;
private LLEntry(@NotNull Send<Buffer> key, @NotNull Send<Buffer> value) {
super(DROP);
this.key = key.receive().makeReadOnly();
this.value = value.receive().makeReadOnly();
this.key = key.receive();
this.value = value.receive();
assert isAllAccessible();
}
private LLEntry(@NotNull Buffer key, @NotNull Buffer value) {
super(DROP);
this.key = key.makeReadOnly();
this.value = value.makeReadOnly();
this.key = key;
this.value = value;
assert isAllAccessible();
}
private boolean isAllAccessible() {
assert key != null && key.isAccessible();
assert value != null && value.isAccessible();
assert this.isAccessible();
assert this.isOwned();
return true;
}
public static LLEntry of(@NotNull Send<Buffer> key, @NotNull Send<Buffer> value) {
return new LLEntry(key, value);
}
public static LLEntry of(@NotNull Buffer key, @NotNull Buffer value) {
return new LLEntry(key, value);
}
@ -102,19 +61,6 @@ public class LLEntry extends ResourceSupport<LLEntry, LLEntry> {
private void ensureOwned() {
assert isAllAccessible();
if (!isOwned()) {
if (!isAccessible()) {
throw this.createResourceClosedException();
} else {
throw new IllegalStateException("Resource not owned");
}
}
}
@Override
protected void makeInaccessible() {
this.key = null;
this.value = null;
}
@Override
@ -145,20 +91,22 @@ public class LLEntry extends ResourceSupport<LLEntry, LLEntry> {
}
@Override
protected RuntimeException createResourceClosedException() {
return new IllegalStateException("Closed");
}
@Override
protected Owned<LLEntry> prepareSend() {
Send<Buffer> keySend;
Send<Buffer> valueSend;
keySend = Objects.requireNonNull(this.key).send();
valueSend = Objects.requireNonNull(this.value).send();
return drop -> {
var instance = new LLEntry(keySend, valueSend);
drop.attach(instance);
return instance;
};
public void close() {
try {
if (key != null && key.isAccessible()) {
key.close();
}
} catch (Throwable ex) {
logger.error("Failed to close key", ex);
}
try {
if (value != null && value.isAccessible()) {
value.close();
}
} catch (Throwable ex) {
logger.error("Failed to close value", ex);
}
key = null;
value = null;
}
}

View File

@ -15,18 +15,18 @@ public interface LLSingleton extends LLKeyValueDatabaseStructure {
BufferAllocator getAllocator();
Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot);
Mono<Buffer> get(@Nullable LLSnapshot snapshot);
Mono<Void> set(Mono<Send<Buffer>> value);
Mono<Void> set(Mono<Buffer> value);
default Mono<Send<Buffer>> update(BinarySerializationFunction updater,
default Mono<Buffer> update(BinarySerializationFunction updater,
UpdateReturnMode updateReturnMode) {
return this
.updateAndGetDelta(updater)
.transform(prev -> LLUtils.resolveLLDelta(prev, updateReturnMode));
}
Mono<Send<LLDelta>> updateAndGetDelta(BinarySerializationFunction updater);
Mono<LLDelta> updateAndGetDelta(BinarySerializationFunction updater);
String getColumnName();

View File

@ -20,7 +20,6 @@ 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.disk.rocksdb.RocksIteratorObj;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.cavallium.dbengine.database.serialization.SerializationException;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import it.cavallium.dbengine.lucene.RandomSortField;
@ -63,6 +62,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.AbstractImmutableNativeReference;
import org.rocksdb.ReadOptions;
import org.rocksdb.RocksDB;
import reactor.core.publisher.Flux;
@ -117,12 +117,19 @@ public class LLUtils {
}
}
public static boolean responseToBoolean(Buffer response) {
try (response) {
assert response.readableBytes() == 1;
return response.getByte(response.readerOffset()) == 1;
}
}
public static byte[] booleanToResponse(boolean bool) {
return bool ? RESPONSE_TRUE : RESPONSE_FALSE;
}
public static Send<Buffer> booleanToResponseByteBuffer(BufferAllocator alloc, boolean bool) {
return alloc.allocate(1).writeByte(bool ? (byte) 1 : 0).send();
public static Buffer booleanToResponseByteBuffer(BufferAllocator alloc, boolean bool) {
return alloc.allocate(1).writeByte(bool ? (byte) 1 : 0);
}
@Nullable
@ -731,26 +738,26 @@ public class LLUtils {
* @param smallRange true if the range is small
* @return the passed instance of ReadOptions, or a new one if the passed readOptions is null
*/
public static RocksObj<ReadOptions> generateCustomReadOptions(@Nullable RocksObj<ReadOptions> readOptions,
public static ReadOptions generateCustomReadOptions(@Nullable ReadOptions readOptions,
boolean canFillCache,
boolean boundedRange,
boolean smallRange) {
if (readOptions == null) {
//noinspection resource
readOptions = new RocksObj<>(new ReadOptions());
readOptions = new ReadOptions();
}
if (boundedRange || smallRange) {
readOptions.v().setFillCache(canFillCache);
readOptions.setFillCache(canFillCache);
} else {
if (readOptions.v().readaheadSize() <= 0) {
readOptions.v().setReadaheadSize(4 * 1024 * 1024); // 4MiB
if (readOptions.readaheadSize() <= 0) {
readOptions.setReadaheadSize(4 * 1024 * 1024); // 4MiB
}
readOptions.v().setFillCache(false);
readOptions.v().setVerifyChecksums(false);
readOptions.setFillCache(false);
readOptions.setVerifyChecksums(false);
}
if (FORCE_DISABLE_CHECKSUM_VERIFICATION) {
readOptions.v().setVerifyChecksums(false);
readOptions.setVerifyChecksums(false);
}
return readOptions;
@ -865,22 +872,22 @@ public class LLUtils {
});
}
public static Mono<Send<Buffer>> resolveLLDelta(Mono<Send<LLDelta>> prev, UpdateReturnMode updateReturnMode) {
public static Mono<Buffer> resolveLLDelta(Mono<LLDelta> prev, UpdateReturnMode updateReturnMode) {
return prev.handle((deltaToReceive, sink) -> {
try (var delta = deltaToReceive.receive()) {
try (var delta = deltaToReceive) {
switch (updateReturnMode) {
case GET_NEW_VALUE -> {
var current = delta.current();
var current = delta.currentUnsafe();
if (current != null) {
sink.next(current);
sink.next(current.copy());
} else {
sink.complete();
}
}
case GET_OLD_VALUE -> {
var previous = delta.previous();
var previous = delta.previousUnsafe();
if (previous != null) {
sink.next(previous);
sink.next(previous.copy());
} else {
sink.complete();
}
@ -917,27 +924,25 @@ public class LLUtils {
});
}
public static <U> Mono<Delta<U>> mapLLDelta(Mono<Send<LLDelta>> mono,
SerializationFunction<@NotNull Send<Buffer>, @Nullable U> mapper) {
return mono.handle((deltaToReceive, sink) -> {
try (var delta = deltaToReceive.receive()) {
try (Send<Buffer> prev = delta.previous()) {
try (Send<Buffer> curr = delta.current()) {
U newPrev;
U newCurr;
if (prev != null) {
newPrev = mapper.apply(prev);
} else {
newPrev = null;
}
if (curr != null) {
newCurr = mapper.apply(curr);
} else {
newCurr = null;
}
sink.next(new Delta<>(newPrev, newCurr));
}
public static <U> Mono<Delta<U>> mapLLDelta(Mono<LLDelta> mono,
SerializationFunction<@NotNull Buffer, @Nullable U> mapper) {
return mono.handle((delta, sink) -> {
try (delta) {
Buffer prev = delta.previousUnsafe();
Buffer curr = delta.currentUnsafe();
U newPrev;
U newCurr;
if (prev != null) {
newPrev = mapper.apply(prev);
} else {
newPrev = null;
}
if (curr != null) {
newCurr = mapper.apply(curr);
} else {
newCurr = null;
}
sink.next(new Delta<>(newPrev, newCurr));
} catch (SerializationException ex) {
sink.error(ex);
}
@ -948,34 +953,6 @@ public class LLUtils {
return !Objects.equals(delta.previous(), delta.current());
}
public static Mono<Send<Buffer>> lazyRetain(Buffer buf) {
return Mono.fromSupplier(() -> {
if (buf != null && buf.isAccessible()) {
return buf.copy().send();
} else {
return null;
}
});
}
public static Mono<Send<LLRange>> lazyRetainRange(LLRange range) {
return Mono.fromSupplier(() -> {
if (range != null && range.isAccessible()) {
return range.copy().send();
} else {
return null;
}
});
}
public static Mono<Send<Buffer>> lazyRetain(Callable<Send<Buffer>> bufCallable) {
return Mono.fromCallable(bufCallable);
}
public static Mono<Send<LLRange>> lazyRetainRange(Callable<Send<LLRange>> rangeCallable) {
return Mono.fromCallable(rangeCallable);
}
public static boolean isDirect(Buffer key) {
var readableComponents = key.countReadableComponents();
if (readableComponents == 0) {
@ -1014,10 +991,10 @@ public class LLUtils {
iterable.forEach(LLUtils::onNextDropped);
} else if (next instanceof SafeCloseable safeCloseable) {
safeCloseable.close();
} else if (next instanceof RocksIteratorObj rocksIteratorObj) {
rocksIteratorObj.close();
} else if (next instanceof RocksObj<?> rocksObj) {
rocksObj.close();
} else if (next instanceof AbstractImmutableNativeReference rocksObj) {
if (rocksObj.isOwningHandle()) {
rocksObj.close();
}
} else if (next instanceof UpdateAtomicResultDelta delta) {
delta.delta().close();
} else if (next instanceof UpdateAtomicResultCurrent cur) {

View File

@ -0,0 +1,61 @@
package it.cavallium.dbengine.database;
import io.netty5.buffer.api.Send;
import java.util.function.Supplier;
public abstract class RangeSupplier implements SafeCloseable, Supplier<LLRange> {
public static RangeSupplier of(Supplier<LLRange> supplier) {
return new SimpleSupplier(supplier);
}
public static RangeSupplier of(Send<LLRange> supplier) {
return new CopySupplier(supplier.receive());
}
public static RangeSupplier ofOwned(LLRange supplier) {
return new CopySupplier(supplier);
}
public static RangeSupplier ofShared(LLRange supplier) {
return new SimpleSupplier(supplier::copy);
}
private static final class SimpleSupplier extends RangeSupplier {
private final Supplier<LLRange> supplier;
public SimpleSupplier(Supplier<LLRange> supplier) {
this.supplier = supplier;
}
@Override
public LLRange get() {
return supplier.get();
}
@Override
public void close() {
}
}
private static final class CopySupplier extends RangeSupplier {
private final LLRange supplier;
public CopySupplier(LLRange supplier) {
this.supplier = supplier;
}
@Override
public LLRange get() {
return supplier.copy();
}
@Override
public void close() {
supplier.close();
}
}
}

View File

@ -2,8 +2,10 @@ package it.cavallium.dbengine.database.collections;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.BufferAllocator;
import it.cavallium.dbengine.database.BufSupplier;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.serialization.Serializer;
import java.util.function.Supplier;
import org.jetbrains.annotations.NotNull;
public class DatabaseEmpty {
@ -34,7 +36,7 @@ public class DatabaseEmpty {
private DatabaseEmpty() {
}
public static DatabaseStageEntry<Nothing> create(LLDictionary dictionary, Buffer key, Runnable onClose) {
public static DatabaseStageEntry<Nothing> create(LLDictionary dictionary, BufSupplier key, Runnable onClose) {
return new DatabaseMapSingle<>(dictionary, key, nothingSerializer(dictionary.getAllocator()), onClose);
}

View File

@ -20,8 +20,8 @@ public class DatabaseInt implements LLKeyValueDatabaseStructure {
}
public Mono<Integer> get(@Nullable LLSnapshot snapshot) {
return singleton.get(snapshot).handle((dataSend, sink) -> {
try (var data = dataSend.receive()) {
return singleton.get(snapshot).handle((data, sink) -> {
try (data) {
sink.next(serializer.deserialize(data));
} catch (SerializationException e) {
sink.error(e);
@ -31,9 +31,13 @@ public class DatabaseInt implements LLKeyValueDatabaseStructure {
public Mono<Void> set(int value) {
return singleton.set(Mono.fromCallable(() -> {
try (var buf = singleton.getAllocator().allocate(Integer.BYTES)) {
var buf = singleton.getAllocator().allocate(Integer.BYTES);
try {
serializer.serialize(value, buf);
return buf.send();
return buf;
} catch (Throwable ex) {
buf.close();
throw ex;
}
}));
}

View File

@ -24,8 +24,8 @@ public class DatabaseLong implements LLKeyValueDatabaseStructure {
}
public Mono<Long> get(@Nullable LLSnapshot snapshot) {
return singleton.get(snapshot).handle((dataSend, sink) -> {
try (var data = dataSend.receive()) {
return singleton.get(snapshot).handle((data, sink) -> {
try (data) {
if (data.readableBytes() == 4) {
sink.next((long) (int) bugSerializer.deserialize(data));
} else {
@ -75,8 +75,8 @@ public class DatabaseLong implements LLKeyValueDatabaseStructure {
buf.writeLong(count);
return buf;
}
}, updateReturnMode).map(send -> {
try (var buf = send.receive()) {
}, updateReturnMode).map(buf -> {
try (buf) {
return buf.readLong();
}
}).single();
@ -84,10 +84,14 @@ public class DatabaseLong implements LLKeyValueDatabaseStructure {
public Mono<Void> set(long value) {
return singleton.set(Mono.fromCallable(() -> {
try (var buf = singleton.getAllocator().allocate(Long.BYTES)) {
var buf = singleton.getAllocator().allocate(Long.BYTES);
try {
serializer.serialize(value, buf);
return buf.send();
} catch (Throwable ex) {
buf.close();
throw ex;
}
return buf;
}));
}

View File

@ -3,7 +3,9 @@ package it.cavallium.dbengine.database.collections;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Resource;
import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.BufSupplier;
import it.cavallium.dbengine.database.Delta;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
@ -123,8 +125,8 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
});
}
private void deserializeValue(T keySuffix, Send<Buffer> valueToReceive, SynchronousSink<U> sink) {
try (var value = valueToReceive.receive()) {
private void deserializeValue(T keySuffix, Buffer value, SynchronousSink<U> sink) {
try (value) {
try {
sink.next(valueSerializer.deserialize(value));
} catch (IndexOutOfBoundsException ex) {
@ -210,20 +212,17 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
public Mono<Object2ObjectSortedMap<T, U>> get(@Nullable CompositeSnapshot snapshot, boolean existsAlmostCertainly) {
return dictionary
.getRange(resolveSnapshot(snapshot), rangeMono, false, true)
.<Entry<T, U>>handle((entrySend, sink) -> {
.<Entry<T, U>>handle((entry, sink) -> {
Entry<T, U> deserializedEntry;
try {
try (var entry = entrySend.receive()) {
try (entry) {
T key;
try (var serializedKey = entry.getKey().receive()) {
splitPrefix(serializedKey).close();
suffixKeyLengthConsistency(serializedKey.readableBytes());
key = deserializeSuffix(serializedKey);
}
U value;
try (var valueBuf = entry.getValue().receive()) {
value = valueSerializer.deserialize(valueBuf);
}
var serializedKey = entry.getKeyUnsafe();
var serializedValue = entry.getValueUnsafe();
splitPrefix(serializedKey).close();
suffixKeyLengthConsistency(serializedKey.readableBytes());
key = deserializeSuffix(serializedKey);
U value = valueSerializer.deserialize(serializedValue);
deserializedEntry = Map.entry(key, value);
}
sink.next(deserializedEntry);
@ -265,14 +264,14 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<DatabaseStageEntry<U>> at(@Nullable CompositeSnapshot snapshot, T keySuffix) {
return Mono.fromCallable(() ->
new DatabaseMapSingle<>(dictionary, serializeKeySuffixToKey(keySuffix), valueSerializer, null));
new DatabaseMapSingle<>(dictionary, BufSupplier.ofOwned(serializeKeySuffixToKey(keySuffix)), valueSerializer, null));
}
@Override
public Mono<Boolean> containsKey(@Nullable CompositeSnapshot snapshot, T keySuffix) {
return dictionary
.isRangeEmpty(resolveSnapshot(snapshot),
Mono.fromCallable(() -> LLRange.singleUnsafe(serializeKeySuffixToKey(keySuffix)).send()),
Mono.fromCallable(() -> LLRange.singleUnsafe(serializeKeySuffixToKey(keySuffix))),
true
)
.map(empty -> !empty);
@ -282,18 +281,18 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
public Mono<U> getValue(@Nullable CompositeSnapshot snapshot, T keySuffix, boolean existsAlmostCertainly) {
return dictionary
.get(resolveSnapshot(snapshot),
Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send())
Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix))
)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
}
@Override
public Mono<Void> putValue(T keySuffix, U value) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()).single();
var valueMono = Mono.fromCallable(() -> serializeValue(value).send()).single();
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix)).single();
var valueMono = Mono.fromCallable(() -> serializeValue(value)).single();
return dictionary
.put(keyMono, valueMono, LLDictionaryResultType.VOID)
.doOnNext(Send::close)
.doOnNext(Resource::close)
.then();
}
@ -306,7 +305,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
public Mono<U> updateValue(T keySuffix,
UpdateReturnMode updateReturnMode,
SerializationFunction<@Nullable U, @Nullable U> updater) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
return dictionary
.update(keyMono, getSerializedUpdater(updater), updateReturnMode)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
@ -315,11 +314,11 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<Delta<U>> updateValueAndGetDelta(T keySuffix,
SerializationFunction<@Nullable U, @Nullable U> updater) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
return dictionary
.updateAndGetDelta(keyMono, getSerializedUpdater(updater))
.transform(mono -> LLUtils.mapLLDelta(mono, serializedToReceive -> {
try (var serialized = serializedToReceive.receive()) {
.transform(mono -> LLUtils.mapLLDelta(mono, serialized -> {
try (serialized) {
return valueSerializer.deserialize(serialized);
}
}));
@ -328,21 +327,23 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
public BinarySerializationFunction getSerializedUpdater(
SerializationFunction<@Nullable U, @Nullable U> updater) {
return oldSerialized -> {
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);
try (oldSerialized) {
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);
}
}
};
}
public KVSerializationFunction<@NotNull T, @Nullable Send<Buffer>, @Nullable Buffer> getSerializedUpdater(
public KVSerializationFunction<@NotNull T, @Nullable Buffer, @Nullable Buffer> getSerializedUpdater(
KVSerializationFunction<@NotNull T, @Nullable U, @Nullable U> updater) {
return (key, oldSerialized) -> {
try (oldSerialized) {
@ -350,8 +351,8 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
if (oldSerialized == null) {
result = updater.apply(key, null);
} else {
try (var oldSerializedReceived = oldSerialized.receive()) {
result = updater.apply(key, valueSerializer.deserialize(oldSerializedReceived));
try (oldSerialized) {
result = updater.apply(key, valueSerializer.deserialize(oldSerialized));
}
}
if (result == null) {
@ -365,8 +366,8 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<U> putValueAndGetPrevious(T keySuffix, U value) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var valueMono = Mono.fromCallable(() -> serializeValue(value).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
var valueMono = Mono.fromCallable(() -> serializeValue(value));
return dictionary
.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
@ -374,30 +375,27 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<Boolean> putValueAndGetChanged(T keySuffix, U value) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var valueMono = Mono.fromCallable(() -> serializeValue(value).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
var valueMono = Mono.fromCallable(() -> serializeValue(value));
return dictionary
.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle((Send<Buffer> valueToReceive, SynchronousSink<U> sink) -> deserializeValue(keySuffix,
valueToReceive,
sink
))
.handle((Buffer valueBuf, SynchronousSink<U> sink) -> deserializeValue(keySuffix, valueBuf, sink))
.map(oldValue -> !Objects.equals(oldValue, value))
.defaultIfEmpty(value != null);
}
@Override
public Mono<Void> remove(T keySuffix) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
return dictionary
.remove(keyMono, LLDictionaryResultType.VOID)
.doOnNext(Send::close)
.doOnNext(Resource::close)
.then();
}
@Override
public Mono<U> removeAndGetPrevious(T keySuffix) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
return dictionary
.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
@ -405,7 +403,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<Boolean> removeAndGetStatus(T keySuffix) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix));
return dictionary
.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE)
.map(LLUtils::responseToBoolean);
@ -414,9 +412,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Flux<Optional<U>> getMulti(@Nullable CompositeSnapshot snapshot, Flux<T> keys, boolean existsAlmostCertainly) {
var mappedKeys = keys
.<Send<Buffer>>handle((keySuffix, sink) -> {
.<Buffer>handle((keySuffix, sink) -> {
try {
sink.next(serializeKeySuffixToKey(keySuffix).send());
sink.next(serializeKeySuffixToKey(keySuffix));
} catch (Throwable ex) {
sink.error(ex);
}
@ -451,9 +449,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
}
private void serializeEntrySink(Entry<T,U> entry, SynchronousSink<Send<LLEntry>> sink) {
private void serializeEntrySink(Entry<T,U> entry, SynchronousSink<LLEntry> sink) {
try {
sink.next(serializeEntry(entry.getKey(), entry.getValue()).send());
sink.next(serializeEntry(entry.getKey(), entry.getValue()));
} catch (Throwable e) {
sink.error(e);
}
@ -462,9 +460,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<Void> putMulti(Flux<Entry<T, U>> entries) {
var serializedEntries = entries
.<Send<LLEntry>>handle((entry, sink) -> {
.<LLEntry>handle((entry, sink) -> {
try {
sink.next(serializeEntry(entry.getKey(), entry.getValue()).send());
sink.next(serializeEntry(entry.getKey(), entry.getValue()));
} catch (Throwable e) {
sink.error(e);
}
@ -476,9 +474,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
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) -> {
var serializedKeys = sharedKeys.<Buffer>handle((key, sink) -> {
try {
Send<Buffer> serializedKey = serializeKeySuffixToKey(key).send();
Buffer serializedKey = serializeKeySuffixToKey(key);
sink.next(serializedKey);
} catch (Throwable ex) {
sink.error(ex);
@ -493,39 +491,43 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
return getAllStages(snapshot, rangeMono, false, smallRange);
}
private Send<LLRange> getPatchedRange(@NotNull Send<LLRange> rangeSend, @Nullable T keyMin, @Nullable T keyMax)
private LLRange getPatchedRange(@NotNull LLRange range, @Nullable T keyMin, @Nullable T keyMax)
throws SerializationException {
try (var range = rangeSend.receive()) {
try (Send<Buffer> keyMinBuf = serializeSuffixForRange(keyMin)) {
try (Send<Buffer> keyMaxBuf = serializeSuffixForRange(keyMax)) {
Send<Buffer> keyMinBufSend;
try (range) {
try (Buffer keyMinBuf = serializeSuffixForRange(keyMin)) {
try (Buffer keyMaxBuf = serializeSuffixForRange(keyMax)) {
Buffer keyMinBufSend;
if (keyMinBuf == null) {
keyMinBufSend = range.getMin();
keyMinBufSend = range.getMinUnsafe().copy();
} else {
keyMinBufSend = keyMinBuf;
}
Send<Buffer> keyMaxBufSend;
Buffer keyMaxBufSend;
if (keyMaxBuf == null) {
keyMaxBufSend = range.getMax();
keyMaxBufSend = range.getMaxUnsafe().copy();
} else {
keyMaxBufSend = keyMaxBuf;
}
return LLRange.of(keyMinBufSend, keyMaxBufSend).send();
return LLRange.ofUnsafe(keyMinBufSend, keyMaxBufSend);
}
}
}
}
private Send<Buffer> serializeSuffixForRange(@Nullable T key) throws SerializationException {
private Buffer serializeSuffixForRange(@Nullable T key) throws SerializationException {
if (key == null) {
return null;
}
try (var keyWithoutExtBuf = keyPrefix == null ? alloc.allocate(keySuffixLength + keyExtLength)
var keyWithoutExtBuf = keyPrefix == null ? alloc.allocate(keySuffixLength + keyExtLength)
// todo: use a read-only copy
: keyPrefix.copy()) {
: keyPrefix.copy();
try {
keyWithoutExtBuf.ensureWritable(keySuffixLength + keyExtLength);
serializeSuffix(key, keyWithoutExtBuf);
return keyWithoutExtBuf.send();
return keyWithoutExtBuf;
} catch (Throwable ex) {
keyWithoutExtBuf.close();
throw ex;
}
}
@ -542,10 +544,10 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
if (keyMin == null && keyMax == null) {
return getAllStages(snapshot, smallRange);
} else {
Mono<Send<LLRange>> boundedRangeMono = rangeMono
.handle((fullRangeSend, sink) -> {
Mono<LLRange> boundedRangeMono = rangeMono
.handle((fullRange, sink) -> {
try {
sink.next(getPatchedRange(fullRangeSend, keyMin, keyMax));
sink.next(getPatchedRange(fullRange, keyMin, keyMax));
} catch (SerializationException e) {
sink.error(e);
}
@ -555,11 +557,10 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
private Flux<Entry<T, DatabaseStageEntry<U>>> getAllStages(@Nullable CompositeSnapshot snapshot,
Mono<Send<LLRange>> sliceRangeMono, boolean reverse, boolean smallRange) {
Mono<LLRange> sliceRangeMono, boolean reverse, boolean smallRange) {
return dictionary
.getRangeKeys(resolveSnapshot(snapshot), sliceRangeMono, reverse, smallRange)
.handle((keyBufToReceive, sink) -> {
var keyBuf = keyBufToReceive.receive();
.handle((keyBuf, sink) -> {
try {
assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
// Remove prefix. Keep only the suffix and the ext
@ -569,7 +570,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
try (var keyBufCopy = keyBuf.copy()) {
keySuffix = deserializeSuffix(keyBufCopy);
}
var subStage = new DatabaseMapSingle<>(dictionary, toKey(keyBuf), valueSerializer, null);
var subStage = new DatabaseMapSingle<>(dictionary, BufSupplier.ofOwned(toKey(keyBuf)), valueSerializer, null);
sink.next(Map.entry(keySuffix, subStage));
} catch (Throwable ex) {
keyBuf.close();
@ -596,10 +597,10 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
if (keyMin == null && keyMax == null) {
return getAllValues(snapshot, smallRange);
} else {
Mono<Send<LLRange>> boundedRangeMono = rangeMono
.handle((fullRangeSend, sink) -> {
Mono<LLRange> boundedRangeMono = rangeMono
.handle((fullRange, sink) -> {
try {
sink.next(getPatchedRange(fullRangeSend, keyMin, keyMax));
sink.next(getPatchedRange(fullRange, keyMin, keyMax));
} catch (SerializationException e) {
sink.error(e);
}
@ -609,14 +610,14 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
}
private Flux<Entry<T, U>> getAllValues(@Nullable CompositeSnapshot snapshot,
Mono<Send<LLRange>> sliceRangeMono,
Mono<LLRange> sliceRangeMono,
boolean reverse, boolean smallRange) {
return dictionary
.getRange(resolveSnapshot(snapshot), sliceRangeMono, reverse, smallRange)
.<Entry<T, U>>handle((serializedEntryToReceive, sink) -> {
.<Entry<T, U>>handle((serializedEntry, sink) -> {
try {
Entry<T, U> entry;
try (var serializedEntry = serializedEntryToReceive.receive()) {
try (serializedEntry) {
var keyBuf = serializedEntry.getKeyUnsafe();
assert keyBuf != null;
assert keyBuf.readableBytes() == keyPrefixLength + keySuffixLength + keyExtLength;
@ -646,16 +647,18 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override
public Mono<Void> clear() {
if (range.isAll()) {
return dictionary.clear();
} else if (range.isSingle()) {
return dictionary
.remove(Mono.fromCallable(range::getSingle), LLDictionaryResultType.VOID)
.doOnNext(Send::close)
.then();
} else {
return dictionary.setRange(rangeMono, Flux.empty(), false);
}
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);
}
}

View File

@ -15,6 +15,7 @@ import it.cavallium.dbengine.database.LLDictionaryResultType;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.RangeSupplier;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.collections.DatabaseEmpty.Nothing;
import it.cavallium.dbengine.database.serialization.SerializationException;
@ -47,8 +48,8 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
@Override
public void drop(DatabaseMapDictionaryDeep<?, ?, ?> obj) {
try {
if (obj.range != null) {
obj.range.close();
if (obj.rangeSupplier != null) {
obj.rangeSupplier.close();
}
} catch (Throwable ex) {
LOG.error("Failed to close range", ex);
@ -95,9 +96,9 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
protected final int keyPrefixLength;
protected final int keySuffixLength;
protected final int keyExtLength;
protected final Mono<Send<LLRange>> rangeMono;
protected final Mono<LLRange> rangeMono;
protected LLRange range;
protected RangeSupplier rangeSupplier;
protected Buffer keyPrefix;
protected Buffer keySuffixAndExtZeroBuffer;
protected Runnable onClose;
@ -239,13 +240,13 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
assert prefixKey == null || prefixKey.isAccessible();
assert keyPrefixLength == 0 || !LLUtils.equals(firstKey, nextRangeKey);
if (keyPrefixLength == 0) {
this.range = LLRange.all();
this.rangeSupplier = RangeSupplier.ofOwned(LLRange.all());
firstKey.close();
nextRangeKey.close();
} else {
this.range = LLRange.ofUnsafe(firstKey, nextRangeKey);
this.rangeSupplier = RangeSupplier.ofOwned(LLRange.ofUnsafe(firstKey, nextRangeKey));
}
this.rangeMono = LLUtils.lazyRetainRange(this.range);
this.rangeMono = Mono.fromSupplier(rangeSupplier);
assert subStageKeysConsistency(keyPrefixLength + keySuffixLength + keyExtLength);
} catch (Throwable t) {
nextRangeKey.close();
@ -277,10 +278,10 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
int keyPrefixLength,
int keySuffixLength,
int keyExtLength,
Mono<Send<LLRange>> rangeMono,
Send<LLRange> range,
Send<Buffer> keyPrefix,
Send<Buffer> keySuffixAndExtZeroBuffer,
Mono<LLRange> rangeMono,
RangeSupplier rangeSupplier,
Buffer keyPrefix,
Buffer keySuffixAndExtZeroBuffer,
Runnable onClose) {
super((Drop<DatabaseMapDictionaryDeep<T,U,US>>) (Drop) DROP);
this.dictionary = dictionary;
@ -292,9 +293,9 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
this.keyExtLength = keyExtLength;
this.rangeMono = rangeMono;
this.range = range.receive();
this.keyPrefix = keyPrefix.receive();
this.keySuffixAndExtZeroBuffer = keySuffixAndExtZeroBuffer.receive();
this.rangeSupplier = rangeSupplier;
this.keyPrefix = keyPrefix;
this.keySuffixAndExtZeroBuffer = keySuffixAndExtZeroBuffer;
this.onClose = onClose;
}
@ -373,7 +374,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
return dictionary
.getRangeKeyPrefixes(resolveSnapshot(snapshot), rangeMono, keyPrefixLength + keySuffixLength, smallRange)
.flatMapSequential(groupKeyWithoutExtSend_ -> Mono.using(
groupKeyWithoutExtSend_::receive,
() -> groupKeyWithoutExtSend_,
groupKeyWithoutExtSend -> this.subStageGetter
.subStage(dictionary, snapshot, Mono.fromCallable(() -> groupKeyWithoutExtSend.copy().send()))
.handle((us, sink) -> {
@ -428,19 +429,18 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
@Override
public Mono<Void> clear() {
return Mono
.defer(() -> {
if (range.isAll()) {
return dictionary.clear();
} else if (range.isSingle()) {
return dictionary
.remove(Mono.fromCallable(range::getSingle), LLDictionaryResultType.VOID)
.doOnNext(Send::close)
.then();
} else {
return dictionary.setRange(rangeMono, Flux.empty(), false);
}
});
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);
}
protected T deserializeSuffix(@NotNull Buffer keySuffix) throws SerializationException {
@ -468,7 +468,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
protected Owned<DatabaseMapDictionaryDeep<T, U, US>> prepareSend() {
var keyPrefix = this.keyPrefix.send();
var keySuffixAndExtZeroBuffer = this.keySuffixAndExtZeroBuffer.send();
var range = this.range.send();
var rangeSupplier = this.rangeSupplier;
var onClose = this.onClose;
return drop -> {
var instance = new DatabaseMapDictionaryDeep<>(dictionary,
@ -479,9 +479,9 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
keySuffixLength,
keyExtLength,
rangeMono,
range,
keyPrefix,
keySuffixAndExtZeroBuffer,
rangeSupplier,
keyPrefix.receive(),
keySuffixAndExtZeroBuffer.receive(),
onClose
);
drop.attach(instance);
@ -493,7 +493,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
protected void makeInaccessible() {
this.keyPrefix = null;
this.keySuffixAndExtZeroBuffer = null;
this.range = null;
this.rangeSupplier = null;
this.onClose = null;
}
@ -539,23 +539,23 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
.dictionary
.getRange(deepMap.resolveSnapshot(snapshot), Mono.zip(savedProgressKey1Opt, deepMap.rangeMono).handle((tuple, sink) -> {
var firstKey = tuple.getT1();
try (var fullRange = tuple.getT2().receive()) {
try (var fullRange = tuple.getT2()) {
if (firstKey.isPresent()) {
try (var key1Buf = deepMap.alloc.allocate(keySuffix1Serializer.getSerializedBinaryLength())) {
keySuffix1Serializer.serialize(firstKey.get(), key1Buf);
sink.next(LLRange.of(key1Buf.send(), fullRange.getMax()).send());
sink.next(LLRange.of(key1Buf.send(), fullRange.getMax()));
} catch (SerializationException e) {
sink.error(e);
}
} else {
sink.next(fullRange.send());
sink.next(fullRange);
}
}
}), false, false)
.concatMapIterable(entrySend -> {
.concatMapIterable(entry -> {
K1 key1 = null;
Object key2 = null;
try (var entry = entrySend.receive()) {
try (entry) {
var keyBuf = entry.getKeyUnsafe();
var valueBuf = entry.getValueUnsafe();
try {

View File

@ -7,6 +7,7 @@ import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.client.BadBlock;
import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.BufSupplier;
import it.cavallium.dbengine.database.Delta;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
@ -18,6 +19,7 @@ import it.cavallium.dbengine.database.serialization.SerializationException;
import it.cavallium.dbengine.database.serialization.SerializationFunction;
import it.cavallium.dbengine.database.serialization.Serializer;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.Nullable;
@ -34,10 +36,8 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
private static final Drop<DatabaseMapSingle<?>> DROP = new Drop<>() {
@Override
public void drop(DatabaseMapSingle<?> obj) {
try {
obj.key.close();
} catch (Throwable ex) {
LOG.error("Failed to close key", ex);
if (obj.keySupplier != null) {
obj.keySupplier.close();
}
if (obj.onClose != null) {
obj.onClose.run();
@ -56,19 +56,18 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
};
private final LLDictionary dictionary;
private final Mono<Send<Buffer>> keyMono;
private final Mono<Buffer> keyMono;
private final Serializer<U> serializer;
private Buffer key;
private BufSupplier keySupplier;
private Runnable onClose;
@SuppressWarnings({"unchecked", "rawtypes"})
public DatabaseMapSingle(LLDictionary dictionary, Buffer key, Serializer<U> serializer,
public DatabaseMapSingle(LLDictionary dictionary, BufSupplier keySupplier, Serializer<U> serializer,
Runnable onClose) {
super((Drop<DatabaseMapSingle<U>>) (Drop) DROP);
this.dictionary = dictionary;
this.key = key;
this.keyMono = LLUtils.lazyRetain(this.key);
this.keySupplier = keySupplier;
this.keyMono = Mono.fromSupplier(() -> keySupplier.get());
this.serializer = serializer;
this.onClose = onClose;
}
@ -81,18 +80,20 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
}
}
private void deserializeValue(Send<Buffer> value, SynchronousSink<U> sink) {
private void deserializeValue(Buffer value, SynchronousSink<U> sink) {
try {
U deserializedValue;
try (var valueBuf = value.receive()) {
deserializedValue = serializer.deserialize(valueBuf);
try (value) {
deserializedValue = serializer.deserialize(value);
}
sink.next(deserializedValue);
} catch (IndexOutOfBoundsException ex) {
var exMessage = ex.getMessage();
if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) {
LOG.error("Unexpected zero-bytes value at "
+ dictionary.getDatabaseName() + ":" + dictionary.getColumnName() + ":" + LLUtils.toStringSafe(key));
try (var key = keySupplier.get()) {
LOG.error("Unexpected zero-bytes value at "
+ dictionary.getDatabaseName() + ":" + dictionary.getColumnName() + ":" + LLUtils.toStringSafe(key));
}
sink.complete();
} else {
sink.error(ex);
@ -102,12 +103,16 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
}
}
private Send<Buffer> serializeValue(U value) throws SerializationException {
private Buffer serializeValue(U value) throws SerializationException {
var valSizeHint = serializer.getSerializedSizeHint();
if (valSizeHint == -1) valSizeHint = 128;
try (var valBuf = dictionary.getAllocator().allocate(valSizeHint)) {
var valBuf = dictionary.getAllocator().allocate(valSizeHint);
try {
serializer.serialize(value, valBuf);
return valBuf.send();
return valBuf;
} catch (Throwable ex) {
valBuf.close();
throw ex;
}
}
@ -140,7 +145,7 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
if (result == null) {
return null;
} else {
return serializeValue(result).receive();
return serializeValue(result);
}
}, updateReturnMode)
.handle(this::deserializeValue);
@ -160,11 +165,11 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
if (result == null) {
return null;
} else {
return serializeValue(result).receive();
return serializeValue(result);
}
}).transform(mono -> LLUtils.mapLLDelta(mono, serialized -> {
try (var valueBuf = serialized.receive()) {
return serializer.deserialize(valueBuf);
try (serialized) {
return serializer.deserialize(serialized);
}
}));
}
@ -179,19 +184,19 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
@Override
public Mono<Long> leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) {
return dictionary
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send), false)
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::singleUnsafe), false)
.map(empty -> empty ? 0L : 1L);
}
@Override
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
return dictionary
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send), true);
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::singleUnsafe), true);
}
@Override
public Flux<BadBlock> badBlocks() {
return dictionary.badBlocks(keyMono.map(LLRange::single).map(ResourceSupport::send));
return dictionary.badBlocks(keyMono.map(LLRange::singleUnsafe));
}
@Override
@ -201,11 +206,10 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
@Override
protected Owned<DatabaseMapSingle<U>> prepareSend() {
var keySend = this.key.send();
var keySupplier = this.keySupplier;
var onClose = this.onClose;
return drop -> {
var key = keySend.receive();
var instance = new DatabaseMapSingle<>(dictionary, key, serializer, onClose);
var instance = new DatabaseMapSingle<>(dictionary, keySupplier, serializer, onClose);
drop.attach(instance);
return instance;
};
@ -213,7 +217,7 @@ public class DatabaseMapSingle<U> extends ResourceSupport<DatabaseStage<U>, Data
@Override
protected void makeInaccessible() {
this.key = null;
this.keySupplier = null;
this.onClose = null;
}
}

View File

@ -70,11 +70,11 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
}
}
private void deserializeValue(Send<Buffer> value, SynchronousSink<U> sink) {
private void deserializeValue(Buffer value, SynchronousSink<U> sink) {
try {
U deserializedValue;
try (var valueBuf = value.receive()) {
deserializedValue = serializer.deserialize(valueBuf);
try (value) {
deserializedValue = serializer.deserialize(value);
}
sink.next(deserializedValue);
} catch (IndexOutOfBoundsException ex) {
@ -91,12 +91,16 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
}
}
private Send<Buffer> serializeValue(U value) throws SerializationException {
private Buffer serializeValue(U value) throws SerializationException {
var valSizeHint = serializer.getSerializedSizeHint();
if (valSizeHint == -1) valSizeHint = 128;
try (var valBuf = singleton.getAllocator().allocate(valSizeHint)) {
var valBuf = singleton.getAllocator().allocate(valSizeHint);
try {
serializer.serialize(value, valBuf);
return valBuf.send();
return valBuf;
} catch (Throwable ex) {
valBuf.close();
throw ex;
}
}
@ -115,7 +119,7 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
public Mono<U> setAndGetPrevious(U value) {
return Flux
.concat(singleton.get(null), singleton.set(Mono.fromCallable(() -> serializeValue(value))).then(Mono.empty()))
.singleOrEmpty()
.last()
.handle(this::deserializeValue);
}
@ -135,7 +139,7 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
if (result == null) {
return null;
} else {
return serializeValue(result).receive();
return serializeValue(result);
}
}
}, updateReturnMode)
@ -157,12 +161,12 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
if (result == null) {
return null;
} else {
return serializeValue(result).receive();
return serializeValue(result);
}
}
}).transform(mono -> LLUtils.mapLLDelta(mono, serialized -> {
try (var valueBuf = serialized.receive()) {
return serializer.deserialize(valueBuf);
try (serialized) {
return serializer.deserialize(serialized);
}
}));
}
@ -176,7 +180,7 @@ public class DatabaseSingleton<U> extends ResourceSupport<DatabaseStage<U>, Data
public Mono<U> clearAndGetPrevious() {
return Flux
.concat(singleton.get(null), singleton.set(Mono.empty()).then(Mono.empty()))
.singleOrEmpty()
.last()
.handle(this::deserializeValue);
}

View File

@ -3,6 +3,7 @@ package it.cavallium.dbengine.database.collections;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.BufSupplier;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.serialization.Serializer;
import org.jetbrains.annotations.Nullable;
@ -20,7 +21,7 @@ public class SubStageGetterSingle<T> implements SubStageGetter<T, DatabaseStageE
public Mono<DatabaseStageEntry<T>> subStage(LLDictionary dictionary,
@Nullable CompositeSnapshot snapshot,
Mono<Send<Buffer>> keyPrefixMono) {
return keyPrefixMono.map(keyPrefix -> new DatabaseMapSingle<>(dictionary, keyPrefix.receive(), serializer, null));
return keyPrefixMono.map(keyPrefix -> new DatabaseMapSingle<>(dictionary, BufSupplier.of(keyPrefix), serializer, null));
}
}

View File

@ -20,7 +20,6 @@ import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.RepeatedElementList;
import it.cavallium.dbengine.database.disk.rocksdb.RocksIteratorObj;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.cavallium.dbengine.database.serialization.SerializationException;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -32,6 +31,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.AbstractImmutableNativeReference;
import org.rocksdb.AbstractSlice;
import org.rocksdb.ColumnFamilyHandle;
import org.rocksdb.CompactRangeOptions;
@ -65,7 +65,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
private final T db;
private final boolean nettyDirect;
private final BufferAllocator alloc;
private final RocksObj<ColumnFamilyHandle> cfh;
private final ColumnFamilyHandle cfh;
protected final MeterRegistry meterRegistry;
protected final StampedLock closeLock;
@ -100,7 +100,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
boolean nettyDirect,
BufferAllocator alloc,
String databaseName,
RocksObj<ColumnFamilyHandle> cfh,
ColumnFamilyHandle cfh,
MeterRegistry meterRegistry,
StampedLock closeLock) {
this.db = db;
@ -109,7 +109,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
this.cfh = cfh;
String columnName;
try {
columnName = new String(cfh.v().getName(), StandardCharsets.UTF_8);
columnName = new String(cfh.getName(), StandardCharsets.UTF_8);
} catch (RocksDBException e) {
throw new IllegalStateException(e);
}
@ -246,30 +246,30 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
/**
* This method should not modify or move the writerIndex/readerIndex of the key
*/
static RocksObj<? extends AbstractSlice<?>> setIterateBound(boolean allowNettyDirect,
RocksObj<ReadOptions> readOpts, IterateBound boundType, Buffer key) {
static AbstractSlice<?> setIterateBound(boolean allowNettyDirect,
ReadOptions readOpts, IterateBound boundType, Buffer key) {
requireNonNull(key);
RocksObj<? extends AbstractSlice<?>> slice;
AbstractSlice<?> slice;
if (allowNettyDirect && USE_DIRECT_BUFFER_BOUNDS && isReadOnlyDirect(key)) {
ByteBuffer keyInternalByteBuffer = ((ReadableComponent) key).readableBuffer();
assert keyInternalByteBuffer.position() == 0;
slice = new RocksObj<>(new DirectSlice(keyInternalByteBuffer, key.readableBytes()));
assert slice.v().size() == key.readableBytes();
slice = new DirectSlice(keyInternalByteBuffer, key.readableBytes());
assert slice.size() == key.readableBytes();
} else {
slice = new RocksObj<>(new Slice(requireNonNull(LLUtils.toArray(key))));
slice = new Slice(requireNonNull(LLUtils.toArray(key)));
}
if (boundType == IterateBound.LOWER) {
readOpts.v().setIterateLowerBound(slice.v());
readOpts.setIterateLowerBound(slice);
} else {
readOpts.v().setIterateUpperBound(slice.v());
readOpts.setIterateUpperBound(slice);
}
return slice;
}
static RocksObj<Slice> newEmptyReleasableSlice() {
static Slice newEmptyReleasableSlice() {
var arr = new byte[0];
return new RocksObj<>(new Slice(arr));
return new Slice(arr);
}
/**
@ -277,7 +277,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
*/
@NotNull
public RocksIteratorObj newRocksIterator(boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
LLRange range,
boolean reverse) throws RocksDBException {
assert !Schedulers.isInNonBlockingThread() : "Called getRocksIterator in a nonblocking thread";
@ -307,7 +307,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
return db;
}
protected RocksObj<ColumnFamilyHandle> getCfh() {
protected ColumnFamilyHandle getCfh() {
return cfh;
}
@ -315,7 +315,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
RocksDBUtils.ensureOpen(db, cfh);
}
protected void ensureOwned(RocksObj<?> rocksObject) {
protected void ensureOwned(AbstractImmutableNativeReference rocksObject) {
RocksDBUtils.ensureOwned(rocksObject);
}
@ -331,7 +331,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public @Nullable Buffer get(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException {
public @Nullable Buffer get(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -366,7 +366,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
assert resultBuffer.writerOffset() == 0;
var resultWritable = ((WritableComponent) resultBuffer).writableBuffer();
var keyMayExist = db.keyMayExist(cfh.v(), readOptions.v(), keyNioBuffer.rewind(),
var keyMayExist = db.keyMayExist(cfh, readOptions, keyNioBuffer.rewind(),
resultWritable.clear());
KeyMayExistEnum keyMayExistState = keyMayExist.exists;
int keyMayExistValueLength = keyMayExist.valueLength;
@ -397,7 +397,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
resultWritable.clear();
readAttemptsCount++;
// real data size
size = db.get(cfh.v(), readOptions.v(), keyNioBuffer.rewind(), resultWritable.clear());
size = db.get(cfh, readOptions, keyNioBuffer.rewind(), resultWritable.clear());
if (size == RocksDB.NOT_FOUND) {
resultBuffer.close();
readValueNotFoundWithMayExistBloomBufferSize.record(0);
@ -423,7 +423,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
assert resultBuffer.writerOffset() == 0;
readAttemptsCount++;
size = db.get(cfh.v(), readOptions.v(), keyNioBuffer.rewind(), resultWritable.clear());
size = db.get(cfh, readOptions, keyNioBuffer.rewind(), resultWritable.clear());
if (size == RocksDB.NOT_FOUND) {
readValueNotFoundWithMayExistBloomBufferSize.record(0);
resultBuffer.close();
@ -452,7 +452,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
byte[] keyArray = LLUtils.toArray(key);
requireNonNull(keyArray);
Holder<byte[]> data = new Holder<>();
if (db.keyMayExist(cfh.v(), readOptions.v(), keyArray, data)) {
if (db.keyMayExist(cfh, readOptions, keyArray, data)) {
// todo: "data.getValue().length > 0" is checked because keyMayExist is broken, and sometimes it
// returns an empty array, as if it exists
if (data.getValue() != null && data.getValue().length > 0) {
@ -460,7 +460,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
return LLUtils.fromByteArray(alloc, data.getValue());
} else {
readAttemptsCount++;
byte[] result = db.get(cfh.v(), readOptions.v(), keyArray);
byte[] result = db.get(cfh, readOptions, keyArray);
if (result == null) {
if (data.getValue() != null) {
readValueNotFoundWithBloomBufferSize.record(0);
@ -487,7 +487,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public void put(@NotNull RocksObj<WriteOptions> writeOptions, Buffer key, Buffer value) throws RocksDBException {
public void put(@NotNull WriteOptions writeOptions, Buffer key, Buffer value) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -534,7 +534,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
try {
db.put(cfh.v(), writeOptions.v(), keyNioBuffer, valueNioBuffer);
db.put(cfh, writeOptions, keyNioBuffer, valueNioBuffer);
} finally {
if (mustCloseValue) {
value.close();
@ -546,7 +546,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
}
} else {
db.put(cfh.v(), writeOptions.v(), LLUtils.toArray(key), LLUtils.toArray(value));
db.put(cfh, writeOptions, LLUtils.toArray(key), LLUtils.toArray(value));
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -554,7 +554,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public boolean exists(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException {
public boolean exists(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -578,8 +578,8 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
assert keyNioBuffer.limit() == key.readableBytes();
}
try {
if (db.keyMayExist(cfh.v(), keyNioBuffer)) {
int size = db.get(cfh.v(), readOptions.v(), keyNioBuffer.position(0), LLUtils.EMPTY_BYTE_BUFFER);
if (db.keyMayExist(cfh, keyNioBuffer)) {
int size = db.get(cfh, readOptions, keyNioBuffer.position(0), LLUtils.EMPTY_BYTE_BUFFER);
boolean found = size != RocksDB.NOT_FOUND;
if (found) {
readValueFoundWithBloomSimpleBufferSize.record(size);
@ -602,12 +602,12 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
byte[] keyBytes = LLUtils.toArray(key);
Holder<byte[]> data = new Holder<>();
boolean mayExistHit = false;
if (db.keyMayExist(cfh.v(), readOptions.v(), keyBytes, data)) {
if (db.keyMayExist(cfh, readOptions, keyBytes, data)) {
mayExistHit = true;
if (data.getValue() != null) {
size = data.getValue().length;
} else {
size = db.get(cfh.v(), readOptions.v(), keyBytes, NO_DATA);
size = db.get(cfh, readOptions, keyBytes, NO_DATA);
}
}
boolean found = size != RocksDB.NOT_FOUND;
@ -628,7 +628,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public boolean mayExists(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException {
public boolean mayExists(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -652,7 +652,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
assert keyNioBuffer.limit() == key.readableBytes();
}
try {
return db.keyMayExist(cfh.v(), readOptions.v(), keyNioBuffer);
return db.keyMayExist(cfh, readOptions, keyNioBuffer);
} finally {
if (mustCloseKey) {
key.close();
@ -660,7 +660,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
} else {
byte[] keyBytes = LLUtils.toArray(key);
return db.keyMayExist(cfh.v(), readOptions.v(), keyBytes, null);
return db.keyMayExist(cfh, readOptions, keyBytes, null);
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -668,7 +668,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public void delete(RocksObj<WriteOptions> writeOptions, Buffer key) throws RocksDBException {
public void delete(WriteOptions writeOptions, Buffer key) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -693,14 +693,14 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
assert keyNioBuffer.limit() == key.readableBytes();
}
try {
db.delete(cfh.v(), writeOptions.v(), keyNioBuffer);
db.delete(cfh, writeOptions, keyNioBuffer);
} finally {
if (mustCloseKey) {
key.close();
}
}
} else {
db.delete(cfh.v(), writeOptions.v(), LLUtils.toArray(key));
db.delete(cfh, writeOptions, LLUtils.toArray(key));
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -708,20 +708,20 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
}
@Override
public void delete(RocksObj<WriteOptions> writeOptions, byte[] key) throws RocksDBException {
public void delete(WriteOptions writeOptions, byte[] key) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
ensureOwned(writeOptions);
keyBufferSize.record(key.length);
db.delete(cfh.v(), writeOptions.v(), key);
db.delete(cfh, writeOptions, key);
} finally {
closeLock.unlockRead(closeReadLock);
}
}
@Override
public List<byte[]> multiGetAsList(RocksObj<ReadOptions> readOptions, List<byte[]> keys) throws RocksDBException {
public List<byte[]> multiGetAsList(ReadOptions readOptions, List<byte[]> keys) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
@ -729,8 +729,8 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
for (byte[] key : keys) {
keyBufferSize.record(key.length);
}
var columnFamilyHandles = new RepeatedElementList<>(cfh.v(), keys.size());
return db.multiGetAsList(readOptions.v(), columnFamilyHandles, keys);
var columnFamilyHandles = new RepeatedElementList<>(cfh, keys.size());
return db.multiGetAsList(readOptions, columnFamilyHandles, keys);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -741,31 +741,31 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
db.suggestCompactRange(cfh.v());
db.suggestCompactRange(cfh);
} finally {
closeLock.unlockRead(closeReadLock);
}
}
@Override
public void compactRange(byte[] begin, byte[] end, RocksObj<CompactRangeOptions> options) throws RocksDBException {
public void compactRange(byte[] begin, byte[] end, CompactRangeOptions options) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
ensureOwned(options);
db.compactRange(cfh.v(), begin, end, options.v());
db.compactRange(cfh, begin, end, options);
} finally {
closeLock.unlockRead(closeReadLock);
}
}
@Override
public void flush(RocksObj<FlushOptions> options) throws RocksDBException {
public void flush(FlushOptions options) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
ensureOwned(options);
db.flush(options.v(), cfh.v());
db.flush(options, cfh);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -787,20 +787,20 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
return db.getLongProperty(cfh.v(), property);
return db.getLongProperty(cfh, property);
} finally {
closeLock.unlockRead(closeReadLock);
}
}
@Override
public void write(RocksObj<WriteOptions> writeOptions, WriteBatch writeBatch) throws RocksDBException {
public void write(WriteOptions writeOptions, WriteBatch writeBatch) throws RocksDBException {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
ensureOwned(writeOptions);
ensureOwned(writeBatch);
db.write(writeOptions.v(), writeBatch);
db.write(writeOptions, writeBatch);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -809,14 +809,14 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
/**
* @return true if committed successfully
*/
protected abstract boolean commitOptimistically(RocksObj<Transaction> tx) throws RocksDBException;
protected abstract boolean commitOptimistically(Transaction tx) throws RocksDBException;
protected abstract RocksObj<Transaction> beginTransaction(@NotNull RocksObj<WriteOptions> writeOptions,
RocksObj<TransactionOptions> txOpts);
protected abstract Transaction beginTransaction(@NotNull WriteOptions writeOptions,
TransactionOptions txOpts);
@Override
public final @NotNull UpdateAtomicResult updateAtomic(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
public final @NotNull UpdateAtomicResult updateAtomic(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws IOException {
@ -858,15 +858,15 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
timer.record(duration, TimeUnit.NANOSECONDS);
}
protected abstract @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
protected abstract @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws IOException;
@Override
@NotNull
public RocksIteratorObj newIterator(@NotNull RocksObj<ReadOptions> readOptions,
public RocksIteratorObj newIterator(@NotNull ReadOptions readOptions,
@Nullable Buffer min,
@Nullable Buffer max) {
var closeReadLock = closeLock.readLock();
@ -875,8 +875,8 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
ensureOwned(readOptions);
ensureOwned(min);
ensureOwned(max);
RocksObj<? extends AbstractSlice<?>> sliceMin;
RocksObj<? extends AbstractSlice<?>> sliceMax;
AbstractSlice<?> sliceMin;
AbstractSlice<?> sliceMax;
if (min != null) {
sliceMin = setIterateBound(nettyDirect, readOptions, IterateBound.LOWER, min);
} else {
@ -889,7 +889,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
sliceMax = null;
}
try {
var it = db.newIterator(cfh.v(), readOptions.v());
var it = db.newIterator(cfh, readOptions);
try {
return new RocksIteratorObj(it,
sliceMin,
@ -932,7 +932,8 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
try {
newData = updater.apply(prevDataToSendToUpdater);
} finally {
if (prevDataToSendToUpdater != newData && prevDataToSendToUpdater != null) {
if (prevDataToSendToUpdater != newData && prevDataToSendToUpdater != null
&& prevDataToSendToUpdater.isAccessible()) {
prevDataToSendToUpdater.close();
}
}
@ -943,7 +944,7 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
return RocksDBUtils.getLevels(db, cfh.v());
return RocksDBUtils.getLevels(db, cfh);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -954,14 +955,14 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
RocksDBUtils.forceCompaction(db, db.getName(), cfh.v(), volumeId, logger);
RocksDBUtils.forceCompaction(db, db.getName(), cfh, volumeId, logger);
} finally {
closeLock.unlockRead(closeReadLock);
}
}
@Override
public RocksObj<ColumnFamilyHandle> getColumnFamilyHandle() {
public ColumnFamilyHandle getColumnFamilyHandle() {
return cfh;
}

View File

@ -10,7 +10,6 @@ import io.netty5.buffer.api.Send;
import io.netty5.util.internal.PlatformDependent;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.RocksDBColumn;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
@ -28,7 +27,7 @@ public class CappedWriteBatch extends WriteBatch {
private final RocksDBColumn db;
private final BufferAllocator alloc;
private final int cap;
private final RocksObj<WriteOptions> writeOptions;
private final WriteOptions writeOptions;
private final List<Buffer> buffersToRelease;
private final List<ByteBuffer> byteBuffersToRelease;
@ -42,7 +41,7 @@ public class CappedWriteBatch extends WriteBatch {
int cap,
int reservedWriteBatchSize,
long maxWriteBatchSize,
RocksObj<WriteOptions> writeOptions) {
WriteOptions writeOptions) {
super(reservedWriteBatchSize);
this.db = db;
this.alloc = alloc;

View File

@ -5,7 +5,6 @@ import static it.cavallium.dbengine.database.disk.LLTempHugePqEnv.getColumnOptio
import com.google.common.primitives.Ints;
import io.micrometer.core.instrument.composite.CompositeMeterRegistry;
import io.netty5.buffer.api.BufferAllocator;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
import java.io.Closeable;
@ -24,10 +23,10 @@ import org.rocksdb.RocksDBException;
public class HugePqEnv implements Closeable {
private final RocksDB db;
private final ArrayList<RocksObj<ColumnFamilyHandle>> defaultCfh;
private final Int2ObjectMap<RocksObj<ColumnFamilyHandle>> cfhs = new Int2ObjectOpenHashMap<>();
private final ArrayList<ColumnFamilyHandle> defaultCfh;
private final Int2ObjectMap<ColumnFamilyHandle> cfhs = new Int2ObjectOpenHashMap<>();
public HugePqEnv(RocksDB db, ArrayList<RocksObj<ColumnFamilyHandle>> defaultCfh) {
public HugePqEnv(RocksDB db, ArrayList<ColumnFamilyHandle> defaultCfh) {
this.db = db;
this.defaultCfh = defaultCfh;
}
@ -35,8 +34,7 @@ public class HugePqEnv implements Closeable {
@Override
public void close() throws IOException {
for (var cfh : defaultCfh) {
db.destroyColumnFamilyHandle(cfh.v());
cfh.close();
db.destroyColumnFamilyHandle(cfh);
}
try {
db.closeE();
@ -46,7 +44,7 @@ public class HugePqEnv implements Closeable {
}
public int createColumnFamily(int name, AbstractComparator comparator) throws RocksDBException {
var cfh = new RocksObj<>(db.createColumnFamily(new ColumnFamilyDescriptor(Ints.toByteArray(name), getColumnOptions(comparator))));
var cfh = db.createColumnFamily(new ColumnFamilyDescriptor(Ints.toByteArray(name), getColumnOptions(comparator)));
synchronized (cfhs) {
var prev = cfhs.put(name, cfh);
if (prev != null) {
@ -57,19 +55,18 @@ public class HugePqEnv implements Closeable {
}
public void deleteColumnFamily(int db) throws RocksDBException {
RocksObj<ColumnFamilyHandle> cfh;
ColumnFamilyHandle cfh;
synchronized (cfhs) {
cfh = cfhs.remove(db);
}
if (cfh != null) {
this.db.dropColumnFamily(cfh.v());
this.db.destroyColumnFamilyHandle(cfh.v());
cfh.close();
this.db.dropColumnFamily(cfh);
this.db.destroyColumnFamilyHandle(cfh);
}
}
public StandardRocksDBColumn openDb(int hugePqId) {
RocksObj<ColumnFamilyHandle> cfh;
ColumnFamilyHandle cfh;
synchronized (cfhs) {
cfh = Objects.requireNonNull(cfhs.get(hugePqId), () -> "column " + hugePqId + " does not exist");
}

View File

@ -4,22 +4,26 @@ import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLEntry;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.rocksdb.ReadOptions;
public class LLLocalEntryReactiveRocksIterator extends LLLocalReactiveRocksIterator<Send<LLEntry>> {
public class LLLocalEntryReactiveRocksIterator extends LLLocalReactiveRocksIterator<LLEntry> {
public LLLocalEntryReactiveRocksIterator(RocksDBColumn db,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean reverse,
boolean smallRange) {
super(db, range, allowNettyDirect, readOptions, true, reverse, smallRange);
}
@Override
public Send<LLEntry> getEntry(Send<Buffer> key, Send<Buffer> value) {
return LLEntry.of(key, value).send();
public LLEntry getEntry(Buffer key, Buffer value) {
assert key != null;
assert value != null;
return LLEntry.of(
key.touch("iteration entry key"),
value.touch("iteration entry value")
);
}
}

View File

@ -4,23 +4,27 @@ import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLEntry;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.rocksdb.ReadOptions;
public class LLLocalGroupedEntryReactiveRocksIterator extends
LLLocalGroupedReactiveRocksIterator<Send<LLEntry>> {
LLLocalGroupedReactiveRocksIterator<LLEntry> {
public LLLocalGroupedEntryReactiveRocksIterator(RocksDBColumn db,
int prefixLength,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean smallRange) {
super(db, prefixLength, range, allowNettyDirect, readOptions, false, true, smallRange);
}
@Override
public Send<LLEntry> getEntry(Send<Buffer> key, Send<Buffer> value) {
return LLEntry.of(key, value).send();
public LLEntry getEntry(Buffer key, Buffer value) {
assert key != null;
assert value != null;
return LLEntry.of(
key.touch("iteration entry key"),
value.touch("iteration entry value")
);
}
}

View File

@ -3,22 +3,21 @@ package it.cavallium.dbengine.database.disk;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.rocksdb.ReadOptions;
public class LLLocalGroupedKeyReactiveRocksIterator extends LLLocalGroupedReactiveRocksIterator<Send<Buffer>> {
public class LLLocalGroupedKeyReactiveRocksIterator extends LLLocalGroupedReactiveRocksIterator<Buffer> {
public LLLocalGroupedKeyReactiveRocksIterator(RocksDBColumn db,
int prefixLength,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean smallRange) {
super(db, prefixLength, range, allowNettyDirect, readOptions, true, false, smallRange);
}
@Override
public Send<Buffer> getEntry(Send<Buffer> key, Send<Buffer> value) {
public Buffer getEntry(Buffer key, Buffer value) {
if (value != null) {
value.close();
}

View File

@ -11,7 +11,6 @@ import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
import java.util.List;
import org.apache.logging.log4j.LogManager;
@ -59,7 +58,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
private final int prefixLength;
private LLRange range;
private final boolean allowNettyDirect;
private RocksObj<ReadOptions> readOptions;
private ReadOptions readOptions;
private final boolean canFillCache;
private final boolean readValues;
private final boolean smallRange;
@ -67,23 +66,21 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
@SuppressWarnings({"unchecked", "rawtypes"})
public LLLocalGroupedReactiveRocksIterator(RocksDBColumn db,
int prefixLength,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean canFillCache,
boolean readValues,
boolean smallRange) {
super((Drop<LLLocalGroupedReactiveRocksIterator<T>>) (Drop) DROP);
try (range) {
this.db = db;
this.prefixLength = prefixLength;
this.range = range.receive();
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions != null ? readOptions : new RocksObj<>(new ReadOptions());
this.canFillCache = canFillCache;
this.readValues = readValues;
this.smallRange = smallRange;
}
this.db = db;
this.prefixLength = prefixLength;
this.range = range;
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions != null ? readOptions : new ReadOptions();
this.canFillCache = canFillCache;
this.readValues = readValues;
this.smallRange = smallRange;
}
public final Flux<List<T>> flux() {
@ -129,7 +126,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
try {
rocksIterator.next();
T entry = getEntry(key.send(), value == null ? null : value.send());
T entry = getEntry(key, value);
values.add(entry);
} finally {
if (value != null) {
@ -161,7 +158,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
}, RocksIterWithReadOpts::close);
}
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
public abstract T getEntry(@Nullable Buffer key, @Nullable Buffer value);
@Override
protected final RuntimeException createResourceClosedException() {
@ -174,7 +171,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
var readOptions = this.readOptions;
return drop -> new LLLocalGroupedReactiveRocksIterator<>(db,
prefixLength,
range,
range.receive(),
allowNettyDirect,
readOptions,
canFillCache,
@ -182,7 +179,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
smallRange
) {
@Override
public T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value) {
public T getEntry(@Nullable Buffer key, @Nullable Buffer value) {
return LLLocalGroupedReactiveRocksIterator.this.getEntry(key, value);
}
};

View File

@ -11,7 +11,6 @@ import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.rocksdb.ReadOptions;
@ -57,31 +56,29 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
private final int prefixLength;
private LLRange rangeShared;
private final boolean allowNettyDirect;
private RocksObj<ReadOptions> readOptions;
private ReadOptions readOptions;
private final boolean canFillCache;
private final boolean smallRange;
public LLLocalKeyPrefixReactiveRocksIterator(RocksDBColumn db,
int prefixLength,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean canFillCache,
boolean smallRange) {
super(DROP);
try (range) {
this.db = db;
this.prefixLength = prefixLength;
this.rangeShared = range.receive();
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions != null ? readOptions : new RocksObj<>(new ReadOptions());
this.canFillCache = canFillCache;
this.smallRange = smallRange;
}
this.db = db;
this.prefixLength = prefixLength;
this.rangeShared = range;
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions != null ? readOptions : new ReadOptions();
this.canFillCache = canFillCache;
this.smallRange = smallRange;
}
public Flux<Send<Buffer>> flux() {
public Flux<Buffer> flux() {
return Flux.generate(() -> {
var readOptions = generateCustomReadOptions(this.readOptions,
canFillCache,
@ -132,7 +129,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
);
}
sink.next(groupKeyPrefix.send());
sink.next(groupKeyPrefix);
} else {
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(rangeShared));
@ -165,7 +162,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
var readOptions = this.readOptions;
return drop -> new LLLocalKeyPrefixReactiveRocksIterator(db,
prefixLength,
range,
range.receive(),
allowNettyDirect,
readOptions,
canFillCache,

View File

@ -3,22 +3,21 @@ package it.cavallium.dbengine.database.disk;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.rocksdb.ReadOptions;
public class LLLocalKeyReactiveRocksIterator extends LLLocalReactiveRocksIterator<Send<Buffer>> {
public class LLLocalKeyReactiveRocksIterator extends LLLocalReactiveRocksIterator<Buffer> {
public LLLocalKeyReactiveRocksIterator(RocksDBColumn db,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean reverse,
boolean smallRange) {
super(db, range, allowNettyDirect, readOptions, false, reverse, smallRange);
}
@Override
public Send<Buffer> getEntry(Send<Buffer> key, Send<Buffer> value) {
public Buffer getEntry(Buffer key, Buffer value) {
if (value != null) {
value.close();
}

View File

@ -23,7 +23,6 @@ import it.cavallium.dbengine.database.RocksDBMapProperty;
import it.cavallium.dbengine.database.RocksDBStringProperty;
import it.cavallium.dbengine.database.TableWithProperties;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.cavallium.dbengine.rpc.current.data.Column;
import it.cavallium.dbengine.rpc.current.data.ColumnOptions;
import it.cavallium.dbengine.rpc.current.data.DatabaseLevel;
@ -55,6 +54,7 @@ import org.apache.commons.lang3.time.StopWatch;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.AbstractImmutableNativeReference;
import org.rocksdb.BlockBasedTableConfig;
import org.rocksdb.BloomFilter;
import org.rocksdb.Cache;
@ -125,10 +125,10 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
private Statistics statistics;
private Cache standardCache;
private Cache compressedCache;
private final Map<Column, RocksObj<ColumnFamilyHandle>> handles;
private final Map<Column, ColumnFamilyHandle> handles;
private final HashMap<String, PersistentCache> persistentCaches;
private final ConcurrentHashMap<Long, RocksObj<Snapshot>> snapshotsHandles = new ConcurrentHashMap<>();
private final ConcurrentHashMap<Long, Snapshot> snapshotsHandles = new ConcurrentHashMap<>();
private final AtomicLong nextSnapshotNumbers = new AtomicLong(1);
private final StampedLock closeLock = new StampedLock();
private volatile boolean closed = false;
@ -467,13 +467,13 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
this.handles = new HashMap<>();
if (enableColumnsBug && !inMemory) {
for (int i = 0; i < columns.size(); i++) {
this.handles.put(columns.get(i), new RocksObj<>(handles.get(i)));
this.handles.put(columns.get(i), handles.get(i));
}
} else {
handles: for (ColumnFamilyHandle handle : handles) {
for (Column column : columns) {
if (Arrays.equals(column.name().getBytes(StandardCharsets.US_ASCII), handle.getName())) {
this.handles.put(column, new RocksObj<>(handle));
this.handles.put(column, handle);
continue handles;
}
}
@ -527,11 +527,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
RocksDBUtils.ensureOpen(db, null);
}
protected void ensureOwned(org.rocksdb.RocksObject rocksObject) {
RocksDBUtils.ensureOwned(rocksObject);
}
protected void ensureOwned(RocksObj<?> rocksObject) {
protected void ensureOwned(AbstractImmutableNativeReference rocksObject) {
RocksDBUtils.ensureOwned(rocksObject);
}
@ -571,7 +567,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
throw new IllegalArgumentException("Persistent cache " + persistentCacheId.get() + " is not defined");
}
public Map<Column, RocksObj<ColumnFamilyHandle>> getAllColumnFamilyHandles() {
public Map<Column, ColumnFamilyHandle> getAllColumnFamilyHandles() {
return this.handles;
}
@ -586,7 +582,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
ensureOpen();
var cfh = handles.get(column);
ensureOwned(cfh);
return RocksDBUtils.getLevels(db, cfh.v());
return RocksDBUtils.getLevels(db, cfh);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -598,7 +594,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
ensureOpen();
var cfh = handles.get(column);
ensureOwned(cfh);
return RocksDBUtils.getColumnFiles(db, cfh.v(), excludeLastLevel);
return RocksDBUtils.getColumnFiles(db, cfh, excludeLastLevel);
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -610,7 +606,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
ensureOpen();
for (var cfh : this.handles.values()) {
ensureOwned(cfh);
RocksDBUtils.forceCompaction(db, name, cfh.v(), volumeId, logger);
RocksDBUtils.forceCompaction(db, name, cfh, volumeId, logger);
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -658,7 +654,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
if (closed) {
return 0d;
}
return database.getLongProperty(cfh.v(), propertyName);
return database.getLongProperty(cfh, propertyName);
} catch (RocksDBException e) {
if ("NotFound".equals(e.getMessage())) {
return 0d;
@ -721,7 +717,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
snapshotsHandles.forEach((id, snapshot) -> {
try {
if (db.isOwningHandle()) {
db.releaseSnapshot(snapshot.v());
db.releaseSnapshot(snapshot);
snapshot.close();
}
} catch (Exception ex2) {
@ -1033,7 +1029,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
return stats;
}
private RocksObj<Snapshot> getSnapshotLambda(LLSnapshot snapshot) {
private Snapshot getSnapshotLambda(LLSnapshot snapshot) {
var closeReadSnapLock = closeLock.readLock();
try {
ensureOpen();
@ -1099,7 +1095,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
var closeReadLock = closeLock.readLock();
try {
ensureOpen();
RocksObj<ColumnFamilyHandle> cfh;
ColumnFamilyHandle cfh;
try {
cfh = getCfh(columnName);
ensureOwned(cfh);
@ -1112,7 +1108,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
}
}
private RocksDBColumn getRocksDBColumn(RocksDB db, RocksObj<ColumnFamilyHandle> cfh) {
private RocksDBColumn getRocksDBColumn(RocksDB db, ColumnFamilyHandle cfh) {
var nettyDirect = databaseOptions.allowNettyDirect();
var closeLock = getCloseLock();
if (db instanceof OptimisticTransactionDB optimisticTransactionDB) {
@ -1138,9 +1134,9 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
}
}
private RocksObj<ColumnFamilyHandle> getCfh(byte[] columnName) throws RocksDBException {
private ColumnFamilyHandle getCfh(byte[] columnName) throws RocksDBException {
var cfh = handles.get(ColumnUtils.special(ColumnUtils.toString(columnName)));
assert enableColumnsBug || Arrays.equals(cfh.v().getName(), columnName);
assert enableColumnsBug || Arrays.equals(cfh.getName(), columnName);
return cfh;
}
@ -1239,7 +1235,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
return db.getMapProperty(property.getName());
} else {
var cfh = requireNonNull(handles.get(column));
return db.getMapProperty(cfh.v(), property.getName());
return db.getMapProperty(cfh, property.getName());
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -1270,7 +1266,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
return db.getProperty(property.getName());
} else {
var cfh = requireNonNull(handles.get(column));
return db.getProperty(cfh.v(), property.getName());
return db.getProperty(cfh, property.getName());
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -1301,7 +1297,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
return db.getLongProperty(property.getName());
} else {
var cfh = requireNonNull(handles.get(column));
return db.getLongProperty(cfh.v(), property.getName());
return db.getLongProperty(cfh, property.getName());
}
} finally {
closeLock.unlockRead(closeReadLock);
@ -1362,7 +1358,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
aggregatedStats
.append(entry.getKey().name())
.append("\n")
.append(db.getProperty(entry.getValue().v(), "rocksdb.stats"))
.append(db.getProperty(entry.getValue(), "rocksdb.stats"))
.append("\n");
}
return aggregatedStats.toString();
@ -1388,7 +1384,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
try {
if (closed) return null;
ensureOpen();
return db.getPropertiesOfAllTables(handle.getValue().v());
return db.getPropertiesOfAllTables(handle.getValue());
} finally {
closeLock.unlockRead(closeReadLock);
}
@ -1453,7 +1449,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
try {
ensureOpen();
return snapshotTime.recordCallable(() -> {
var snapshot = new RocksObj<>(db.getSnapshot());
var snapshot = db.getSnapshot();
long currentSnapshotSequenceNumber = nextSnapshotNumbers.getAndIncrement();
this.snapshotsHandles.put(currentSnapshotSequenceNumber, snapshot);
return new LLSnapshot(currentSnapshotSequenceNumber);
@ -1476,7 +1472,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
if (!db.isOwningHandle()) {
return null;
}
db.releaseSnapshot(dbSnapshot.v());
db.releaseSnapshot(dbSnapshot);
return null;
} finally {
closeLock.unlockRead(closeReadLock);
@ -1497,7 +1493,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
flushAndCloseDb(db,
standardCache,
compressedCache,
new ArrayList<>(handles.values().stream().map(RocksObj::v).toList())
new ArrayList<>(handles.values())
);
handles.values().forEach(columnFamilyHandleRocksObj -> {
if (columnFamilyHandleRocksObj.isAccessible()) {

View File

@ -7,7 +7,7 @@ import io.netty5.buffer.api.Owned;
import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.util.function.Supplier;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.rocksdb.ReadOptions;
@ -29,13 +29,6 @@ public final class LLLocalMigrationReactiveRocksIterator extends
} catch (Throwable ex) {
logger.error("Failed to close range", ex);
}
try {
if (obj.readOptions != null) {
obj.readOptions.close();
}
} catch (Throwable ex) {
logger.error("Failed to close readOptions", ex);
}
}
@Override
@ -51,17 +44,17 @@ public final class LLLocalMigrationReactiveRocksIterator extends
private final RocksDBColumn db;
private LLRange rangeShared;
private RocksObj<ReadOptions> readOptions;
private Supplier<ReadOptions> readOptions;
@SuppressWarnings({"unchecked", "rawtypes"})
public LLLocalMigrationReactiveRocksIterator(RocksDBColumn db,
Send<LLRange> range,
Send<RocksObj<ReadOptions>> readOptions) {
Supplier<ReadOptions> readOptions) {
super((Drop<LLLocalMigrationReactiveRocksIterator>) (Drop) DROP);
try (range) {
this.db = db;
this.rangeShared = range.receive();
this.readOptions = readOptions.receive();
this.readOptions = readOptions;
}
}
@ -69,7 +62,7 @@ public final class LLLocalMigrationReactiveRocksIterator extends
public Flux<ByteEntry> flux() {
return Flux.generate(() -> {
var readOptions = generateCustomReadOptions(this.readOptions, false, false, false);
var readOptions = generateCustomReadOptions(this.readOptions.get(), false, false, false);
return new RocksIterWithReadOpts(readOptions, db.newRocksIterator(false, readOptions, rangeShared, false));
}, (tuple, sink) -> {
try {
@ -97,7 +90,7 @@ public final class LLLocalMigrationReactiveRocksIterator extends
@Override
protected Owned<LLLocalMigrationReactiveRocksIterator> prepareSend() {
var range = this.rangeShared.send();
var readOptions = this.readOptions.send();
var readOptions = this.readOptions;
return drop -> new LLLocalMigrationReactiveRocksIterator(db,
range,
readOptions

View File

@ -11,7 +11,6 @@ import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.Nullable;
@ -57,29 +56,27 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
private final RocksDBColumn db;
private LLRange rangeShared;
private final boolean allowNettyDirect;
private RocksObj<ReadOptions> readOptions;
private ReadOptions readOptions;
private final boolean readValues;
private final boolean reverse;
private final boolean smallRange;
@SuppressWarnings({"unchecked", "rawtypes"})
public LLLocalReactiveRocksIterator(RocksDBColumn db,
Send<LLRange> range,
LLRange range,
boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
boolean readValues,
boolean reverse,
boolean smallRange) {
super((Drop<LLLocalReactiveRocksIterator<T>>) (Drop) DROP);
try (range) {
this.db = db;
this.rangeShared = range.receive();
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions;
this.readValues = readValues;
this.reverse = reverse;
this.smallRange = smallRange;
}
this.db = db;
this.rangeShared = range;
this.allowNettyDirect = allowNettyDirect;
this.readOptions = readOptions;
this.readValues = readValues;
this.reverse = reverse;
this.smallRange = smallRange;
}
public final Flux<T> flux() {
@ -99,7 +96,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
} else {
key = LLUtils.fromByteArray(db.getAllocator(), rocksIterator.key());
}
try (key) {
try {
Buffer value;
if (readValues) {
if (allowNettyDirect) {
@ -126,12 +123,17 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
} else {
rocksIterator.next();
}
sink.next(getEntry(key.send(), value == null ? null : value.send()));
sink.next(getEntry(key, value));
} finally {
if (value != null) {
if (value != null && value.isAccessible()) {
value.close();
}
}
} catch (Throwable ex) {
if (key.isAccessible()) {
key.close();
}
throw ex;
}
} else {
if (logger.isTraceEnabled()) {
@ -149,7 +151,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
}, RocksIterWithReadOpts::close);
}
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
public abstract T getEntry(@Nullable Buffer key, @Nullable Buffer value);
@Override
protected final RuntimeException createResourceClosedException() {
@ -160,8 +162,8 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
protected Owned<LLLocalReactiveRocksIterator<T>> prepareSend() {
var range = this.rangeShared.send();
var readOptions = this.readOptions;
return drop -> new LLLocalReactiveRocksIterator<>(db,
range,
return drop -> new LLLocalReactiveRocksIterator<T>(db,
range.receive(),
allowNettyDirect,
readOptions,
readValues,
@ -169,7 +171,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
smallRange
) {
@Override
public T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value) {
public T getEntry(@Nullable Buffer key, @Nullable Buffer value) {
return LLLocalReactiveRocksIterator.this.getEntry(key, value);
}
};

View File

@ -10,7 +10,6 @@ import it.cavallium.dbengine.database.LLSingleton;
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateReturnMode;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.Callable;
@ -27,16 +26,16 @@ import reactor.core.scheduler.Schedulers;
public class LLLocalSingleton implements LLSingleton {
private final RocksDBColumn db;
private final Function<LLSnapshot, RocksObj<Snapshot>> snapshotResolver;
private final Function<LLSnapshot, Snapshot> snapshotResolver;
private final byte[] name;
private final String columnName;
private final Mono<Send<Buffer>> nameMono;
private final Mono<Buffer> nameMono;
private final String databaseName;
private final Scheduler dbWScheduler;
private final Scheduler dbRScheduler;
public LLLocalSingleton(RocksDBColumn db,
Function<LLSnapshot, RocksObj<Snapshot>> snapshotResolver,
Function<LLSnapshot, Snapshot> snapshotResolver,
String databaseName,
byte[] name,
String columnName,
@ -50,18 +49,17 @@ public class LLLocalSingleton implements LLSingleton {
this.columnName = columnName;
this.nameMono = Mono.fromCallable(() -> {
var alloc = db.getAllocator();
try (var nameBuf = alloc.allocate(this.name.length)) {
nameBuf.writeBytes(this.name);
return nameBuf.send();
}
var nameBuf = alloc.allocate(this.name.length);
nameBuf.writeBytes(this.name);
return nameBuf;
});
this.dbWScheduler = dbWScheduler;
this.dbRScheduler = dbRScheduler;
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Initialized in a nonblocking thread");
}
try (var readOptions = new RocksObj<>(new ReadOptions());
var writeOptions = new RocksObj<>(new WriteOptions())) {
try (var readOptions = new ReadOptions();
var writeOptions = new WriteOptions()) {
if (defaultValue != null && db.get(readOptions, this.name, true) == null) {
db.put(writeOptions, this.name, defaultValue);
}
@ -72,11 +70,11 @@ public class LLLocalSingleton implements LLSingleton {
return Mono.fromCallable(callable).subscribeOn(write ? dbWScheduler : dbRScheduler);
}
private RocksObj<ReadOptions> generateReadOptions(LLSnapshot snapshot) {
private ReadOptions generateReadOptions(LLSnapshot snapshot) {
if (snapshot != null) {
return new RocksObj<>(new ReadOptions().setSnapshot(snapshotResolver.apply(snapshot).v()));
return new ReadOptions().setSnapshot(snapshotResolver.apply(snapshot));
} else {
return new RocksObj<>(new ReadOptions());
return new ReadOptions();
}
}
@ -86,54 +84,52 @@ public class LLLocalSingleton implements LLSingleton {
}
@Override
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot) {
return nameMono.publishOn(dbRScheduler).handle((nameSend, sink) -> {
try (Buffer name = nameSend.receive()) {
public Mono<Buffer> get(@Nullable LLSnapshot snapshot) {
return nameMono.publishOn(dbRScheduler).handle((name, sink) -> {
try (name) {
Buffer result;
try (var readOptions = generateReadOptions(snapshot)) {
result = db.get(readOptions, name);
}
if (result != null) {
sink.next(result.send());
sink.next(result);
} else {
sink.complete();
}
} catch (RocksDBException ex) {
sink.error(new IOException("Failed to read " + Arrays.toString(name), ex));
sink.error(new IOException("Failed to read " + LLUtils.toString(name), ex));
}
});
}
@Override
public Mono<Void> set(Mono<Send<Buffer>> valueMono) {
public Mono<Void> set(Mono<Buffer> valueMono) {
return Mono.zip(nameMono, valueMono).publishOn(dbWScheduler).handle((tuple, sink) -> {
var nameSend = tuple.getT1();
var valueSend = tuple.getT2();
try (Buffer name = nameSend.receive();
Buffer value = valueSend.receive();
var writeOptions = new RocksObj<>(new WriteOptions())) {
var name = tuple.getT1();
var value = tuple.getT2();
try (name; value; var writeOptions = new WriteOptions()) {
db.put(writeOptions, name, value);
sink.next(true);
} catch (RocksDBException ex) {
sink.error(new IOException("Failed to write " + Arrays.toString(name), ex));
sink.error(new IOException("Failed to write " + LLUtils.toString(name), ex));
}
}).switchIfEmpty(unset().thenReturn(true)).then();
}
private Mono<Void> unset() {
return nameMono.publishOn(dbWScheduler).handle((nameSend, sink) -> {
try (Buffer name = nameSend.receive(); var writeOptions = new RocksObj<>(new WriteOptions())) {
return nameMono.publishOn(dbWScheduler).handle((name, sink) -> {
try (name; var writeOptions = new WriteOptions()) {
db.delete(writeOptions, name);
} catch (RocksDBException ex) {
sink.error(new IOException("Failed to read " + Arrays.toString(name), ex));
sink.error(new IOException("Failed to read " + LLUtils.toString(name), ex));
}
});
}
@Override
public Mono<Send<Buffer>> update(BinarySerializationFunction updater,
public Mono<Buffer> update(BinarySerializationFunction updater,
UpdateReturnMode updateReturnMode) {
return Mono.usingWhen(nameMono, keySend -> runOnDb(true, () -> {
return Mono.usingWhen(nameMono, key -> runOnDb(true, () -> {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called update in a nonblocking thread");
}
@ -143,9 +139,9 @@ public class LLLocalSingleton implements LLSingleton {
case GET_OLD_VALUE -> UpdateAtomicResultMode.PREVIOUS;
};
UpdateAtomicResult result;
try (var key = keySend.receive();
var readOptions = new RocksObj<>(new ReadOptions());
var writeOptions = new RocksObj<>(new WriteOptions())) {
try (key;
var readOptions = new ReadOptions();
var writeOptions = new WriteOptions()) {
result = db.updateAtomic(readOptions, writeOptions, key, updater, returnMode);
}
return switch (updateReturnMode) {
@ -158,15 +154,15 @@ public class LLLocalSingleton implements LLSingleton {
}
@Override
public Mono<Send<LLDelta>> updateAndGetDelta(BinarySerializationFunction updater) {
return Mono.usingWhen(nameMono, keySend -> runOnDb(true, () -> {
public Mono<LLDelta> updateAndGetDelta(BinarySerializationFunction updater) {
return Mono.usingWhen(nameMono, key -> runOnDb(true, () -> {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called update in a nonblocking thread");
}
UpdateAtomicResult result;
try (var key = keySend.receive();
var readOptions = new RocksObj<>(new ReadOptions());
var writeOptions = new RocksObj<>(new WriteOptions())) {
try (key;
var readOptions = new ReadOptions();
var writeOptions = new WriteOptions()) {
result = db.updateAtomic(readOptions, writeOptions, key, updater, DELTA);
}
return ((UpdateAtomicResultDelta) result).delta();

View File

@ -1,6 +1,5 @@
package it.cavallium.dbengine.database.disk;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
@ -64,9 +63,9 @@ public class LLTempHugePqEnv implements Closeable {
List.of(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, getColumnOptions(null))),
cfh
);
var cfhObjs = new ArrayList<RocksObj<ColumnFamilyHandle>>(cfh.size());
var cfhObjs = new ArrayList<ColumnFamilyHandle>(cfh.size());
for (ColumnFamilyHandle columnFamilyHandle : cfh) {
cfhObjs.add(new RocksObj<>(columnFamilyHandle));
cfhObjs.add(columnFamilyHandle);
}
env = new HugePqEnv(db, cfhObjs);
initialized = true;

View File

@ -10,7 +10,6 @@ import io.netty5.buffer.api.MemoryManager;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLDelta;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.cavallium.dbengine.lucene.ExponentialPageLimits;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
@ -40,7 +39,7 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
boolean nettyDirect,
BufferAllocator alloc,
String databaseName,
RocksObj<ColumnFamilyHandle> cfh,
ColumnFamilyHandle cfh,
MeterRegistry meterRegistry,
StampedLock closeLock) {
super(db, nettyDirect, alloc, databaseName, cfh, meterRegistry, closeLock);
@ -55,9 +54,9 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
}
@Override
protected boolean commitOptimistically(RocksObj<Transaction> tx) throws RocksDBException {
protected boolean commitOptimistically(Transaction tx) throws RocksDBException {
try {
tx.v().commit();
tx.commit();
return true;
} catch (RocksDBException ex) {
var status = ex.getStatus() != null ? ex.getStatus().getCode() : Code.Ok;
@ -69,19 +68,19 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
}
@Override
protected RocksObj<Transaction> beginTransaction(@NotNull RocksObj<WriteOptions> writeOptions,
RocksObj<TransactionOptions> txOpts) {
return new RocksObj<>(getDb().beginTransaction(writeOptions.v()));
protected Transaction beginTransaction(@NotNull WriteOptions writeOptions,
TransactionOptions txOpts) {
return getDb().beginTransaction(writeOptions);
}
@Override
public void write(RocksObj<WriteOptions> writeOptions, WriteBatch writeBatch) throws RocksDBException {
getDb().write(writeOptions.v(), writeBatch);
public void write(WriteOptions writeOptions, WriteBatch writeBatch) throws RocksDBException {
getDb().write(writeOptions, writeBatch);
}
@Override
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws IOException {
@ -92,16 +91,16 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called update in a nonblocking thread");
}
try (var txOpts = new RocksObj<>(new TransactionOptions());
try (var txOpts = new TransactionOptions();
var tx = beginTransaction(writeOptions, txOpts)) {
boolean committedSuccessfully;
int retries = 0;
ExponentialPageLimits retryTime = null;
Send<Buffer> sentPrevData;
Send<Buffer> sentCurData;
Buffer sentPrevData;
Buffer sentCurData;
boolean changed;
do {
var prevDataArray = tx.v().getForUpdate(readOptions.v(), cfh.v(), keyArray, true);
var prevDataArray = tx.getForUpdate(readOptions, cfh, keyArray, true);
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB,
"Reading {}: {} (before update)",
@ -139,7 +138,7 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB, "Deleting {} (after update)", LLUtils.toStringSafe(key));
}
tx.v().delete(cfh.v(), keyArray, true);
tx.delete(cfh, keyArray, true);
changed = true;
committedSuccessfully = commitOptimistically(tx);
} else if (newData != null && (prevData == null || !LLUtils.equals(prevData, newData))) {
@ -150,19 +149,19 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
LLUtils.toStringSafe(newData)
);
}
tx.v().put(cfh.v(), keyArray, newDataArray);
tx.put(cfh, keyArray, newDataArray);
changed = true;
committedSuccessfully = commitOptimistically(tx);
} else {
changed = false;
committedSuccessfully = true;
tx.v().rollback();
tx.rollback();
}
sentPrevData = prevData == null ? null : prevData.send();
sentCurData = newData == null ? null : newData.send();
sentPrevData = prevData == null ? null : prevData.copy();
sentCurData = newData == null ? null : newData.copy();
if (!committedSuccessfully) {
tx.v().undoGetForUpdate(cfh.v(), keyArray);
tx.v().rollback();
tx.undoGetForUpdate(cfh, keyArray);
tx.rollback();
if (sentPrevData != null) {
sentPrevData.close();
}
@ -222,7 +221,7 @@ public final class OptimisticRocksDBColumn extends AbstractRocksDBColumn<Optimis
yield new UpdateAtomicResultPrevious(sentPrevData);
}
case BINARY_CHANGED -> new UpdateAtomicResultBinaryChanged(changed);
case DELTA -> new UpdateAtomicResultDelta(LLDelta.of(sentPrevData, sentCurData).send());
case DELTA -> new UpdateAtomicResultDelta(LLDelta.of(sentPrevData, sentCurData));
};
}
} catch (Throwable ex) {

View File

@ -9,7 +9,6 @@ import io.netty5.buffer.api.MemoryManager;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLDelta;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.IOException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.StampedLock;
@ -32,27 +31,27 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
boolean nettyDirect,
BufferAllocator alloc,
String dbName,
RocksObj<ColumnFamilyHandle> cfh,
ColumnFamilyHandle cfh,
MeterRegistry meterRegistry,
StampedLock closeLock) {
super(db, nettyDirect, alloc, dbName, cfh, meterRegistry, closeLock);
}
@Override
protected boolean commitOptimistically(RocksObj<Transaction> tx) throws RocksDBException {
tx.v().commit();
protected boolean commitOptimistically(Transaction tx) throws RocksDBException {
tx.commit();
return true;
}
@Override
protected RocksObj<Transaction> beginTransaction(@NotNull RocksObj<WriteOptions> writeOptions,
RocksObj<TransactionOptions> txOpts) {
return new RocksObj<>(getDb().beginTransaction(writeOptions.v(), txOpts.v()));
protected Transaction beginTransaction(@NotNull WriteOptions writeOptions,
TransactionOptions txOpts) {
return getDb().beginTransaction(writeOptions, txOpts);
}
@Override
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws IOException {
@ -63,15 +62,15 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called update in a nonblocking thread");
}
try (var txOpts = new RocksObj<>(new TransactionOptions());
try (var txOpts = new TransactionOptions();
var tx = beginTransaction(writeOptions, txOpts)) {
Send<Buffer> sentPrevData;
Send<Buffer> sentCurData;
Buffer sentPrevData;
Buffer sentCurData;
boolean changed;
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB, "Reading {} (before update lock)", LLUtils.toStringSafe(key));
}
var prevDataArray = tx.v().getForUpdate(readOptions.v(), cfh.v(), keyArray, true);
var prevDataArray = tx.getForUpdate(readOptions, cfh, keyArray, true);
try {
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB,
@ -112,9 +111,9 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
logger.trace(MARKER_ROCKSDB, "Deleting {} (after update)", LLUtils.toStringSafe(key));
}
writeValueBufferSize.record(0);
tx.v().delete(cfh.v(), keyArray, true);
tx.delete(cfh, keyArray, true);
changed = true;
tx.v().commit();
tx.commit();
} else if (newData != null && (prevData == null || !LLUtils.equals(prevData, newData))) {
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB,
@ -124,19 +123,19 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
);
}
writeValueBufferSize.record(newDataArray.length);
tx.v().put(cfh.v(), keyArray, newDataArray);
tx.put(cfh, keyArray, newDataArray);
changed = true;
tx.v().commit();
tx.commit();
} else {
changed = false;
tx.v().rollback();
tx.rollback();
}
sentPrevData = prevData == null ? null : prevData.send();
sentCurData = newData == null ? null : newData.send();
sentPrevData = prevData == null ? null : prevData.copy();
sentCurData = newData == null ? null : newData.copy();
}
}
} finally {
tx.v().undoGetForUpdate(cfh.v(), keyArray);
tx.undoGetForUpdate(cfh, keyArray);
}
recordAtomicUpdateTime(changed, sentPrevData != null, sentCurData != null, initNanoTime);
return switch (returnMode) {
@ -162,7 +161,7 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
yield new UpdateAtomicResultPrevious(sentPrevData);
}
case BINARY_CHANGED -> new UpdateAtomicResultBinaryChanged(changed);
case DELTA -> new UpdateAtomicResultDelta(LLDelta.of(sentPrevData, sentCurData).send());
case DELTA -> new UpdateAtomicResultDelta(LLDelta.of(sentPrevData, sentCurData));
};
}
} catch (Throwable ex) {

View File

@ -6,7 +6,6 @@ import io.netty5.buffer.api.BufferAllocator;
import it.cavallium.dbengine.database.LLRange;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksIteratorObj;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.IOException;
import java.util.List;
import org.jetbrains.annotations.NotNull;
@ -25,11 +24,11 @@ public sealed interface RocksDBColumn permits AbstractRocksDBColumn {
* This method should not modify or move the writerIndex/readerIndex of the buffers inside the range
*/
@NotNull RocksIteratorObj newRocksIterator(boolean allowNettyDirect,
RocksObj<ReadOptions> readOptions,
ReadOptions readOptions,
LLRange range,
boolean reverse) throws RocksDBException;
default byte @Nullable [] get(@NotNull RocksObj<ReadOptions> readOptions,
default byte @Nullable [] get(@NotNull ReadOptions readOptions,
byte[] key,
boolean existsAlmostCertainly)
throws RocksDBException {
@ -46,15 +45,15 @@ public sealed interface RocksDBColumn permits AbstractRocksDBColumn {
}
@Nullable
Buffer get(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException;
Buffer get(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException;
boolean exists(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException;
boolean exists(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException;
boolean mayExists(@NotNull RocksObj<ReadOptions> readOptions, Buffer key) throws RocksDBException;
boolean mayExists(@NotNull ReadOptions readOptions, Buffer key) throws RocksDBException;
void put(@NotNull RocksObj<WriteOptions> writeOptions, Buffer key, Buffer value) throws RocksDBException;
void put(@NotNull WriteOptions writeOptions, Buffer key, Buffer value) throws RocksDBException;
default void put(@NotNull RocksObj<WriteOptions> writeOptions, byte[] key, byte[] value) throws RocksDBException {
default void put(@NotNull WriteOptions writeOptions, byte[] key, byte[] value) throws RocksDBException {
var allocator = getAllocator();
try (var keyBuf = allocator.allocate(key.length)) {
keyBuf.writeBytes(key);
@ -66,33 +65,33 @@ public sealed interface RocksDBColumn permits AbstractRocksDBColumn {
}
}
@NotNull RocksIteratorObj newIterator(@NotNull RocksObj<ReadOptions> readOptions, @Nullable Buffer min, @Nullable Buffer max);
@NotNull RocksIteratorObj newIterator(@NotNull ReadOptions readOptions, @Nullable Buffer min, @Nullable Buffer max);
@NotNull UpdateAtomicResult updateAtomic(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
@NotNull UpdateAtomicResult updateAtomic(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws RocksDBException, IOException;
void delete(RocksObj<WriteOptions> writeOptions, Buffer key) throws RocksDBException;
void delete(WriteOptions writeOptions, Buffer key) throws RocksDBException;
void delete(RocksObj<WriteOptions> writeOptions, byte[] key) throws RocksDBException;
void delete(WriteOptions writeOptions, byte[] key) throws RocksDBException;
List<byte[]> multiGetAsList(RocksObj<ReadOptions> readOptions, List<byte[]> keys) throws RocksDBException;
List<byte[]> multiGetAsList(ReadOptions readOptions, List<byte[]> keys) throws RocksDBException;
void write(RocksObj<WriteOptions> writeOptions, WriteBatch writeBatch) throws RocksDBException;
void write(WriteOptions writeOptions, WriteBatch writeBatch) throws RocksDBException;
void suggestCompactRange() throws RocksDBException;
void compactRange(byte[] begin, byte[] end, RocksObj<CompactRangeOptions> options) throws RocksDBException;
void compactRange(byte[] begin, byte[] end, CompactRangeOptions options) throws RocksDBException;
void flush(RocksObj<FlushOptions> options) throws RocksDBException;
void flush(FlushOptions options) throws RocksDBException;
void flushWal(boolean sync) throws RocksDBException;
long getLongProperty(String property) throws RocksDBException;
RocksObj<ColumnFamilyHandle> getColumnFamilyHandle();
ColumnFamilyHandle getColumnFamilyHandle();
BufferAllocator getAllocator();

View File

@ -2,13 +2,13 @@ package it.cavallium.dbengine.database.disk;
import static com.google.common.collect.Lists.partition;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import it.cavallium.dbengine.rpc.current.data.Column;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import org.apache.logging.log4j.Logger;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.AbstractImmutableNativeReference;
import org.rocksdb.ColumnFamilyHandle;
import org.rocksdb.CompactionJobInfo;
import org.rocksdb.CompactionOptions;
@ -97,7 +97,7 @@ public class RocksDBUtils {
}
}
public static void ensureOpen(RocksDB db, @Nullable RocksObj<ColumnFamilyHandle> cfh) {
public static void ensureOpen(RocksDB db, @Nullable ColumnFamilyHandle cfh) {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called in a nonblocking thread");
}
@ -105,15 +105,9 @@ public class RocksDBUtils {
ensureOwned(cfh);
}
public static void ensureOwned(@Nullable org.rocksdb.RocksObject rocksObject) {
public static void ensureOwned(@Nullable AbstractImmutableNativeReference rocksObject) {
if (rocksObject != null && !rocksObject.isOwningHandle()) {
throw new IllegalStateException("Not owning handle");
}
}
public static void ensureOwned(@Nullable RocksObj<?> rocksObject) {
if (rocksObject != null && !rocksObject.isAccessible()) {
throw new IllegalStateException("Not owning handle");
}
}
}

View File

@ -2,10 +2,9 @@ package it.cavallium.dbengine.database.disk;
import it.cavallium.dbengine.database.SafeCloseable;
import it.cavallium.dbengine.database.disk.rocksdb.RocksIteratorObj;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import org.rocksdb.ReadOptions;
public record RocksIterWithReadOpts(RocksObj<ReadOptions> readOptions, RocksIteratorObj iter) implements SafeCloseable {
public record RocksIterWithReadOpts(ReadOptions readOptions, RocksIteratorObj iter) implements SafeCloseable {
@Override
public void close() {

View File

@ -7,7 +7,6 @@ import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.BufferAllocator;
import it.cavallium.dbengine.database.LLDelta;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.IOException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.StampedLock;
@ -26,26 +25,26 @@ public final class StandardRocksDBColumn extends AbstractRocksDBColumn<RocksDB>
boolean nettyDirect,
BufferAllocator alloc,
String dbName,
RocksObj<ColumnFamilyHandle> cfh,
ColumnFamilyHandle cfh,
MeterRegistry meterRegistry,
StampedLock closeLock) {
super(db, nettyDirect, alloc, dbName, cfh, meterRegistry, closeLock);
}
@Override
protected boolean commitOptimistically(RocksObj<Transaction> tx) {
protected boolean commitOptimistically(Transaction tx) {
throw new UnsupportedOperationException("Transactions not supported");
}
@Override
protected RocksObj<Transaction> beginTransaction(@NotNull RocksObj<WriteOptions> writeOptions,
RocksObj<TransactionOptions> txOpts) {
protected Transaction beginTransaction(@NotNull WriteOptions writeOptions,
TransactionOptions txOpts) {
throw new UnsupportedOperationException("Transactions not supported");
}
@Override
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull RocksObj<ReadOptions> readOptions,
@NotNull RocksObj<WriteOptions> writeOptions,
public @NotNull UpdateAtomicResult updateAtomicImpl(@NotNull ReadOptions readOptions,
@NotNull WriteOptions writeOptions,
Buffer key,
BinarySerializationFunction updater,
UpdateAtomicResultMode returnMode) throws IOException {
@ -117,15 +116,14 @@ public final class StandardRocksDBColumn extends AbstractRocksDBColumn<RocksDB>
yield RESULT_NOTHING;
}
case CURRENT -> {
yield new UpdateAtomicResultCurrent(newData != null ? newData.send() : null);
yield new UpdateAtomicResultCurrent(newData != null ? newData.copy() : null);
}
case PREVIOUS -> {
yield new UpdateAtomicResultPrevious(prevData != null ? prevData.send() : null);
yield new UpdateAtomicResultPrevious(prevData != null ? prevData.copy() : null);
}
case BINARY_CHANGED -> new UpdateAtomicResultBinaryChanged(changed);
case DELTA -> new UpdateAtomicResultDelta(LLDelta
.of(prevData != null ? prevData.send() : null, newData != null ? newData.send() : null)
.send());
.of(prevData != null ? prevData.copy() : null, newData != null ? newData.copy() : null));
};
}
}

View File

@ -3,4 +3,4 @@ package it.cavallium.dbengine.database.disk;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
public final record UpdateAtomicResultCurrent(Send<Buffer> current) implements UpdateAtomicResult {}
public final record UpdateAtomicResultCurrent(Buffer current) implements UpdateAtomicResult {}

View File

@ -3,4 +3,4 @@ package it.cavallium.dbengine.database.disk;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.database.LLDelta;
public final record UpdateAtomicResultDelta(Send<LLDelta> delta) implements UpdateAtomicResult {}
public final record UpdateAtomicResultDelta(LLDelta delta) implements UpdateAtomicResult {}

View File

@ -3,4 +3,4 @@ package it.cavallium.dbengine.database.disk;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Send;
public final record UpdateAtomicResultPrevious(Send<Buffer> previous) implements UpdateAtomicResult {}
public final record UpdateAtomicResultPrevious(Buffer previous) implements UpdateAtomicResult {}

View File

@ -43,8 +43,8 @@ public class RocksIteratorObj extends ResourceSupport<RocksIteratorObj, RocksIte
};
private RocksIterator rocksIterator;
private RocksObj<? extends AbstractSlice<?>> sliceMin;
private RocksObj<? extends AbstractSlice<?>> sliceMax;
private AbstractSlice<?> sliceMin;
private AbstractSlice<?> sliceMax;
private Buffer min;
private Buffer max;
private final boolean allowNettyDirect;
@ -58,8 +58,8 @@ public class RocksIteratorObj extends ResourceSupport<RocksIteratorObj, RocksIte
private Object seekingTo;
public RocksIteratorObj(RocksIterator rocksIterator,
RocksObj<? extends AbstractSlice<?>> sliceMin,
RocksObj<? extends AbstractSlice<?>> sliceMax,
AbstractSlice<?> sliceMin,
AbstractSlice<?> sliceMax,
Buffer min,
Buffer max,
boolean allowNettyDirect,
@ -87,8 +87,8 @@ public class RocksIteratorObj extends ResourceSupport<RocksIteratorObj, RocksIte
}
private RocksIteratorObj(RocksIterator rocksIterator,
RocksObj<? extends AbstractSlice<?>> sliceMin,
RocksObj<? extends AbstractSlice<?>> sliceMax,
AbstractSlice<?> sliceMin,
AbstractSlice<?> sliceMax,
Buffer min,
Buffer max,
boolean allowNettyDirect,

View File

@ -1,62 +0,0 @@
package it.cavallium.dbengine.database.disk.rocksdb;
import io.netty5.buffer.api.Drop;
import io.netty5.buffer.api.Owned;
import io.netty5.buffer.api.internal.ResourceSupport;
import org.jetbrains.annotations.NotNull;
import org.rocksdb.AbstractNativeReference;
public class RocksObj<T extends AbstractNativeReference> extends ResourceSupport<RocksObj<T>, RocksObj<T>> {
private static final Drop<RocksObj<?>> DROP = new Drop<>() {
@Override
public void drop(RocksObj obj) {
if (obj.val != null) {
if (obj.val.isAccessible()) obj.val.close();
}
}
@Override
public Drop<RocksObj<?>> fork() {
return this;
}
@Override
public void attach(RocksObj obj) {
}
};
private T val;
public RocksObj(T val) {
//noinspection unchecked
super((Drop<RocksObj<T>>) (Drop<?>) DROP);
this.val = val;
}
@NotNull
public T v() {
return val;
}
@Override
protected void makeInaccessible() {
this.val = null;
}
@Override
protected RuntimeException createResourceClosedException() {
return new IllegalStateException("Closed");
}
@Override
protected Owned<RocksObj<T>> prepareSend() {
var val = this.val;
return drop -> {
var instance = new RocksObj<>(val);
drop.attach(instance);
return instance;
};
}
}

View File

@ -2,6 +2,7 @@ package it.cavallium.dbengine.database.memory;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.BufferAllocator;
import io.netty5.buffer.api.Resource;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.client.BadBlock;
import it.cavallium.dbengine.database.LLDelta;
@ -78,10 +79,10 @@ public class LLMemoryDictionary implements LLDictionary {
}
}
private Mono<Send<Buffer>> transformResult(Mono<ByteList> result, LLDictionaryResultType resultType) {
private Mono<Buffer> transformResult(Mono<ByteList> result, LLDictionaryResultType resultType) {
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE) {
// Don't retain the result because it has been removed from the skip list
return result.map(this::kk);
return result.map(this::kkB);
} else if (resultType == LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) {
return result
.map(prev -> true)
@ -98,6 +99,16 @@ public class LLMemoryDictionary implements LLDictionary {
}
}
private ByteList kShr(Buffer buf) {
return new BinaryLexicographicList(LLUtils.toArray(buf));
}
private ByteList kOwn(Buffer buf) {
try (buf) {
return new BinaryLexicographicList(LLUtils.toArray(buf));
}
}
private Send<Buffer> kk(ByteList bytesList) {
try (var buffer = getAllocator().allocate(bytesList.size())) {
buffer.writeBytes(bytesList.toByteArray());
@ -132,56 +143,54 @@ public class LLMemoryDictionary implements LLDictionary {
}
}
private ConcurrentNavigableMap<ByteList, ByteList> mapSlice(LLSnapshot snapshot, Send<LLRange> rangeToReceive) {
try (var range = rangeToReceive.receive()) {
if (range.isAll()) {
return snapshots.get(resolveSnapshot(snapshot));
} else if (range.isSingle()) {
var key = k(range.getSingle());
var value = snapshots
.get(resolveSnapshot(snapshot))
.get(key);
if (value != null) {
return new ConcurrentSkipListMap<>(Map.of(key, value));
} else {
return new ConcurrentSkipListMap<>(Map.of());
}
} else if (range.hasMin() && range.hasMax()) {
var min = k(range.getMin());
var max = k(range.getMax());
if (min.compareTo(max) > 0) {
return new ConcurrentSkipListMap<>(Map.of());
}
return snapshots
.get(resolveSnapshot(snapshot))
.subMap(min, true, max, false);
} else if (range.hasMin()) {
return snapshots
.get(resolveSnapshot(snapshot))
.tailMap(k(range.getMin()), true);
private ConcurrentNavigableMap<ByteList, ByteList> mapSlice(LLSnapshot snapshot, LLRange range) {
if (range.isAll()) {
return snapshots.get(resolveSnapshot(snapshot));
} else if (range.isSingle()) {
var key = k(range.getSingle());
var value = snapshots
.get(resolveSnapshot(snapshot))
.get(key);
if (value != null) {
return new ConcurrentSkipListMap<>(Map.of(key, value));
} else {
return snapshots
.get(resolveSnapshot(snapshot))
.headMap(k(range.getMax()), false);
return new ConcurrentSkipListMap<>(Map.of());
}
} else if (range.hasMin() && range.hasMax()) {
var min = k(range.getMin());
var max = k(range.getMax());
if (min.compareTo(max) > 0) {
return new ConcurrentSkipListMap<>(Map.of());
}
return snapshots
.get(resolveSnapshot(snapshot))
.subMap(min, true, max, false);
} else if (range.hasMin()) {
return snapshots
.get(resolveSnapshot(snapshot))
.tailMap(k(range.getMin()), true);
} else {
return snapshots
.get(resolveSnapshot(snapshot))
.headMap(k(range.getMax()), false);
}
}
@Override
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
public Mono<Buffer> get(@Nullable LLSnapshot snapshot, Mono<Buffer> keyMono) {
return Mono.usingWhen(keyMono,
key -> Mono
.fromCallable(() -> snapshots.get(resolveSnapshot(snapshot)).get(k(key)))
.map(this::kk)
.fromCallable(() -> snapshots.get(resolveSnapshot(snapshot)).get(kShr(key)))
.map(this::kkB)
.onErrorMap(cause -> new IOException("Failed to read", cause)),
key -> Mono.fromRunnable(key::close)
);
}
@Override
public Mono<Send<Buffer>> put(Mono<Send<Buffer>> keyMono, Mono<Send<Buffer>> valueMono, LLDictionaryResultType resultType) {
var kMono = keyMono.map(this::k);
var vMono = valueMono.map(this::k);
public Mono<Buffer> put(Mono<Buffer> keyMono, Mono<Buffer> valueMono, LLDictionaryResultType resultType) {
var kMono = keyMono.map(this::kOwn);
var vMono = valueMono.map(this::kOwn);
return Mono
.zip(kMono, vMono)
.mapNotNull(tuple -> mainDb.put(tuple.getT1(), tuple.getT2()))
@ -195,38 +204,38 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Mono<Send<LLDelta>> updateAndGetDelta(Mono<Send<Buffer>> keyMono, BinarySerializationFunction updater) {
public Mono<LLDelta> updateAndGetDelta(Mono<Buffer> keyMono, BinarySerializationFunction updater) {
return Mono.usingWhen(keyMono,
key -> Mono.fromCallable(() -> {
try (key) {
if (updateMode == UpdateMode.DISALLOW) {
throw new UnsupportedOperationException("update() is disallowed");
}
AtomicReference<Send<Buffer>> oldRef = new AtomicReference<>(null);
var newValue = mainDb.compute(k(key), (_unused, old) -> {
if (old != null) {
oldRef.set(kk(old));
}
Buffer v;
try (var oldToSend = old != null ? kk(old).receive() : null) {
v = updater.apply(oldToSend);
} catch (SerializationException e) {
throw new IllegalStateException(e);
}
try {
if (v != null) {
return k(v.send());
} else {
return null;
}
} finally {
if (v != null) {
v.close();
}
}
});
return LLDelta.of(oldRef.get(), newValue != null ? kk(newValue) : null).send();
if (updateMode == UpdateMode.DISALLOW) {
throw new UnsupportedOperationException("update() is disallowed");
}
AtomicReference<ByteList> oldRef = new AtomicReference<>(null);
var newValue = mainDb.compute(kShr(key), (_unused, old) -> {
if (old != null) {
oldRef.set(old);
}
Buffer v;
var oldToSend = old != null ? kkB(old) : null;
try {
assert oldToSend == null || oldToSend.isAccessible();
v = updater.apply(oldToSend);
assert v == null || v.isAccessible();
} catch (SerializationException e) {
throw new IllegalStateException(e);
} finally {
if (oldToSend != null && oldToSend.isAccessible()) {
oldToSend.close();
}
}
if (v != null) {
return kOwn(v);
} else {
return null;
}
});
var oldVal = oldRef.get();
return LLDelta.of(oldVal != null ? kkB(oldRef.get()) : null, newValue != null ? kkB(newValue) : null);
}),
key -> Mono.fromRunnable(key::close)
);
@ -238,15 +247,15 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Mono<Send<Buffer>> remove(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
public Mono<Buffer> remove(Mono<Buffer> keyMono, LLDictionaryResultType resultType) {
return Mono.usingWhen(keyMono,
key -> Mono
.fromCallable(() -> mainDb.remove(k(key)))
.fromCallable(() -> mainDb.remove(kShr(key)))
// Don't retain the result because it has been removed from the skip list
.mapNotNull(bytesList -> switch (resultType) {
case VOID -> null;
case PREVIOUS_VALUE_EXISTENCE -> LLUtils.booleanToResponseByteBuffer(allocator, true);
case PREVIOUS_VALUE -> kk(bytesList);
case PREVIOUS_VALUE -> kkB(bytesList);
})
.switchIfEmpty(Mono.defer(() -> {
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) {
@ -261,10 +270,10 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Flux<Optional<Buffer>> getMulti(@Nullable LLSnapshot snapshot, Flux<Send<Buffer>> keys) {
public Flux<Optional<Buffer>> getMulti(@Nullable LLSnapshot snapshot, Flux<Buffer> keys) {
return keys.map(key -> {
try (var t2 = key.receive()) {
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(t2.copy().send()));
try (key) {
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(key.copy().send()));
if (v != null) {
return Optional.of(kkB(v));
} else {
@ -275,87 +284,79 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Mono<Void> putMulti(Flux<Send<LLEntry>> entries) {
return entries.doOnNext(entryToReceive -> {
try (var entry = entryToReceive.receive()) {
try (var key = entry.getKey().receive()) {
try (var val = entry.getValue().receive()) {
mainDb.put(k(key.copy().send()), k(val.send()));
}
}
public Mono<Void> putMulti(Flux<LLEntry> entries) {
return entries.doOnNext(entry -> {
try (entry) {
mainDb.put(k(entry.getKeyUnsafe().send()), k(entry.getValueUnsafe().send()));
}
}).then();
}
@Override
public <K> Flux<Boolean> updateMulti(Flux<K> keys,
Flux<Send<Buffer>> serializedKeys,
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Buffer> updateFunction) {
Flux<Buffer> serializedKeys,
KVSerializationFunction<K, @Nullable Buffer, @Nullable Buffer> updateFunction) {
return Flux.error(new UnsupportedOperationException("Not implemented"));
}
@Override
public Flux<Send<LLEntry>> getRange(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
public Flux<LLEntry> getRange(@Nullable LLSnapshot snapshot,
Mono<LLRange> rangeMono,
boolean reverse,
boolean smallRange) {
return Flux.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var element = snapshots.get(resolveSnapshot(snapshot)).get(k(single.copy().send()));
if (element != null) {
return LLEntry.of(single.send(), kk(element)).send();
} else {
return null;
}
return Flux.usingWhen(rangeMono, range -> {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var element = snapshots.get(resolveSnapshot(snapshot)).get(k(single.copy().send()));
if (element != null) {
return LLEntry.of(single, kkB(element));
} else {
return null;
}
}).flux();
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
.flatMapIterable(map -> {
if (reverse) {
return map.descendingMap().entrySet();
} else {
return map.entrySet();
}
})
.map(entry -> LLEntry.of(kk(entry.getKey()), kk(entry.getValue())).send());
}
}
}).flux();
} else {
return Mono
.fromCallable(() -> mapSlice(snapshot, range))
.flatMapIterable(map -> {
if (reverse) {
return map.descendingMap().entrySet();
} else {
return map.entrySet();
}
})
.map(entry -> LLEntry.of(kkB(entry.getKey()), kkB(entry.getValue())));
}
}, range -> Mono.fromRunnable(range::close));
}
@Override
public Flux<List<Send<LLEntry>>> getRangeGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
public Flux<List<LLEntry>> getRangeGrouped(@Nullable LLSnapshot snapshot,
Mono<LLRange> rangeMono,
int prefixLength, boolean smallRange) {
return Flux.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
return Flux.usingWhen(rangeMono, range -> {
try (range) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var element = snapshots.get(resolveSnapshot(snapshot)).get(k(single.copy().send()));
if (element != null) {
return List.of(LLEntry.of(single.send(), kk(element)).send());
return List.of(LLEntry.of(single, kkB(element)));
} else {
return List.<Send<LLEntry>>of();
return List.<LLEntry>of();
}
}
}).flux();
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
.fromCallable(() -> mapSlice(snapshot, range))
.flatMapIterable(SortedMap::entrySet)
.groupBy(k -> k.getKey().subList(0, prefixLength))
.flatMap(groupedFlux -> groupedFlux
.map(entry -> LLEntry.of(kk(entry.getKey()), kk(entry.getValue())).send())
.map(entry -> LLEntry.of(kkB(entry.getKey()), kkB(entry.getValue())))
.collectList()
);
}
@ -364,34 +365,35 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Flux<Send<Buffer>> getRangeKeys(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
public Flux<Buffer> getRangeKeys(@Nullable LLSnapshot snapshot,
Mono<LLRange> rangeMono,
boolean reverse,
boolean smallRange) {
return Flux.usingWhen(rangeMono,
rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var contains = snapshots.get(resolveSnapshot(snapshot)).containsKey(k(single.copy().send()));
return contains ? single.send() : null;
}
}).flux();
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
.<ByteList>flatMapIterable(map -> {
if (reverse) {
return map.descendingMap().keySet();
} else {
return map.keySet();
}
})
.map(this::kk);
}
range -> {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
var single = singleToReceive.receive();
try {
var contains = snapshots.get(resolveSnapshot(snapshot)).containsKey(k(single.copy().send()));
return contains ? single : null;
} catch (Throwable ex) {
single.close();
throw ex;
}
}).flux();
} else {
return Mono
.fromCallable(() -> mapSlice(snapshot, range))
.<ByteList>flatMapIterable(map -> {
if (reverse) {
return map.descendingMap().keySet();
} else {
return map.keySet();
}
})
.map(this::kkB);
}
},
range -> Mono.fromRunnable(range::close)
@ -399,31 +401,34 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Flux<List<Send<Buffer>>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
public Flux<List<Buffer>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
Mono<LLRange> rangeMono,
int prefixLength, boolean smallRange) {
return Flux.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
return Flux.usingWhen(rangeMono, range -> {
try (range) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var single = singleToReceive.receive();
try {
var containsElement = snapshots.get(resolveSnapshot(snapshot)).containsKey(k(single.copy().send()));
if (containsElement) {
return List.of(single.send());
return List.of(single);
} else {
return List.<Send<Buffer>>of();
return List.<Buffer>of();
}
} catch (Throwable ex) {
single.close();
throw ex;
}
}).flux();
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
.fromCallable(() -> mapSlice(snapshot, range))
.flatMapIterable(SortedMap::entrySet)
.groupBy(k -> k.getKey().subList(0, prefixLength))
.flatMap(groupedFlux -> groupedFlux
.map(entry -> kk(entry.getKey()))
.map(entry -> kkB(entry.getKey()))
.collectList()
);
}
@ -433,11 +438,11 @@ public class LLMemoryDictionary implements LLDictionary {
@SuppressWarnings("RedundantCast")
@Override
public Flux<Send<Buffer>> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
public Flux<Buffer> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot,
Mono<LLRange> rangeMono,
int prefixLength, boolean smallRange) {
return Flux.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
return Flux.usingWhen(rangeMono, range -> {
try (range) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
@ -445,62 +450,58 @@ public class LLMemoryDictionary implements LLDictionary {
var k = k(single.copy().send());
var containsElement = snapshots.get(resolveSnapshot(snapshot)).containsKey(k);
if (containsElement) {
return kk(k.subList(0, prefixLength));
return kkB(k.subList(0, prefixLength));
} else {
return null;
}
}
}).flux();
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
.fromCallable(() -> mapSlice(snapshot, range))
.flatMapIterable(SortedMap::entrySet)
.map(k -> (ByteList) k.getKey().subList(0, prefixLength))
.distinctUntilChanged()
.map(this::kk);
.map(this::kkB);
}
}
}, range -> Mono.fromRunnable(range::close));
}
@Override
public Flux<BadBlock> badBlocks(Mono<Send<LLRange>> rangeMono) {
public Flux<BadBlock> badBlocks(Mono<LLRange> rangeMono) {
return Flux.empty();
}
@Override
public Mono<Void> setRange(Mono<Send<LLRange>> rangeMono, Flux<Send<LLEntry>> entries, boolean smallRange) {
return Mono.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
Mono<Void> clearMono;
if (range.isSingle()) {
var singleToReceive = range.getSingle();
clearMono = Mono.fromRunnable(() -> {
try (var single = singleToReceive.receive()) {
var k = k(single.copy().send());
mainDb.remove(k);
}
});
} else {
var rangeToReceive2 = range.copy().send();
clearMono = Mono.fromRunnable(() -> mapSlice(null, rangeToReceive2).clear());
}
var r = r(range.copy().send());
return clearMono
.thenMany(entries)
.doOnNext(entryToReceive -> {
try (var entry = entryToReceive.receive()) {
if (!isInsideRange(r, k(entry.getKey()))) {
throw new IndexOutOfBoundsException("Trying to set a key outside the range!");
}
mainDb.put(k(entry.getKey()), k(entry.getValue()));
}
})
.then();
public Mono<Void> setRange(Mono<LLRange> rangeMono, Flux<LLEntry> entries, boolean smallRange) {
return Mono.usingWhen(rangeMono, range -> {
Mono<Void> clearMono;
if (range.isSingle()) {
var singleToReceive = range.getSingle();
clearMono = Mono.fromRunnable(() -> {
try (var single = singleToReceive.receive()) {
var k = k(single.copy().send());
mainDb.remove(k);
}
});
} else {
clearMono = Mono.fromRunnable(() -> mapSlice(null, range).clear());
}
var r = r(range.copy().send());
return clearMono
.thenMany(entries)
.doOnNext(entry -> {
try (entry) {
if (!isInsideRange(r, kShr(entry.getKeyUnsafe()))) {
throw new IndexOutOfBoundsException("Trying to set a key outside the range!");
}
mainDb.put(kShr(entry.getKeyUnsafe()), kShr(entry.getValueUnsafe()));
}
})
.then();
}, range -> Mono.fromRunnable(range::close));
}
@ -524,15 +525,15 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fillCache) {
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono, boolean fillCache) {
return getRangeKeys(snapshot, rangeMono, false, false)
.doOnNext(buf -> buf.receive().close())
.doOnNext(Resource::close)
.count()
.map(count -> count == 0);
}
@Override
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fast) {
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono, boolean fast) {
return Mono.usingWhen(rangeMono,
range -> Mono.fromCallable(() -> (long) mapSlice(snapshot, range).size()),
range -> Mono.fromRunnable(range::close)
@ -540,39 +541,38 @@ public class LLMemoryDictionary implements LLDictionary {
}
@Override
public Mono<Send<LLEntry>> getOne(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
public Mono<LLEntry> getOne(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
return getRange(snapshot, rangeMono, false, false)
.take(1, true)
.singleOrEmpty();
}
@Override
public Mono<Send<Buffer>> getOneKey(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
public Mono<Buffer> getOneKey(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
return getRangeKeys(snapshot, rangeMono, false, false)
.take(1, true)
.singleOrEmpty();
}
@Override
public Mono<Send<LLEntry>> removeOne(Mono<Send<LLRange>> rangeMono) {
return Mono.usingWhen(rangeMono, rangeToReceive -> {
try (var range = rangeToReceive.receive()) {
public Mono<LLEntry> removeOne(Mono<LLRange> rangeMono) {
return Mono.usingWhen(rangeMono, range -> {
try (range) {
if (range.isSingle()) {
var singleToReceive = range.getSingle();
return Mono.fromCallable(() -> {
try (var single = singleToReceive.receive()) {
var element = mainDb.remove(k(single.copy().send()));
if (element != null) {
return LLEntry.of(single.send(), kk(element)).send();
return LLEntry.of(single, kkB(element));
} else {
return null;
}
}
});
} else {
var rangeToReceive2 = range.send();
return Mono
.fromCallable(() -> mapSlice(null, rangeToReceive2))
.fromCallable(() -> mapSlice(null, range))
.mapNotNull(map -> {
var it = map.entrySet().iterator();
if (it.hasNext()) {
@ -583,7 +583,7 @@ public class LLMemoryDictionary implements LLDictionary {
return null;
}
})
.map(entry -> LLEntry.of(kk(entry.getKey()), kk(entry.getValue())).send());
.map(entry -> LLEntry.of(kkB(entry.getKey()), kkB(entry.getValue())));
}
}
}, range -> Mono.fromRunnable(range::close));

View File

@ -73,7 +73,7 @@ public class LLMemoryKeyValueDatabase implements LLKeyValueDatabase {
.transform(mono -> {
if (defaultValue != null) {
return mono.switchIfEmpty(singleton
.set(Mono.fromSupplier(() -> allocator.copyOf(defaultValue).send()))
.set(Mono.fromSupplier(() -> allocator.copyOf(defaultValue)))
.then(Mono.empty()));
} else {
return mono;

View File

@ -18,17 +18,14 @@ public class LLMemorySingleton implements LLSingleton {
private final LLMemoryDictionary dict;
private final String columnNameString;
private final byte[] singletonName;
private final Mono<Send<Buffer>> singletonNameBufMono;
private final Mono<Buffer> singletonNameBufMono;
public LLMemorySingleton(LLMemoryDictionary dict, String columnNameString, byte[] singletonName) {
this.dict = dict;
this.columnNameString = columnNameString;
this.singletonName = singletonName;
this.singletonNameBufMono = Mono.fromCallable(() -> dict
.getAllocator()
.allocate(singletonName.length)
.writeBytes(singletonName)
.send());
this.singletonNameBufMono = Mono.fromSupplier(() -> dict.getAllocator().allocate(singletonName.length)
.writeBytes(singletonName));
}
@Override
@ -42,12 +39,12 @@ public class LLMemorySingleton implements LLSingleton {
}
@Override
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot) {
public Mono<Buffer> get(@Nullable LLSnapshot snapshot) {
return dict.get(snapshot, singletonNameBufMono);
}
@Override
public Mono<Void> set(Mono<Send<Buffer>> value) {
public Mono<Void> set(Mono<Buffer> value) {
var bbKey = singletonNameBufMono;
return dict
.put(bbKey, value, LLDictionaryResultType.VOID)
@ -55,13 +52,13 @@ public class LLMemorySingleton implements LLSingleton {
}
@Override
public Mono<Send<Buffer>> update(BinarySerializationFunction updater,
public Mono<Buffer> update(BinarySerializationFunction updater,
UpdateReturnMode updateReturnMode) {
return dict.update(singletonNameBufMono, updater, updateReturnMode);
}
@Override
public Mono<Send<LLDelta>> updateAndGetDelta(BinarySerializationFunction updater) {
public Mono<LLDelta> updateAndGetDelta(BinarySerializationFunction updater) {
return dict.updateAndGetDelta(singletonNameBufMono, updater);
}

View File

@ -246,12 +246,12 @@ public class LLQuicConnection implements LLDatabaseConnection {
}
@Override
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot) {
public Mono<Buffer> get(@Nullable LLSnapshot snapshot) {
return sendRequest(new SingletonGet(singletonId, NullableLLSnapshot.ofNullable(snapshot)))
.cast(BinaryOptional.class)
.mapNotNull(result -> {
if (result.val().isPresent()) {
return allocator.copyOf(QuicUtils.toArrayNoCopy(result.val().get().val())).send();
return allocator.copyOf(QuicUtils.toArrayNoCopy(result.val().get().val()));
} else {
return null;
}
@ -259,13 +259,13 @@ public class LLQuicConnection implements LLDatabaseConnection {
}
@Override
public Mono<Void> set(Mono<Send<Buffer>> valueMono) {
public Mono<Void> set(Mono<Buffer> valueMono) {
return QuicUtils.toBytes(valueMono)
.flatMap(valueSendOpt -> sendRequest(new SingletonSet(singletonId, valueSendOpt)).then());
}
@Override
public Mono<Send<Buffer>> update(BinarySerializationFunction updater, UpdateReturnMode updateReturnMode) {
public Mono<Buffer> update(BinarySerializationFunction updater, UpdateReturnMode updateReturnMode) {
return LLQuicConnection.this.<BinaryOptional, SingletonUpdateOldData>sendUpdateRequest(new SingletonUpdateInit(singletonId, updateReturnMode), prev -> {
byte[] oldData = toArrayNoCopy(prev);
Buffer oldDataBuf;
@ -289,7 +289,7 @@ public class LLQuicConnection implements LLDatabaseConnection {
}
}).mapNotNull(result -> {
if (result.val().isPresent()) {
return allocator.copyOf(QuicUtils.toArrayNoCopy(result.val().get().val())).send();
return allocator.copyOf(QuicUtils.toArrayNoCopy(result.val().get().val()));
} else {
return null;
}
@ -297,7 +297,7 @@ public class LLQuicConnection implements LLDatabaseConnection {
}
@Override
public Mono<Send<LLDelta>> updateAndGetDelta(BinarySerializationFunction updater) {
public Mono<LLDelta> updateAndGetDelta(BinarySerializationFunction updater) {
return Mono.error(new UnsupportedOperationException());
}

View File

@ -43,9 +43,9 @@ public class QuicUtils {
}
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
public static NullableBytes toBytes(Optional<Send<Buffer>> valueSendOpt) {
public static NullableBytes toBytes(Optional<Buffer> valueSendOpt) {
if (valueSendOpt.isPresent()) {
try (var value = valueSendOpt.get().receive()) {
try (var value = valueSendOpt.get()) {
var bytes = new byte[value.readableBytes()];
value.copyInto(value.readerOffset(), bytes, 0, bytes.length);
return NullableBytes.ofNullable(ByteList.of(bytes));
@ -55,9 +55,9 @@ public class QuicUtils {
}
}
public static Mono<NullableBytes> toBytes(Mono<Send<Buffer>> valueSendOptMono) {
public static Mono<NullableBytes> toBytes(Mono<Buffer> valueSendOptMono) {
return valueSendOptMono.map(valueSendOpt -> {
try (var value = valueSendOpt.receive()) {
try (var value = valueSendOpt) {
var bytes = new byte[value.readableBytes()];
value.copyInto(value.readerOffset(), bytes, 0, bytes.length);
return NullableBytes.ofNullable(ByteList.of(bytes));

View File

@ -2,7 +2,7 @@ package it.cavallium.dbengine.database.serialization;
import java.io.IOException;
public class SerializationException extends IOException {
public class SerializationException extends RuntimeException {
public SerializationException() {
super();

View File

@ -6,7 +6,6 @@ import it.cavallium.dbengine.database.SafeCloseable;
import it.cavallium.dbengine.database.disk.LLTempHugePqEnv;
import it.cavallium.dbengine.database.disk.HugePqEnv;
import it.cavallium.dbengine.database.disk.StandardRocksDBColumn;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.util.concurrent.atomic.AtomicBoolean;
import org.jetbrains.annotations.Nullable;
import org.rocksdb.ReadOptions;
@ -41,12 +40,12 @@ public class HugePqArray<V> implements IArray<V>, SafeCloseable {
this.virtualSize = size;
}
private static RocksObj<ReadOptions> newReadOptions() {
return new RocksObj<>(new ReadOptions().setVerifyChecksums(false));
private static ReadOptions newReadOptions() {
return new ReadOptions().setVerifyChecksums(false);
}
private static RocksObj<WriteOptions> newWriteOptions() {
return new RocksObj<>(new WriteOptions().setDisableWAL(true).setSync(false));
private static WriteOptions newWriteOptions() {
return new WriteOptions().setDisableWAL(true).setSync(false);
}
public HugePqCodec<V> getValueCodec() {

View File

@ -11,7 +11,6 @@ import it.cavallium.dbengine.database.disk.StandardRocksDBColumn;
import it.cavallium.dbengine.database.disk.UpdateAtomicResultMode;
import it.cavallium.dbengine.database.disk.UpdateAtomicResultPrevious;
import it.cavallium.dbengine.database.disk.rocksdb.RocksIteratorObj;
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -51,12 +50,12 @@ public class HugePqPriorityQueue<T> implements PriorityQueue<T>, Reversable<Reve
this.codec = codec;
}
private static RocksObj<ReadOptions> newReadOptions() {
return new RocksObj<>(new ReadOptions().setVerifyChecksums(false));
private static ReadOptions newReadOptions() {
return new ReadOptions().setVerifyChecksums(false);
}
private static RocksObj<WriteOptions> newWriteOptions() {
return new RocksObj<>(new WriteOptions().setDisableWAL(true).setSync(false));
private static WriteOptions newWriteOptions() {
return new WriteOptions().setDisableWAL(true).setSync(false);
}
private Buffer allocate(int size) {
@ -190,7 +189,7 @@ public class HugePqPriorityQueue<T> implements PriorityQueue<T>, Reversable<Reve
public void clear() {
ensureThread();
try (var wb = new WriteBatch(); var wo = newWriteOptions()) {
wb.deleteRange(rocksDB.getColumnFamilyHandle().v(), new byte[0], getBiggestKey());
wb.deleteRange(rocksDB.getColumnFamilyHandle(), new byte[0], getBiggestKey());
size = 0;
rocksDB.write(wo, wb);
} catch (RocksDBException e) {

View File

@ -4,8 +4,12 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.MemoryManager;
import io.netty5.buffer.api.internal.LeakDetection;
import io.netty5.buffer.api.internal.LifecycleTracer;
import io.netty5.buffer.api.pool.PoolArenaMetric;
import io.netty5.buffer.api.pool.PooledBufferAllocator;
import io.netty5.util.ResourceLeakDetector;
import io.netty5.util.ResourceLeakDetector.Level;
import io.netty5.util.internal.PlatformDependent;
import it.cavallium.dbengine.client.LuceneIndex;
import it.cavallium.dbengine.client.LuceneIndexImpl;
@ -13,6 +17,7 @@ import it.cavallium.dbengine.database.LLDatabaseConnection;
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLKeyValueDatabase;
import it.cavallium.dbengine.database.LLLuceneIndex;
import it.cavallium.dbengine.database.LLUtils;
import it.cavallium.dbengine.database.UpdateMode;
import it.cavallium.dbengine.database.collections.DatabaseMapDictionary;
import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep;
@ -34,6 +39,10 @@ import reactor.core.publisher.Mono;
public class DbTestUtils {
static {
LLUtils.initHooks();
}
public static final String BIG_STRING = generateBigString();
public static final int MAX_IN_MEMORY_RESULT_ENTRIES = 8192;
@ -93,6 +102,8 @@ public class DbTestUtils {
}
public static void ensureNoLeaks(TestAllocatorImpl allocator, boolean printStats, boolean useClassicException) {
ResourceLeakDetector.setLevel(Level.PARANOID);
System.gc();
if (allocator != null) {
var allocs = getActiveAllocations(allocator, printStats);
if (useClassicException) {

View File

@ -8,6 +8,7 @@ import static it.cavallium.dbengine.SyncUtils.runVoid;
import static org.junit.jupiter.api.Assertions.assertEquals;
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Resource;
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.DbTestUtils.TempDb;
import it.cavallium.dbengine.DbTestUtils.TestAllocator;
@ -36,7 +37,7 @@ import reactor.core.publisher.Mono;
public abstract class TestLLDictionary {
private final Logger log = LogManager.getLogger(this.getClass());
private static final Mono<Send<LLRange>> RANGE_ALL = Mono.fromCallable(() -> LLRange.all().send());
private static final Mono<LLRange> RANGE_ALL = Mono.fromCallable(LLRange::all);
private TestAllocator allocator;
private TempDb tempDb;
private LLKeyValueDatabase db;
@ -80,11 +81,11 @@ public abstract class TestLLDictionary {
private LLDictionary getDict(UpdateMode updateMode) {
var dict = DbTestUtils.tempDictionary(db, updateMode).blockOptional().orElseThrow();
var key1 = Mono.fromCallable(() -> fromString("test-key-1").send());
var key2 = Mono.fromCallable(() -> fromString("test-key-2").send());
var key3 = Mono.fromCallable(() -> fromString("test-key-3").send());
var key4 = Mono.fromCallable(() -> fromString("test-key-4").send());
var value = Mono.fromCallable(() -> fromString("test-value").send());
var key1 = Mono.fromCallable(() -> fromString("test-key-1"));
var key2 = Mono.fromCallable(() -> fromString("test-key-2"));
var key3 = Mono.fromCallable(() -> fromString("test-key-3"));
var key4 = Mono.fromCallable(() -> fromString("test-key-4"));
var value = Mono.fromCallable(() -> fromString("test-value"));
dict.put(key1, value, LLDictionaryResultType.VOID).block();
dict.put(key2, value, LLDictionaryResultType.VOID).block();
dict.put(key3, value, LLDictionaryResultType.VOID).block();
@ -107,8 +108,8 @@ public abstract class TestLLDictionary {
}
}
private String toString(Send<Buffer> b) {
try (var bb = b.receive()) {
private String toString(Buffer bb) {
try (bb) {
byte[] data = new byte[bb.readableBytes()];
bb.copyInto(bb.readerOffset(), data, 0, data.length);
return new String(data, StandardCharsets.UTF_8);
@ -153,8 +154,8 @@ public abstract class TestLLDictionary {
@MethodSource("provideArguments")
public void testGet(UpdateMode updateMode) {
var dict = getDict(updateMode);
var keyEx = Mono.fromCallable(() -> fromString("test-key-1").send());
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent").send());
var keyEx = Mono.fromCallable(() -> fromString("test-key-1"));
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent"));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx).map(this::toString)));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx).map(this::toString)));
Assertions.assertEquals("test-value", run(dict.get(null, keyEx).map(this::toString)));
@ -167,12 +168,12 @@ public abstract class TestLLDictionary {
@MethodSource("providePutArguments")
public void testPutExisting(UpdateMode updateMode, LLDictionaryResultType resultType) {
var dict = getDict(updateMode);
var keyEx = Mono.fromCallable(() -> fromString("test-key-1").send());
var value = Mono.fromCallable(() -> fromString("test-value").send());
var keyEx = Mono.fromCallable(() -> fromString("test-key-1"));
var value = Mono.fromCallable(() -> fromString("test-value"));
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
runVoid(dict.put(keyEx, value, resultType).then().doOnDiscard(Send.class, Send::close));
runVoid(dict.put(keyEx, value, resultType).then().doOnDiscard(Resource.class, Resource::close));
var afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
Assertions.assertEquals(0, afterSize - beforeSize);
@ -182,14 +183,14 @@ public abstract class TestLLDictionary {
@MethodSource("providePutArguments")
public void testPutNew(UpdateMode updateMode, LLDictionaryResultType resultType) {
var dict = getDict(updateMode);
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent").send());
var value = Mono.fromCallable(() -> fromString("test-value").send());
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent"));
var value = Mono.fromCallable(() -> fromString("test-value"));
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
runVoid(dict.put(keyNonEx, value, resultType).then().doOnDiscard(Send.class, Send::close));
runVoid(dict.put(keyNonEx, value, resultType).then().doOnDiscard(Resource.class, Resource::close));
var afterSize = run(dict.sizeRange(null, Mono.fromCallable(() -> LLRange.all().send()), false));
var afterSize = run(dict.sizeRange(null, Mono.fromCallable(LLRange::all), false));
Assertions.assertEquals(1, afterSize - beforeSize);
Assertions.assertTrue(run(dict.getRangeKeys(null, RANGE_ALL, false, false).map(this::toString).collectList()).contains("test-nonexistent"));
@ -207,21 +208,21 @@ public abstract class TestLLDictionary {
@MethodSource("provideUpdateArguments")
public void testUpdateExisting(UpdateMode updateMode, UpdateReturnMode updateReturnMode) {
var dict = getDict(updateMode);
var keyEx = Mono.fromCallable(() -> fromString("test-key-1").send());
var keyEx = Mono.fromCallable(() -> fromString("test-key-1"));
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
long afterSize;
runVoid(updateMode == UpdateMode.DISALLOW,
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Send::close).then()
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).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).doOnNext(Send::close).then()
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).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).doOnNext(Send::close).then()
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).then()
);
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
assertEquals(0, afterSize - beforeSize);
@ -232,21 +233,21 @@ public abstract class TestLLDictionary {
public void testUpdateNew(UpdateMode updateMode, UpdateReturnMode updateReturnMode) {
int expected = updateMode == UpdateMode.DISALLOW ? 0 : 1;
var dict = getDict(updateMode);
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent").send());
var keyNonEx = Mono.fromCallable(() -> fromString("test-nonexistent"));
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
long afterSize;
runVoid(updateMode == UpdateMode.DISALLOW,
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Send::close).then()
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).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).doOnNext(Send::close).then()
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).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).doOnNext(Send::close).then()
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Resource::close).then()
);
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
assertEquals(expected, afterSize - beforeSize);

View File

@ -86,12 +86,16 @@ public abstract class TestLLDictionaryLeaks {
return dict;
}
private Send<Buffer> fromString(String s) {
private Buffer fromString(String s) {
var sb = s.getBytes(StandardCharsets.UTF_8);
try (var b = db.getAllocator().allocate(sb.length)) {
var b = db.getAllocator().allocate(sb.length);
try {
b.writeBytes(sb);
assert b.readableBytes() == sb.length;
return b.send();
return b;
} catch (Throwable ex) {
b.close();
throw ex;
}
}
@ -143,7 +147,7 @@ public abstract class TestLLDictionaryLeaks {
var dict = getDict(updateMode);
var key = Mono.fromCallable(() -> fromString("test-key"));
var value = Mono.fromCallable(() -> fromString("test-value"));
runVoid(dict.put(key, value, resultType).then().doOnDiscard(Send.class, Send::close));
runVoid(dict.put(key, value, resultType).then().doOnDiscard(Buffer.class, Buffer::close));
}
@ParameterizedTest
@ -170,7 +174,8 @@ public abstract class TestLLDictionaryLeaks {
}
private Buffer pass(@Nullable Buffer old) {
return old;
if (old == null) return null;
return old.copy();
}
@ParameterizedTest
@ -201,6 +206,6 @@ public abstract class TestLLDictionaryLeaks {
public void testRemove(UpdateMode updateMode, LLDictionaryResultType resultType) {
var dict = getDict(updateMode);
var key = Mono.fromCallable(() -> fromString("test-key"));
runVoid(dict.remove(key, resultType).then().doOnDiscard(Send.class, Send::close));
runVoid(dict.remove(key, resultType).then().doOnDiscard(Buffer.class, Buffer::close));
}
}