(unfinished) Netty 5 refactoring
This commit is contained in:
parent
0faef5316e
commit
013d26387d
@ -82,8 +82,8 @@ public class LLUtils {
|
|||||||
return bool ? RESPONSE_TRUE : RESPONSE_FALSE;
|
return bool ? RESPONSE_TRUE : RESPONSE_FALSE;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Buffer booleanToResponseByteBuffer(BufferAllocator alloc, boolean bool) {
|
public static Send<Buffer> booleanToResponseByteBuffer(BufferAllocator alloc, boolean bool) {
|
||||||
return alloc.allocate(1).writeByte(bool ? (byte) 1 : 0);
|
return alloc.allocate(1).writeByte(bool ? (byte) 1 : 0).send();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@ -349,18 +349,6 @@ public class LLUtils {
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
|
||||||
public static Buffer toDirectCopy(Buffer buffer) {
|
|
||||||
try {
|
|
||||||
Buffer directCopyBuf = buffer.alloc().buffer(buffer.capacity(), buffer.maxCapacity());
|
|
||||||
directCopyBuf.writeBytes(buffer, 0, buffer.writerIndex());
|
|
||||||
return directCopyBuf;
|
|
||||||
} finally {
|
|
||||||
buffer.release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
*/
|
|
||||||
|
|
||||||
public static Buffer fromByteArray(BufferAllocator alloc, byte[] array) {
|
public static Buffer fromByteArray(BufferAllocator alloc, byte[] array) {
|
||||||
Buffer result = alloc.allocate(array.length);
|
Buffer result = alloc.allocate(array.length);
|
||||||
result.writeBytes(array);
|
result.writeBytes(array);
|
||||||
@ -378,7 +366,6 @@ public class LLUtils {
|
|||||||
|
|
||||||
public static Send<Buffer> compositeBuffer(BufferAllocator alloc, Send<Buffer> buffer) {
|
public static Send<Buffer> compositeBuffer(BufferAllocator alloc, Send<Buffer> buffer) {
|
||||||
try (var composite = buffer.receive().compact()) {
|
try (var composite = buffer.receive().compact()) {
|
||||||
assert composite.countReadableComponents() == 1 || composite.countReadableComponents() == 0;
|
|
||||||
return composite.send();
|
return composite.send();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -387,7 +374,6 @@ public class LLUtils {
|
|||||||
try (buffer1) {
|
try (buffer1) {
|
||||||
try (buffer2) {
|
try (buffer2) {
|
||||||
try (var composite = CompositeBuffer.compose(alloc, buffer1, buffer2).compact()) {
|
try (var composite = CompositeBuffer.compose(alloc, buffer1, buffer2).compact()) {
|
||||||
assert composite.countReadableComponents() == 1 || composite.countReadableComponents() == 0;
|
|
||||||
return composite.send();
|
return composite.send();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -399,7 +385,6 @@ public class LLUtils {
|
|||||||
try (buffer2) {
|
try (buffer2) {
|
||||||
try (buffer3) {
|
try (buffer3) {
|
||||||
try (var composite = CompositeBuffer.compose(alloc, buffer1, buffer2, buffer3).compact()) {
|
try (var composite = CompositeBuffer.compose(alloc, buffer1, buffer2, buffer3).compact()) {
|
||||||
assert composite.countReadableComponents() == 1 || composite.countReadableComponents() == 0;
|
|
||||||
return composite.send();
|
return composite.send();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -407,6 +392,7 @@ public class LLUtils {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SafeVarargs
|
||||||
public static Send<Buffer> compositeBuffer(BufferAllocator alloc, Send<Buffer>... buffers) {
|
public static Send<Buffer> compositeBuffer(BufferAllocator alloc, Send<Buffer>... buffers) {
|
||||||
try {
|
try {
|
||||||
return switch (buffers.length) {
|
return switch (buffers.length) {
|
||||||
@ -416,7 +402,6 @@ public class LLUtils {
|
|||||||
case 3 -> compositeBuffer(alloc, buffers[0], buffers[1], buffers[2]);
|
case 3 -> compositeBuffer(alloc, buffers[0], buffers[1], buffers[2]);
|
||||||
default -> {
|
default -> {
|
||||||
try (var composite = CompositeBuffer.compose(alloc, buffers).compact()) {
|
try (var composite = CompositeBuffer.compose(alloc, buffers).compact()) {
|
||||||
assert composite.countReadableComponents() == 1 || composite.countReadableComponents() == 0;
|
|
||||||
yield composite.send();
|
yield composite.send();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -738,7 +723,10 @@ public class LLUtils {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static boolean isDirect(Buffer key) {
|
public static boolean isDirect(Buffer key) {
|
||||||
if (key.countReadableComponents() == 1) {
|
var readableComponents = key.countReadableComponents();
|
||||||
|
if (readableComponents == 0) {
|
||||||
|
return true;
|
||||||
|
} else if (readableComponents == 1) {
|
||||||
return key.forEachReadable(0, (index, component) -> component.readableBuffer().isDirect()) >= 0;
|
return key.forEachReadable(0, (index, component) -> component.readableBuffer().isDirect()) >= 0;
|
||||||
} else {
|
} else {
|
||||||
return false;
|
return false;
|
||||||
|
@ -44,7 +44,9 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> implem
|
|||||||
private static Send<Buffer> incrementPrefix(BufferAllocator alloc, Send<Buffer> originalKeySend, int prefixLength) {
|
private static Send<Buffer> incrementPrefix(BufferAllocator alloc, Send<Buffer> originalKeySend, int prefixLength) {
|
||||||
try (var originalKey = originalKeySend.receive()) {
|
try (var originalKey = originalKeySend.receive()) {
|
||||||
assert originalKey.readableBytes() >= prefixLength;
|
assert originalKey.readableBytes() >= prefixLength;
|
||||||
try (Buffer copiedBuf = alloc.allocate(originalKey.writerOffset())) {
|
var originalKeyStartOffset = originalKey.readerOffset();
|
||||||
|
var originalKeyLength = originalKey.readableBytes();
|
||||||
|
try (Buffer copiedBuf = alloc.allocate(originalKey.readableBytes())) {
|
||||||
boolean overflowed = true;
|
boolean overflowed = true;
|
||||||
final int ff = 0xFF;
|
final int ff = 0xFF;
|
||||||
int writtenBytes = 0;
|
int writtenBytes = 0;
|
||||||
@ -67,17 +69,19 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> implem
|
|||||||
originalKey.copyInto(0, copiedBuf, 0, (prefixLength - writtenBytes));
|
originalKey.copyInto(0, copiedBuf, 0, (prefixLength - writtenBytes));
|
||||||
}
|
}
|
||||||
|
|
||||||
copiedBuf.writerOffset(copiedBuf.capacity());
|
copiedBuf.writerOffset(originalKeyLength);
|
||||||
|
|
||||||
if (originalKey.writerOffset() - prefixLength > 0) {
|
if (originalKeyLength - prefixLength > 0) {
|
||||||
originalKey.copyInto(prefixLength, copiedBuf, prefixLength, originalKey.writerOffset() - prefixLength);
|
originalKey.copyInto(prefixLength, copiedBuf, prefixLength, originalKeyLength - prefixLength);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (overflowed) {
|
if (overflowed) {
|
||||||
for (int i = 0; i < copiedBuf.writerOffset(); i++) {
|
copiedBuf.ensureWritable(originalKeyLength + 1);
|
||||||
|
copiedBuf.writerOffset(originalKeyLength + 1);
|
||||||
|
for (int i = 0; i < originalKeyLength; i++) {
|
||||||
copiedBuf.setUnsignedByte(i, 0xFF);
|
copiedBuf.setUnsignedByte(i, 0xFF);
|
||||||
}
|
}
|
||||||
copiedBuf.writeByte((byte) 0x00);
|
copiedBuf.setUnsignedByte(originalKeyLength, (byte) 0x00);
|
||||||
}
|
}
|
||||||
return copiedBuf.send();
|
return copiedBuf.send();
|
||||||
}
|
}
|
||||||
@ -118,8 +122,8 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> implem
|
|||||||
for (int i = 0; i < suffixLength + extLength; i++) {
|
for (int i = 0; i < suffixLength + extLength; i++) {
|
||||||
zeroSuffixAndExt.writeByte((byte) 0x0);
|
zeroSuffixAndExt.writeByte((byte) 0x0);
|
||||||
}
|
}
|
||||||
try (Send<Buffer> result = LLUtils.compositeBuffer(alloc, prefixKey.send(), zeroSuffixAndExt.send())) {
|
try (Buffer result = LLUtils.compositeBuffer(alloc, prefixKey.send(), zeroSuffixAndExt.send()).receive()) {
|
||||||
return result;
|
return result.send();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -214,31 +218,32 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> implem
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected DatabaseMapDictionaryDeep(LLDictionary dictionary,
|
protected DatabaseMapDictionaryDeep(LLDictionary dictionary,
|
||||||
Send<Buffer> prefixKey,
|
Send<Buffer> prefixKeyToReceive,
|
||||||
SerializerFixedBinaryLength<T, Send<Buffer>> keySuffixSerializer,
|
SerializerFixedBinaryLength<T, Send<Buffer>> keySuffixSerializer,
|
||||||
SubStageGetter<U, US> subStageGetter,
|
SubStageGetter<U, US> subStageGetter,
|
||||||
int keyExtLength) {
|
int keyExtLength) {
|
||||||
|
try (var prefixKey = prefixKeyToReceive.receive()) {
|
||||||
this.dictionary = dictionary;
|
this.dictionary = dictionary;
|
||||||
this.alloc = dictionary.getAllocator();
|
this.alloc = dictionary.getAllocator();
|
||||||
this.subStageGetter = subStageGetter;
|
this.subStageGetter = subStageGetter;
|
||||||
this.keySuffixSerializer = keySuffixSerializer;
|
this.keySuffixSerializer = keySuffixSerializer;
|
||||||
this.keyPrefix = prefixKey.receive();
|
this.keyPrefix = prefixKey.copy();
|
||||||
assert keyPrefix.isAccessible();
|
assert keyPrefix.isAccessible();
|
||||||
this.keyPrefixLength = keyPrefix.readableBytes();
|
this.keyPrefixLength = keyPrefix.readableBytes();
|
||||||
this.keySuffixLength = keySuffixSerializer.getSerializedBinaryLength();
|
this.keySuffixLength = keySuffixSerializer.getSerializedBinaryLength();
|
||||||
this.keyExtLength = keyExtLength;
|
this.keyExtLength = keyExtLength;
|
||||||
try (Buffer firstKey = firstRangeKey(alloc,
|
try (Buffer firstKey = firstRangeKey(alloc,
|
||||||
keyPrefix.copy().send(),
|
prefixKey.copy().send(),
|
||||||
keyPrefixLength,
|
keyPrefixLength,
|
||||||
keySuffixLength,
|
keySuffixLength,
|
||||||
keyExtLength
|
keyExtLength
|
||||||
).receive()) {
|
).receive().compact()) {
|
||||||
try (Buffer nextRangeKey = nextRangeKey(alloc,
|
try (Buffer nextRangeKey = nextRangeKey(alloc,
|
||||||
keyPrefix.copy().send(),
|
prefixKey.copy().send(),
|
||||||
keyPrefixLength,
|
keyPrefixLength,
|
||||||
keySuffixLength,
|
keySuffixLength,
|
||||||
keyExtLength
|
keyExtLength
|
||||||
).receive()) {
|
).receive().compact()) {
|
||||||
assert keyPrefix.isAccessible();
|
assert keyPrefix.isAccessible();
|
||||||
assert keyPrefixLength == 0 || !LLUtils.equals(firstKey, nextRangeKey);
|
assert keyPrefixLength == 0 || !LLUtils.equals(firstKey, nextRangeKey);
|
||||||
this.range = keyPrefixLength == 0 ? LLRange.all() : LLRange.of(firstKey.send(), nextRangeKey.send());
|
this.range = keyPrefixLength == 0 ? LLRange.all() : LLRange.of(firstKey.send(), nextRangeKey.send());
|
||||||
@ -247,6 +252,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> implem
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
protected boolean suffixKeyConsistency(int keySuffixLength) {
|
protected boolean suffixKeyConsistency(int keySuffixLength) {
|
||||||
|
@ -402,7 +402,8 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
@Nullable WriteOptions writeOptions,
|
@Nullable WriteOptions writeOptions,
|
||||||
Send<Buffer> keyToReceive,
|
Send<Buffer> keyToReceive,
|
||||||
Send<Buffer> valueToReceive) throws RocksDBException {
|
Send<Buffer> valueToReceive) throws RocksDBException {
|
||||||
try (WriteOptions validWriteOptions = Objects.requireNonNullElse(writeOptions, EMPTY_WRITE_OPTIONS)) {
|
WriteOptions validWriteOptions = Objects.requireNonNullElse(writeOptions, EMPTY_WRITE_OPTIONS);
|
||||||
|
try {
|
||||||
try (var key = keyToReceive.receive()) {
|
try (var key = keyToReceive.receive()) {
|
||||||
try (var value = valueToReceive.receive()) {
|
try (var value = valueToReceive.receive()) {
|
||||||
if (databaseOptions.allowNettyDirect()) {
|
if (databaseOptions.allowNettyDirect()) {
|
||||||
@ -424,6 +425,10 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
if (writeOptions != null && !(writeOptions instanceof UnreleasableWriteOptions)) {
|
||||||
|
writeOptions.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -523,7 +528,8 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
int size = RocksDB.NOT_FOUND;
|
int size = RocksDB.NOT_FOUND;
|
||||||
byte[] keyBytes = LLUtils.toArray(key);
|
byte[] keyBytes = LLUtils.toArray(key);
|
||||||
Holder<byte[]> data = new Holder<>();
|
Holder<byte[]> data = new Holder<>();
|
||||||
try (var unmodifiableReadOpts = resolveSnapshot(snapshot)) {
|
var unmodifiableReadOpts = resolveSnapshot(snapshot);
|
||||||
|
try {
|
||||||
if (db.keyMayExist(cfh, unmodifiableReadOpts, keyBytes, data)) {
|
if (db.keyMayExist(cfh, unmodifiableReadOpts, keyBytes, data)) {
|
||||||
if (data.getValue() != null) {
|
if (data.getValue() != null) {
|
||||||
size = data.getValue().length;
|
size = data.getValue().length;
|
||||||
@ -531,6 +537,10 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
size = db.get(cfh, unmodifiableReadOpts, keyBytes, NO_DATA);
|
size = db.get(cfh, unmodifiableReadOpts, keyBytes, NO_DATA);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
if (unmodifiableReadOpts != null && !(unmodifiableReadOpts instanceof UnreleasableReadOptions)) {
|
||||||
|
unmodifiableReadOpts.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return size != RocksDB.NOT_FOUND;
|
return size != RocksDB.NOT_FOUND;
|
||||||
} finally {
|
} finally {
|
||||||
@ -912,17 +922,16 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private Mono<Send<Buffer>> getPreviousData(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
|
private Mono<Send<Buffer>> getPreviousData(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
|
||||||
return Mono
|
|
||||||
.usingWhen(keyMono,
|
|
||||||
keySend -> {
|
|
||||||
try (var key = keySend.receive()) {
|
|
||||||
return switch (resultType) {
|
return switch (resultType) {
|
||||||
case PREVIOUS_VALUE_EXISTENCE -> this
|
case PREVIOUS_VALUE_EXISTENCE -> this
|
||||||
.containsKey(null, keyMono)
|
.containsKey(null, keyMono)
|
||||||
.single()
|
.single()
|
||||||
.map((Boolean bool) -> LLUtils.booleanToResponseByteBuffer(alloc, bool).send());
|
.map((Boolean bool) -> LLUtils.booleanToResponseByteBuffer(alloc, bool));
|
||||||
case PREVIOUS_VALUE -> Mono
|
case PREVIOUS_VALUE -> Mono.usingWhen(
|
||||||
.fromCallable(() -> {
|
keyMono,
|
||||||
|
keySend -> this
|
||||||
|
.runOnDb(() -> {
|
||||||
|
try (var key = keySend.receive()) {
|
||||||
StampedLock lock;
|
StampedLock lock;
|
||||||
long stamp;
|
long stamp;
|
||||||
if (updateMode == UpdateMode.ALLOW) {
|
if (updateMode == UpdateMode.ALLOW) {
|
||||||
@ -952,16 +961,13 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
lock.unlockRead(stamp);
|
lock.unlockRead(stamp);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
})
|
})
|
||||||
.onErrorMap(cause -> new IOException("Failed to read " + LLUtils.toStringSafe(key), cause))
|
.onErrorMap(cause -> new IOException("Failed to read ", cause)),
|
||||||
.subscribeOn(dbScheduler);
|
keySend -> Mono.fromRunnable(keySend::close));
|
||||||
case VOID -> Mono.empty();
|
case VOID -> Mono.empty();
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
},
|
|
||||||
keySend -> Mono.fromRunnable(keySend::close)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
|
public <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
|
||||||
|
@ -2,9 +2,11 @@ package it.cavallium.dbengine.database.memory;
|
|||||||
|
|
||||||
import io.netty.buffer.api.Buffer;
|
import io.netty.buffer.api.Buffer;
|
||||||
import io.netty.buffer.api.BufferAllocator;
|
import io.netty.buffer.api.BufferAllocator;
|
||||||
|
import io.netty.buffer.api.Send;
|
||||||
import it.cavallium.dbengine.client.BadBlock;
|
import it.cavallium.dbengine.client.BadBlock;
|
||||||
import it.cavallium.dbengine.database.Delta;
|
import it.cavallium.dbengine.database.Delta;
|
||||||
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
|
import it.cavallium.dbengine.database.ExtraKeyOperationResult;
|
||||||
|
import it.cavallium.dbengine.database.LLDelta;
|
||||||
import it.cavallium.dbengine.database.LLDictionary;
|
import it.cavallium.dbengine.database.LLDictionary;
|
||||||
import it.cavallium.dbengine.database.LLDictionaryResultType;
|
import it.cavallium.dbengine.database.LLDictionaryResultType;
|
||||||
import it.cavallium.dbengine.database.LLEntry;
|
import it.cavallium.dbengine.database.LLEntry;
|
||||||
@ -81,7 +83,7 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Mono<Buffer> transformResult(Mono<ByteList> result, LLDictionaryResultType resultType) {
|
private Mono<Send<Buffer>> transformResult(Mono<ByteList> result, LLDictionaryResultType resultType) {
|
||||||
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE) {
|
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE) {
|
||||||
// Don't retain the result because it has been removed from the skip list
|
// Don't retain the result because it has been removed from the skip list
|
||||||
return result.map(this::kk);
|
return result.map(this::kk);
|
||||||
@ -89,23 +91,25 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
return result
|
return result
|
||||||
.map(prev -> true)
|
.map(prev -> true)
|
||||||
.defaultIfEmpty(false)
|
.defaultIfEmpty(false)
|
||||||
.map(LLUtils::booleanToResponseByteBuffer);
|
.map((Boolean bool) -> LLUtils.booleanToResponseByteBuffer(allocator, bool));
|
||||||
} else {
|
} else {
|
||||||
return result.then(Mono.empty());
|
return result.then(Mono.empty());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private ByteList k(Buffer buf) {
|
private ByteList k(Send<Buffer> buf) {
|
||||||
return new BinaryLexicographicList(LLUtils.toArray(buf));
|
return new BinaryLexicographicList(LLUtils.toArray(buf.receive()));
|
||||||
}
|
}
|
||||||
|
|
||||||
private Buffer kk(ByteList bytesList) {
|
private Send<Buffer> kk(ByteList bytesList) {
|
||||||
var buffer = getAllocator().buffer(bytesList.size());
|
try (var buffer = getAllocator().allocate(bytesList.size())) {
|
||||||
buffer.writeBytes(bytesList.toByteArray());
|
buffer.writeBytes(bytesList.toByteArray());
|
||||||
return buffer;
|
return buffer.send();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<ByteList, ByteList> mapSlice(LLSnapshot snapshot, LLRange range) {
|
private Map<ByteList, ByteList> mapSlice(LLSnapshot snapshot, Send<LLRange> rangeToReceive) {
|
||||||
|
try (var range = rangeToReceive.receive()) {
|
||||||
if (range.isAll()) {
|
if (range.isAll()) {
|
||||||
return snapshots.get(resolveSnapshot(snapshot));
|
return snapshots.get(resolveSnapshot(snapshot));
|
||||||
} else if (range.isSingle()) {
|
} else if (range.isSingle()) {
|
||||||
@ -137,29 +141,30 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
.headMap(k(range.getMax()), false);
|
.headMap(k(range.getMax()), false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Buffer> get(@Nullable LLSnapshot snapshot, Mono<Buffer> keyMono, boolean existsAlmostCertainly) {
|
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono, boolean existsAlmostCertainly) {
|
||||||
return Mono.usingWhen(keyMono,
|
return Mono.usingWhen(keyMono,
|
||||||
key -> Mono
|
key -> Mono
|
||||||
.fromCallable(() -> snapshots.get(resolveSnapshot(snapshot)).get(k(key)))
|
.fromCallable(() -> snapshots.get(resolveSnapshot(snapshot)).get(k(key)))
|
||||||
.map(this::kk)
|
.map(this::kk)
|
||||||
.onErrorMap(cause -> new IOException("Failed to read " + LLUtils.toStringSafe(key), cause)),
|
.onErrorMap(cause -> new IOException("Failed to read", cause)),
|
||||||
key -> Mono.fromRunnable(key::release)
|
key -> Mono.fromRunnable(key::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Buffer> put(Mono<Buffer> keyMono, Mono<Buffer> valueMono, LLDictionaryResultType resultType) {
|
public Mono<Send<Buffer>> put(Mono<Send<Buffer>> keyMono, Mono<Send<Buffer>> valueMono, LLDictionaryResultType resultType) {
|
||||||
return Mono.usingWhen(keyMono,
|
return Mono.usingWhen(keyMono,
|
||||||
key -> Mono.usingWhen(valueMono,
|
key -> Mono.usingWhen(valueMono,
|
||||||
value -> Mono
|
value -> Mono
|
||||||
.fromCallable(() -> mainDb.put(k(key), k(value)))
|
.fromCallable(() -> mainDb.put(k(key), k(value)))
|
||||||
.transform(result -> this.transformResult(result, resultType))
|
.transform(result -> this.transformResult(result, resultType))
|
||||||
.onErrorMap(cause -> new IOException("Failed to read " + LLUtils.toStringSafe(key), cause)),
|
.onErrorMap(cause -> new IOException("Failed to read", cause)),
|
||||||
value -> Mono.fromRunnable(value::release)
|
value -> Mono.fromRunnable(value::close)
|
||||||
),
|
),
|
||||||
key -> Mono.fromRunnable(key::release)
|
key -> Mono.fromRunnable(key::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -169,17 +174,17 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Delta<Buffer>> updateAndGetDelta(Mono<Buffer> keyMono,
|
public Mono<LLDelta> updateAndGetDelta(Mono<Send<Buffer>> keyMono,
|
||||||
SerializationFunction<@Nullable Buffer, @Nullable Buffer> updater,
|
SerializationFunction<@Nullable Send<Buffer>, @Nullable Send<Buffer>> updater,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return Mono.usingWhen(keyMono,
|
return Mono.usingWhen(keyMono,
|
||||||
key -> Mono.fromCallable(() -> {
|
key -> Mono.fromCallable(() -> {
|
||||||
AtomicReference<Buffer> oldRef = new AtomicReference<>(null);
|
AtomicReference<Send<Buffer>> oldRef = new AtomicReference<>(null);
|
||||||
var newValue = mainDb.compute(k(key), (_unused, old) -> {
|
var newValue = mainDb.compute(k(key), (_unused, old) -> {
|
||||||
if (old != null) {
|
if (old != null) {
|
||||||
oldRef.set(kk(old));
|
oldRef.set(kk(old));
|
||||||
}
|
}
|
||||||
Buffer v = null;
|
Send<Buffer> v = null;
|
||||||
try {
|
try {
|
||||||
v = updater.apply(old != null ? kk(old) : null);
|
v = updater.apply(old != null ? kk(old) : null);
|
||||||
} catch (SerializationException e) {
|
} catch (SerializationException e) {
|
||||||
@ -189,13 +194,13 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
return k(v);
|
return k(v);
|
||||||
} finally {
|
} finally {
|
||||||
if (v != null) {
|
if (v != null) {
|
||||||
v.release();
|
v.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
return new Delta<>(oldRef.get(), kk(newValue));
|
return LLDelta.of(oldRef.get(), kk(newValue));
|
||||||
}),
|
}),
|
||||||
key -> Mono.fromRunnable(key::release)
|
key -> Mono.fromRunnable(key::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -205,183 +210,213 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Buffer> remove(Mono<Buffer> keyMono, LLDictionaryResultType resultType) {
|
public Mono<Send<Buffer>> remove(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
|
||||||
return Mono.usingWhen(keyMono,
|
return Mono.usingWhen(keyMono,
|
||||||
key -> Mono
|
key -> Mono
|
||||||
.fromCallable(() -> mainDb.remove(k(key)))
|
.fromCallable(() -> mainDb.remove(k(key)))
|
||||||
// Don't retain the result because it has been removed from the skip list
|
// Don't retain the result because it has been removed from the skip list
|
||||||
.mapNotNull(bytesList -> switch (resultType) {
|
.mapNotNull(bytesList -> switch (resultType) {
|
||||||
case VOID -> null;
|
case VOID -> null;
|
||||||
case PREVIOUS_VALUE_EXISTENCE -> LLUtils.booleanToResponseByteBuffer(true);
|
case PREVIOUS_VALUE_EXISTENCE -> LLUtils.booleanToResponseByteBuffer(allocator, true);
|
||||||
case PREVIOUS_VALUE -> kk(bytesList);
|
case PREVIOUS_VALUE -> kk(bytesList);
|
||||||
})
|
})
|
||||||
.switchIfEmpty(Mono.defer(() -> {
|
.switchIfEmpty(Mono.defer(() -> {
|
||||||
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) {
|
if (resultType == LLDictionaryResultType.PREVIOUS_VALUE_EXISTENCE) {
|
||||||
return Mono.fromCallable(() -> LLUtils.booleanToResponseByteBuffer(false));
|
return Mono.fromCallable(() -> LLUtils.booleanToResponseByteBuffer(allocator, false));
|
||||||
} else {
|
} else {
|
||||||
return Mono.empty();
|
return Mono.empty();
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
.onErrorMap(cause -> new IOException("Failed to read " + LLUtils.toStringSafe(key), cause)),
|
.onErrorMap(cause -> new IOException("Failed to read", cause)),
|
||||||
key -> Mono.fromRunnable(key::release)
|
key -> Mono.fromRunnable(key::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <K> Flux<Tuple3<K, Buffer, Optional<Buffer>>> getMulti(@Nullable LLSnapshot snapshot,
|
public <K> Flux<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> getMulti(@Nullable LLSnapshot snapshot,
|
||||||
Flux<Tuple2<K, Buffer>> keys,
|
Flux<Tuple2<K, Send<Buffer>>> keys,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return keys
|
return keys
|
||||||
.flatMapSequential(key -> {
|
.map(key -> {
|
||||||
try {
|
try (var t2 = key.getT2().receive()) {
|
||||||
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(key.getT2()));
|
ByteList v = snapshots.get(resolveSnapshot(snapshot)).get(k(t2.copy().send()));
|
||||||
if (v != null) {
|
if (v != null) {
|
||||||
return Flux.just(Tuples.of(key.getT1(), key.getT2().retain(), Optional.of(kk(v))));
|
return Tuples.of(key.getT1(), t2.send(), Optional.of(kk(v)));
|
||||||
} else {
|
} else {
|
||||||
return Flux.just(Tuples.of(key.getT1(), key.getT2().retain(), Optional.empty()));
|
return Tuples.of(key.getT1(), t2.send(), Optional.empty());
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
key.getT2().release();
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<LLEntry> putMulti(Flux<LLEntry> entries, boolean getOldValues) {
|
public Flux<Send<LLEntry>> putMulti(Flux<Send<LLEntry>> entries, boolean getOldValues) {
|
||||||
return entries
|
return entries.handle((entryToReceive, sink) -> {
|
||||||
.handle((entry, sink) -> {
|
try (var entry = entryToReceive.receive()) {
|
||||||
var key = entry.getKey();
|
try (var key = entry.getKey().receive()) {
|
||||||
var val = entry.getValue();
|
try (var val = entry.getValue().receive()) {
|
||||||
try {
|
var v = mainDb.put(k(key.copy().send()), k(val.send()));
|
||||||
var v = mainDb.put(k(key), k(val));
|
|
||||||
if (v == null || !getOldValues) {
|
if (v == null || !getOldValues) {
|
||||||
sink.complete();
|
sink.complete();
|
||||||
} else {
|
} else {
|
||||||
sink.next(new LLEntry(key.retain(), kk(v)));
|
sink.next(LLEntry.of(key.send(), kk(v)).send());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
key.release();
|
|
||||||
val.release();
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <X> Flux<ExtraKeyOperationResult<Buffer, X>> updateMulti(Flux<Tuple2<Buffer, X>> entries,
|
public <X> Flux<ExtraKeyOperationResult<Send<Buffer>, X>> updateMulti(Flux<Tuple2<Send<Buffer>, X>> entries,
|
||||||
BiSerializationFunction<Buffer, X, Buffer> updateFunction) {
|
BiSerializationFunction<Send<Buffer>, X, Send<Buffer>> updateFunction) {
|
||||||
return Flux.error(new UnsupportedOperationException("Not implemented"));
|
return Flux.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<LLEntry> getRange(@Nullable LLSnapshot snapshot,
|
public Flux<Send<LLEntry>> getRange(@Nullable LLSnapshot snapshot,
|
||||||
Mono<LLRange> rangeMono,
|
Mono<Send<LLRange>> rangeMono,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return Flux.usingWhen(rangeMono,
|
return Flux.usingWhen(rangeMono, rangeToReceive -> {
|
||||||
range -> {
|
try (var range = rangeToReceive.receive()) {
|
||||||
if (range.isSingle()) {
|
if (range.isSingle()) {
|
||||||
|
var singleToReceive = range.getSingle();
|
||||||
return Mono.fromCallable(() -> {
|
return Mono.fromCallable(() -> {
|
||||||
var element = snapshots.get(resolveSnapshot(snapshot))
|
try (var single = singleToReceive.receive()) {
|
||||||
.get(k(range.getSingle()));
|
var element = snapshots.get(resolveSnapshot(snapshot)).get(k(single.copy().send()));
|
||||||
return new LLEntry(range.getSingle().retain(), kk(element));
|
return LLEntry.of(single.send(), kk(element)).send();
|
||||||
|
}
|
||||||
}).flux();
|
}).flux();
|
||||||
} else {
|
} else {
|
||||||
|
var rangeToReceive2 = range.send();
|
||||||
return Mono
|
return Mono
|
||||||
.fromCallable(() -> mapSlice(snapshot, range))
|
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
|
||||||
.flatMapMany(map -> Flux.fromIterable(map.entrySet()))
|
.flatMapMany(map -> Flux.fromIterable(map.entrySet()))
|
||||||
.map(entry -> new LLEntry(kk(entry.getKey()), kk(entry.getValue())));
|
.map(entry -> LLEntry.of(kk(entry.getKey()), kk(entry.getValue())).send());
|
||||||
}
|
}
|
||||||
},
|
}
|
||||||
range -> Mono.fromRunnable(range::release)
|
}, range -> Mono.fromRunnable(range::close));
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<List<LLEntry>> getRangeGrouped(@Nullable LLSnapshot snapshot,
|
public Flux<List<Send<LLEntry>>> getRangeGrouped(@Nullable LLSnapshot snapshot,
|
||||||
Mono<LLRange> rangeMono,
|
Mono<Send<LLRange>> rangeMono,
|
||||||
int prefixLength,
|
int prefixLength,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return Flux.error(new UnsupportedOperationException("Not implemented"));
|
return Flux.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<Buffer> getRangeKeys(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
|
public Flux<Send<Buffer>> getRangeKeys(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
||||||
return Flux.usingWhen(rangeMono,
|
return Flux.usingWhen(rangeMono,
|
||||||
range -> {
|
rangeToReceive -> {
|
||||||
|
try (var range = rangeToReceive.receive()) {
|
||||||
if (range.isSingle()) {
|
if (range.isSingle()) {
|
||||||
|
var singleToReceive = range.getSingle();
|
||||||
return Mono.fromCallable(() -> {
|
return Mono.fromCallable(() -> {
|
||||||
var contains = snapshots.get(resolveSnapshot(snapshot))
|
try (var single = singleToReceive.receive()) {
|
||||||
.containsKey(k(range.getSingle()));
|
var contains = snapshots.get(resolveSnapshot(snapshot)).containsKey(k(single.copy().send()));
|
||||||
return contains ? range.getSingle().retain() : null;
|
return contains ? single.send() : null;
|
||||||
|
}
|
||||||
}).flux();
|
}).flux();
|
||||||
} else {
|
} else {
|
||||||
|
var rangeToReceive2 = range.send();
|
||||||
return Mono
|
return Mono
|
||||||
.fromCallable(() -> mapSlice(snapshot, range))
|
.fromCallable(() -> mapSlice(snapshot, rangeToReceive2))
|
||||||
.flatMapMany(map -> Flux.fromIterable(map.entrySet()))
|
.flatMapMany(map -> Flux.fromIterable(map.entrySet()))
|
||||||
.map(entry -> kk(entry.getKey()));
|
.map(entry -> kk(entry.getKey()));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
},
|
},
|
||||||
range -> Mono.fromRunnable(range::release)
|
range -> Mono.fromRunnable(range::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static record BufferWithPrefix(Send<Buffer> buffer, Send<Buffer> prefix) {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<List<Buffer>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
|
public Flux<List<Send<Buffer>>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
|
||||||
Mono<LLRange> rangeMono,
|
Mono<Send<LLRange>> rangeMono,
|
||||||
int prefixLength) {
|
int prefixLength) {
|
||||||
return getRangeKeys(snapshot, rangeMono)
|
return getRangeKeys(snapshot, rangeMono)
|
||||||
.bufferUntilChanged(k -> k.slice(k.readerIndex(), prefixLength), LLUtils::equals);
|
.map(bufferToReceive -> {
|
||||||
|
try(var buffer = bufferToReceive.receive()) {
|
||||||
|
try (var bufferPrefix = buffer.copy(buffer.readerOffset(), prefixLength)) {
|
||||||
|
return new BufferWithPrefix(buffer.send(), bufferPrefix.send());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.windowUntilChanged(bufferTuple -> bufferTuple.prefix().receive(), LLUtils::equals)
|
||||||
|
.flatMapSequential(window -> window.map(tuple -> {
|
||||||
|
try (var ignored = tuple.prefix()) {
|
||||||
|
return tuple.buffer();
|
||||||
|
}
|
||||||
|
}).collectList());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<Buffer> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono, int prefixLength) {
|
public Flux<Send<Buffer>> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot,
|
||||||
|
Mono<Send<LLRange>> rangeMono,
|
||||||
|
int prefixLength) {
|
||||||
return getRangeKeys(snapshot, rangeMono)
|
return getRangeKeys(snapshot, rangeMono)
|
||||||
.distinctUntilChanged(k -> k.slice(k.readerIndex(), prefixLength), (a, b) -> {
|
.map(bufferToReceive -> {
|
||||||
|
try(var buffer = bufferToReceive.receive()) {
|
||||||
|
try (var bufferPrefix = buffer.copy(buffer.readerOffset(), prefixLength)) {
|
||||||
|
return new BufferWithPrefix(buffer.send(), bufferPrefix.send());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.distinctUntilChanged(bufferTuple -> bufferTuple.prefix().receive(), (a, b) -> {
|
||||||
if (LLUtils.equals(a, b)) {
|
if (LLUtils.equals(a, b)) {
|
||||||
b.release();
|
b.close();
|
||||||
return true;
|
return true;
|
||||||
} else {
|
} else {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
.map(k -> k.slice(k.readerIndex(), prefixLength))
|
.map(tuple -> {
|
||||||
|
try (var ignored = tuple.prefix()) {
|
||||||
|
return tuple.buffer();
|
||||||
|
}
|
||||||
|
})
|
||||||
.transform(LLUtils::handleDiscard);
|
.transform(LLUtils::handleDiscard);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<BadBlock> badBlocks(Mono<LLRange> rangeMono) {
|
public Flux<BadBlock> badBlocks(Mono<Send<LLRange>> rangeMono) {
|
||||||
return Flux.empty();
|
return Flux.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Void> setRange(Mono<LLRange> rangeMono, Flux<LLEntry> entries) {
|
public Mono<Void> setRange(Mono<Send<LLRange>> rangeMono, Flux<Send<LLEntry>> entries) {
|
||||||
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
|
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
||||||
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono, boolean fast) {
|
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fast) {
|
||||||
return Mono.usingWhen(rangeMono,
|
return Mono.usingWhen(rangeMono,
|
||||||
range -> Mono.fromCallable(() -> (long) mapSlice(snapshot, range).size()),
|
range -> Mono.fromCallable(() -> (long) mapSlice(snapshot, range).size()),
|
||||||
range -> Mono.fromRunnable(range::release)
|
range -> Mono.fromRunnable(range::close)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<LLEntry> getOne(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
|
public Mono<Send<LLEntry>> getOne(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
||||||
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Buffer> getOneKey(@Nullable LLSnapshot snapshot, Mono<LLRange> rangeMono) {
|
public Mono<Send<Buffer>> getOneKey(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
||||||
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<LLEntry> removeOne(Mono<LLRange> rangeMono) {
|
public Mono<Send<LLEntry>> removeOne(Mono<Send<LLRange>> rangeMono) {
|
||||||
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
return Mono.error(new UnsupportedOperationException("Not implemented"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -14,7 +14,7 @@ public class LLMemorySingleton implements LLSingleton {
|
|||||||
|
|
||||||
private final LLMemoryDictionary dict;
|
private final LLMemoryDictionary dict;
|
||||||
private final byte[] singletonName;
|
private final byte[] singletonName;
|
||||||
private final Mono<Buffer> singletonNameBufMono;
|
private final Mono<Send<Buffer>> singletonNameBufMono;
|
||||||
|
|
||||||
public LLMemorySingleton(LLMemoryDictionary dict, byte[] singletonName) {
|
public LLMemorySingleton(LLMemoryDictionary dict, byte[] singletonName) {
|
||||||
this.dict = dict;
|
this.dict = dict;
|
||||||
@ -22,7 +22,8 @@ public class LLMemorySingleton implements LLSingleton {
|
|||||||
this.singletonNameBufMono = Mono.fromCallable(() -> dict
|
this.singletonNameBufMono = Mono.fromCallable(() -> dict
|
||||||
.getAllocator()
|
.getAllocator()
|
||||||
.allocate(singletonName.length)
|
.allocate(singletonName.length)
|
||||||
.writeBytes(singletonName));
|
.writeBytes(singletonName)
|
||||||
|
.send());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -35,8 +36,8 @@ public class LLMemorySingleton implements LLSingleton {
|
|||||||
return dict
|
return dict
|
||||||
.get(snapshot, singletonNameBufMono, false)
|
.get(snapshot, singletonNameBufMono, false)
|
||||||
.map(b -> {
|
.map(b -> {
|
||||||
try (b) {
|
try (var buf = b.receive()) {
|
||||||
return LLUtils.toArray(b);
|
return LLUtils.toArray(buf);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
@ -44,7 +45,7 @@ public class LLMemorySingleton implements LLSingleton {
|
|||||||
@Override
|
@Override
|
||||||
public Mono<Void> set(byte[] value) {
|
public Mono<Void> set(byte[] value) {
|
||||||
var bbKey = singletonNameBufMono;
|
var bbKey = singletonNameBufMono;
|
||||||
var bbVal = Mono.fromCallable(() -> dict.getAllocator().allocate(value.length).writeBytes(value));
|
var bbVal = Mono.fromCallable(() -> dict.getAllocator().allocate(value.length).writeBytes(value).send());
|
||||||
return dict
|
return dict
|
||||||
.put(bbKey, bbVal, LLDictionaryResultType.VOID)
|
.put(bbKey, bbVal, LLDictionaryResultType.VOID)
|
||||||
.then();
|
.then();
|
||||||
|
@ -4,10 +4,14 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||||||
|
|
||||||
import io.netty.buffer.ByteBuf;
|
import io.netty.buffer.ByteBuf;
|
||||||
import io.netty.buffer.ByteBufAllocator;
|
import io.netty.buffer.ByteBufAllocator;
|
||||||
import io.netty.buffer.ByteBufUtil;
|
|
||||||
import io.netty.buffer.PoolArenaMetric;
|
import io.netty.buffer.PoolArenaMetric;
|
||||||
import io.netty.buffer.PooledByteBufAllocator;
|
import io.netty.buffer.PooledByteBufAllocator;
|
||||||
import io.netty.buffer.UnpooledByteBufAllocator;
|
import io.netty.buffer.UnpooledByteBufAllocator;
|
||||||
|
import io.netty.buffer.api.Buffer;
|
||||||
|
import io.netty.buffer.api.MemoryManager;
|
||||||
|
import io.netty.buffer.api.Send;
|
||||||
|
import io.netty.buffer.api.pool.BufferAllocatorMetric;
|
||||||
|
import io.netty.buffer.api.pool.PooledBufferAllocator;
|
||||||
import it.cavallium.dbengine.database.Column;
|
import it.cavallium.dbengine.database.Column;
|
||||||
import it.cavallium.dbengine.database.LLDatabaseConnection;
|
import it.cavallium.dbengine.database.LLDatabaseConnection;
|
||||||
import it.cavallium.dbengine.database.LLDictionary;
|
import it.cavallium.dbengine.database.LLDictionary;
|
||||||
@ -41,55 +45,26 @@ import reactor.core.scheduler.Schedulers;
|
|||||||
|
|
||||||
public class DbTestUtils {
|
public class DbTestUtils {
|
||||||
|
|
||||||
public static record TestAllocator(ByteBufAllocator allocator) {}
|
public static record TestAllocator(PooledBufferAllocator allocator) {}
|
||||||
|
|
||||||
public static TestAllocator newAllocator() {
|
public static TestAllocator newAllocator() {
|
||||||
return new TestAllocator(new PooledByteBufAllocator(false, 1, 0, 4096, 11, 0, 0, true));
|
return new TestAllocator(new PooledBufferAllocator(MemoryManager.instance(), true, 1, 8192, 9, 0, 0, false));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void destroyAllocator(TestAllocator testAllocator) {
|
public static void destroyAllocator(TestAllocator testAllocator) {
|
||||||
|
testAllocator.allocator().close();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final AtomicInteger dbId = new AtomicInteger(0);
|
public static final AtomicInteger dbId = new AtomicInteger(0);
|
||||||
|
|
||||||
@SuppressWarnings("SameParameterValue")
|
@SuppressWarnings("SameParameterValue")
|
||||||
private static int getActiveBuffers(ByteBufAllocator allocator, boolean printStats) {
|
private static long getUsedMemory(PooledBufferAllocator allocator, boolean printStats) {
|
||||||
int directActive = 0, directAlloc = 0, directDealloc = 0;
|
allocator.trimCurrentThreadCache();
|
||||||
if (allocator instanceof PooledByteBufAllocator alloc) {
|
var usedMemory = ((BufferAllocatorMetric) allocator.metric()).usedMemory();
|
||||||
for (PoolArenaMetric arena : alloc.directArenas()) {
|
|
||||||
directActive += arena.numActiveAllocations();
|
|
||||||
directAlloc += arena.numAllocations();
|
|
||||||
directDealloc += arena.numDeallocations();
|
|
||||||
}
|
|
||||||
} else if (allocator instanceof UnpooledByteBufAllocator alloc) {
|
|
||||||
directActive += alloc.metric().usedDirectMemory();
|
|
||||||
} else {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
if (printStats) {
|
if (printStats) {
|
||||||
System.out.println("directActive " + directActive + " directAlloc " + directAlloc + " directDealloc " + directDealloc);
|
System.out.println("usedMemory=" + usedMemory);
|
||||||
}
|
}
|
||||||
return directActive;
|
return usedMemory;
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("SameParameterValue")
|
|
||||||
private static int getActiveHeapBuffers(ByteBufAllocator allocator, boolean printStats) {
|
|
||||||
int heapActive = 0, heapAlloc = 0, heapDealloc = 0;
|
|
||||||
if (allocator instanceof PooledByteBufAllocator alloc) {
|
|
||||||
for (PoolArenaMetric arena : alloc.heapArenas()) {
|
|
||||||
heapActive += arena.numActiveAllocations();
|
|
||||||
heapAlloc += arena.numAllocations();
|
|
||||||
heapDealloc += arena.numDeallocations();
|
|
||||||
}
|
|
||||||
} else if (allocator instanceof UnpooledByteBufAllocator alloc) {
|
|
||||||
heapActive += alloc.metric().usedHeapMemory();
|
|
||||||
} else {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
if (printStats) {
|
|
||||||
System.out.println("heapActive " + heapActive + " heapAlloc " + heapAlloc + " heapDealloc " + heapDealloc);
|
|
||||||
}
|
|
||||||
return heapActive;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <U> Flux<U> tempDb(TestAllocator alloc, Function<LLKeyValueDatabase, Publisher<U>> action) {
|
public static <U> Flux<U> tempDb(TestAllocator alloc, Function<LLKeyValueDatabase, Publisher<U>> action) {
|
||||||
@ -134,10 +109,6 @@ public class DbTestUtils {
|
|||||||
public static Mono<Void> closeTempDb(TempDb tempDb) {
|
public static Mono<Void> closeTempDb(TempDb tempDb) {
|
||||||
return tempDb.db().close().then(tempDb.connection().disconnect()).then(Mono.fromCallable(() -> {
|
return tempDb.db().close().then(tempDb.connection().disconnect()).then(Mono.fromCallable(() -> {
|
||||||
ensureNoLeaks(tempDb.allocator().allocator(), false);
|
ensureNoLeaks(tempDb.allocator().allocator(), false);
|
||||||
if (tempDb.allocator().allocator() instanceof PooledByteBufAllocator pooledByteBufAllocator) {
|
|
||||||
pooledByteBufAllocator.trimCurrentThreadCache();
|
|
||||||
pooledByteBufAllocator.freeThreadLocalCache();
|
|
||||||
}
|
|
||||||
if (Files.exists(tempDb.path())) {
|
if (Files.exists(tempDb.path())) {
|
||||||
Files.walk(tempDb.path()).sorted(Comparator.reverseOrder()).forEach(file -> {
|
Files.walk(tempDb.path()).sorted(Comparator.reverseOrder()).forEach(file -> {
|
||||||
try {
|
try {
|
||||||
@ -151,10 +122,9 @@ public class DbTestUtils {
|
|||||||
}).subscribeOn(Schedulers.boundedElastic())).then();
|
}).subscribeOn(Schedulers.boundedElastic())).then();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void ensureNoLeaks(ByteBufAllocator allocator, boolean printStats) {
|
public static void ensureNoLeaks(PooledBufferAllocator allocator, boolean printStats) {
|
||||||
if (allocator != null) {
|
if (allocator != null) {
|
||||||
assertEquals(0, getActiveBuffers(allocator, printStats));
|
assertEquals(0L, getUsedMemory(allocator, printStats));
|
||||||
assertEquals(0, getActiveHeapBuffers(allocator, printStats));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -195,26 +165,21 @@ public class DbTestUtils {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public @NotNull Short deserialize(@NotNull ByteBuf serialized) {
|
public @NotNull Short deserialize(@NotNull Send<Buffer> serializedToReceive) {
|
||||||
try {
|
try (var serialized = serializedToReceive.receive()) {
|
||||||
var prevReaderIdx = serialized.readerIndex();
|
var prevReaderIdx = serialized.readerOffset();
|
||||||
var val = serialized.readShort();
|
var val = serialized.readShort();
|
||||||
serialized.readerIndex(prevReaderIdx + Short.BYTES);
|
serialized.readerOffset(prevReaderIdx + Short.BYTES);
|
||||||
return val;
|
return val;
|
||||||
} finally {
|
|
||||||
serialized.release();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public @NotNull ByteBuf serialize(@NotNull Short deserialized) {
|
public @NotNull Send<Buffer> serialize(@NotNull Short deserialized) {
|
||||||
var out = dictionary.getAllocator().directBuffer(Short.BYTES);
|
try (var out = dictionary.getAllocator().allocate(Short.BYTES)) {
|
||||||
try {
|
|
||||||
out.writeShort(deserialized);
|
out.writeShort(deserialized);
|
||||||
out.writerIndex(Short.BYTES);
|
out.writerOffset(Short.BYTES);
|
||||||
return out.retain();
|
return out.send();
|
||||||
} finally {
|
|
||||||
out.release();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1,271 +0,0 @@
|
|||||||
package it.cavallium.dbengine;
|
|
||||||
|
|
||||||
import static it.cavallium.dbengine.client.CompositeDatabasePartLocation.CompositeDatabasePartType.KV_DATABASE;
|
|
||||||
|
|
||||||
import io.netty.buffer.ByteBuf;
|
|
||||||
import io.netty.buffer.PooledByteBufAllocator;
|
|
||||||
import io.netty.buffer.Unpooled;
|
|
||||||
import it.cavallium.dbengine.client.CompositeDatabasePartLocation;
|
|
||||||
import it.cavallium.dbengine.client.CompositeSnapshot;
|
|
||||||
import it.cavallium.dbengine.database.Column;
|
|
||||||
import it.cavallium.dbengine.database.LLKeyValueDatabase;
|
|
||||||
import it.cavallium.dbengine.database.UpdateMode;
|
|
||||||
import it.cavallium.dbengine.database.collections.DatabaseMapDictionary;
|
|
||||||
import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep;
|
|
||||||
import it.cavallium.dbengine.database.collections.SubStageGetterMap;
|
|
||||||
import it.cavallium.dbengine.client.DatabaseOptions;
|
|
||||||
import it.cavallium.dbengine.database.disk.LLLocalDatabaseConnection;
|
|
||||||
import it.cavallium.dbengine.database.serialization.Serializer;
|
|
||||||
import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.nio.file.Files;
|
|
||||||
import java.nio.file.Path;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.concurrent.CompletionException;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import org.jetbrains.annotations.NotNull;
|
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
import reactor.core.publisher.Flux;
|
|
||||||
import reactor.core.publisher.Mono;
|
|
||||||
import reactor.core.scheduler.Schedulers;
|
|
||||||
import reactor.test.StepVerifier;
|
|
||||||
import reactor.util.function.Tuples;
|
|
||||||
|
|
||||||
public class OldDatabaseTests {
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testDatabaseAddKeysAndCheckSize() {
|
|
||||||
LinkedHashSet<String> originalKeys = new LinkedHashSet<>(List.of("K1a", "K1b", "K1c"));
|
|
||||||
|
|
||||||
StepVerifier
|
|
||||||
.create(
|
|
||||||
tempDb()
|
|
||||||
.flatMap(db -> db
|
|
||||||
.getDictionary("testmap", UpdateMode.DISALLOW)
|
|
||||||
.map(dictionary -> DatabaseMapDictionary.simple(dictionary,
|
|
||||||
new FixedStringSerializer(3),
|
|
||||||
Serializer.noop()
|
|
||||||
))
|
|
||||||
.flatMap(collection -> Flux
|
|
||||||
.fromIterable(originalKeys)
|
|
||||||
.flatMap(k1 -> collection.putValue(k1, DUMMY_VALUE.retain()))
|
|
||||||
.then(collection.leavesCount(null, false))
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.expectNext((long) originalKeys.size())
|
|
||||||
.verifyComplete();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testDeepDatabaseAddKeysAndCheckSize() {
|
|
||||||
LinkedHashSet<String> originalSuperKeys = new LinkedHashSet<>(List.of("K1a", "K1b", "K1c"));
|
|
||||||
LinkedHashSet<String> originalSubKeys = new LinkedHashSet<>(List.of("K2aa", "K2bb", "K2cc"));
|
|
||||||
|
|
||||||
StepVerifier
|
|
||||||
.create(
|
|
||||||
tempDb()
|
|
||||||
.flatMap(db -> db
|
|
||||||
.getDictionary("testmap", UpdateMode.DISALLOW)
|
|
||||||
.map(dictionary -> DatabaseMapDictionaryDeep.deepTail(dictionary,
|
|
||||||
new FixedStringSerializer(3),
|
|
||||||
4,
|
|
||||||
new SubStageGetterMap<>(new FixedStringSerializer(4), Serializer.noop())
|
|
||||||
))
|
|
||||||
.flatMap(collection -> Flux
|
|
||||||
.fromIterable(originalSuperKeys)
|
|
||||||
.flatMap(k1 -> collection.at(null, k1))
|
|
||||||
.flatMap(k1at -> Flux
|
|
||||||
.fromIterable(originalSubKeys)
|
|
||||||
.flatMap(k2 -> k1at.putValue(k2, DUMMY_VALUE.retain()))
|
|
||||||
)
|
|
||||||
.then(collection.leavesCount(null, false))
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.expectNext((long) originalSuperKeys.size() * originalSubKeys.size())
|
|
||||||
.verifyComplete();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testDeepDatabaseAddKeysAndConvertToLongerOnes() {
|
|
||||||
LinkedHashSet<String> originalSuperKeys = new LinkedHashSet<>(List.of("K1a", "K1b", "K1c"));
|
|
||||||
LinkedHashSet<String> originalSubKeys = new LinkedHashSet<>(List.of("K2aa", "K2bb", "K2cc"));
|
|
||||||
String newPrefix = "xxx";
|
|
||||||
|
|
||||||
StepVerifier
|
|
||||||
.create(tempDb()
|
|
||||||
.flatMapMany(db -> addKeysAndConvertToLongerOnes(db, originalSuperKeys, originalSubKeys, newPrefix))
|
|
||||||
)
|
|
||||||
.expectNextSequence(originalSuperKeys
|
|
||||||
.stream()
|
|
||||||
.flatMap(superKey -> originalSubKeys
|
|
||||||
.stream()
|
|
||||||
.map(subKey -> Map.entry(newPrefix + superKey, newPrefix + subKey))
|
|
||||||
)
|
|
||||||
.collect(Collectors.toList())
|
|
||||||
)
|
|
||||||
.verifyComplete();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static <U> Mono<? extends LLKeyValueDatabase> tempDb() {
|
|
||||||
var wrkspcPath = Path.of("/tmp/.cache/tempdb-" + DbTestUtils.dbId.incrementAndGet() + "/");
|
|
||||||
return Mono
|
|
||||||
.fromCallable(() -> {
|
|
||||||
if (Files.exists(wrkspcPath)) {
|
|
||||||
Files.walk(wrkspcPath)
|
|
||||||
.sorted(Comparator.reverseOrder())
|
|
||||||
.forEach(file -> {
|
|
||||||
try {
|
|
||||||
Files.delete(file);
|
|
||||||
} catch (IOException ex) {
|
|
||||||
throw new CompletionException(ex);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
Files.createDirectories(wrkspcPath);
|
|
||||||
return null;
|
|
||||||
})
|
|
||||||
.subscribeOn(Schedulers.boundedElastic())
|
|
||||||
.then(new LLLocalDatabaseConnection(PooledByteBufAllocator.DEFAULT, wrkspcPath).connect())
|
|
||||||
.flatMap(conn -> conn.getDatabase("testdb",
|
|
||||||
List.of(Column.dictionary("testmap")),
|
|
||||||
new DatabaseOptions(Map.of(), true, false, true, false, true, true, true, -1)
|
|
||||||
));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final ByteBuf DUMMY_VALUE;
|
|
||||||
static {
|
|
||||||
ByteBuf buf = Unpooled.directBuffer(2, 2);
|
|
||||||
buf.writeByte(0x01);
|
|
||||||
buf.writeByte(0x03);
|
|
||||||
DUMMY_VALUE = buf;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Flux<Entry<String, String>> addKeysAndConvertToLongerOnes(LLKeyValueDatabase db,
|
|
||||||
LinkedHashSet<String> originalSuperKeys,
|
|
||||||
LinkedHashSet<String> originalSubKeys,
|
|
||||||
String newPrefix) {
|
|
||||||
return Flux
|
|
||||||
.defer(() -> Mono
|
|
||||||
.zip(
|
|
||||||
db
|
|
||||||
.getDictionary("testmap", UpdateMode.DISALLOW)
|
|
||||||
.map(dictionary -> DatabaseMapDictionaryDeep.deepTail(dictionary,
|
|
||||||
new FixedStringSerializer(3),
|
|
||||||
4,
|
|
||||||
new SubStageGetterMap<>(new FixedStringSerializer(4), Serializer.noop())
|
|
||||||
)),
|
|
||||||
db
|
|
||||||
.getDictionary("testmap", UpdateMode.DISALLOW)
|
|
||||||
.map(dictionary -> DatabaseMapDictionaryDeep.deepTail(dictionary,
|
|
||||||
new FixedStringSerializer(6),
|
|
||||||
7,
|
|
||||||
new SubStageGetterMap<>(new FixedStringSerializer(7), Serializer.noop())
|
|
||||||
))
|
|
||||||
)
|
|
||||||
.single()
|
|
||||||
.flatMap(tuple -> {
|
|
||||||
var db1 = tuple.getT1();
|
|
||||||
return Flux
|
|
||||||
.fromIterable(originalSuperKeys)
|
|
||||||
.concatMap(superKey -> db1.at(null, superKey))
|
|
||||||
.concatMap(at -> Flux
|
|
||||||
.fromIterable(originalSubKeys)
|
|
||||||
.concatMap(subKey -> at
|
|
||||||
.at(null, subKey)
|
|
||||||
.flatMap(at2 -> at2
|
|
||||||
.set(DUMMY_VALUE.retainedSlice())
|
|
||||||
.doAfterTerminate(at2::release)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.doAfterTerminate(at::release)
|
|
||||||
)
|
|
||||||
.then(db
|
|
||||||
.takeSnapshot()
|
|
||||||
.map(snapshot -> new CompositeSnapshot(Map.of(CompositeDatabasePartLocation.of(KV_DATABASE,
|
|
||||||
db.getDatabaseName()), snapshot)))
|
|
||||||
)
|
|
||||||
.map(snapshot -> Tuples.of(tuple.getT1(), tuple.getT2(), snapshot))
|
|
||||||
.single();
|
|
||||||
})
|
|
||||||
.single()
|
|
||||||
.flatMap(tuple -> tuple.getT1().clear().thenReturn(tuple))
|
|
||||||
.flatMap(tuple -> tuple
|
|
||||||
.getT1()
|
|
||||||
.leavesCount(null, false)
|
|
||||||
.flatMap(count -> count == 0 ? Mono.just(tuple) : Mono.error(new IllegalStateException(
|
|
||||||
"Failed to clear map. Remaining elements after clear: " + count)))
|
|
||||||
)
|
|
||||||
.flatMapMany(tuple -> {
|
|
||||||
var oldDb = tuple.getT1();
|
|
||||||
var newDb = tuple.getT2();
|
|
||||||
var snapshot = tuple.getT3();
|
|
||||||
|
|
||||||
return oldDb
|
|
||||||
.getAllStages(snapshot)
|
|
||||||
.concatMap(parentEntry -> Mono
|
|
||||||
.fromCallable(() -> newPrefix + parentEntry.getKey())
|
|
||||||
.flatMapMany(newId1 -> parentEntry.getValue()
|
|
||||||
.getAllValues(snapshot)
|
|
||||||
.concatMap(entry -> Mono
|
|
||||||
.fromCallable(() -> newPrefix + entry.getKey())
|
|
||||||
.flatMap(newId2 -> newDb
|
|
||||||
.at(null, newId1)
|
|
||||||
.flatMap(newStage -> newStage
|
|
||||||
.putValue(newId2, entry.getValue())
|
|
||||||
.doAfterTerminate(newStage::release)
|
|
||||||
)
|
|
||||||
.thenReturn(Map.entry(newId1, newId2))
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.doAfterTerminate(() -> parentEntry.getValue().release())
|
|
||||||
)
|
|
||||||
.concatWith(db
|
|
||||||
.releaseSnapshot(snapshot.getSnapshot(db))
|
|
||||||
.then(oldDb.close())
|
|
||||||
.then(newDb.close())
|
|
||||||
.then(Mono.empty())
|
|
||||||
);
|
|
||||||
})
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class FixedStringSerializer implements SerializerFixedBinaryLength<String, ByteBuf> {
|
|
||||||
|
|
||||||
private final int size;
|
|
||||||
|
|
||||||
public FixedStringSerializer(int i) {
|
|
||||||
this.size = i;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getSerializedBinaryLength() {
|
|
||||||
return size;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public @NotNull String deserialize(ByteBuf serialized) {
|
|
||||||
try {
|
|
||||||
return serialized.toString(StandardCharsets.US_ASCII);
|
|
||||||
} finally {
|
|
||||||
serialized.release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ByteBuf serialize(@NotNull String deserialized) {
|
|
||||||
var serialized = deserialized.getBytes(StandardCharsets.US_ASCII);
|
|
||||||
var serializedBuf = Unpooled.directBuffer(serialized.length, serialized.length);
|
|
||||||
serializedBuf.writeBytes(serialized);
|
|
||||||
assert serializedBuf.isDirect();
|
|
||||||
return serializedBuf;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -7,6 +7,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
import io.netty.buffer.ByteBuf;
|
import io.netty.buffer.ByteBuf;
|
||||||
|
import io.netty.buffer.api.Buffer;
|
||||||
|
import io.netty.buffer.api.Send;
|
||||||
import it.cavallium.dbengine.DbTestUtils.TempDb;
|
import it.cavallium.dbengine.DbTestUtils.TempDb;
|
||||||
import it.cavallium.dbengine.DbTestUtils.TestAllocator;
|
import it.cavallium.dbengine.DbTestUtils.TestAllocator;
|
||||||
import it.cavallium.dbengine.database.LLDictionary;
|
import it.cavallium.dbengine.database.LLDictionary;
|
||||||
@ -75,7 +77,7 @@ public class TestLLDictionaryLeaks {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private LLDictionary getDict(UpdateMode updateMode) {
|
private LLDictionary getDict(UpdateMode updateMode) {
|
||||||
var dict = DbTestUtils.tempDictionary(db, updateMode).block();
|
var dict = DbTestUtils.tempDictionary(db, updateMode).blockOptional().orElseThrow();
|
||||||
var key1 = Mono.fromCallable(() -> fromString("test-key-1"));
|
var key1 = Mono.fromCallable(() -> fromString("test-key-1"));
|
||||||
var key2 = Mono.fromCallable(() -> fromString("test-key-2"));
|
var key2 = Mono.fromCallable(() -> fromString("test-key-2"));
|
||||||
var key3 = Mono.fromCallable(() -> fromString("test-key-3"));
|
var key3 = Mono.fromCallable(() -> fromString("test-key-3"));
|
||||||
@ -88,11 +90,12 @@ public class TestLLDictionaryLeaks {
|
|||||||
return dict;
|
return dict;
|
||||||
}
|
}
|
||||||
|
|
||||||
private ByteBuf fromString(String s) {
|
private Send<Buffer> fromString(String s) {
|
||||||
var sb = s.getBytes(StandardCharsets.UTF_8);
|
var sb = s.getBytes(StandardCharsets.UTF_8);
|
||||||
var b = db.getAllocator().buffer(sb.length);
|
try (var b = db.getAllocator().allocate(sb.length)) {
|
||||||
b.writeBytes(b);
|
b.writeBytes(b);
|
||||||
return b;
|
return b.send();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void run(Flux<?> publisher) {
|
private void run(Flux<?> publisher) {
|
||||||
@ -131,6 +134,14 @@ public class TestLLDictionaryLeaks {
|
|||||||
public void testNoOp() {
|
public void testNoOp() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNoOpAllocation() {
|
||||||
|
for (int i = 0; i < 10; i++) {
|
||||||
|
var a = allocator.allocator().allocate(i * 512);
|
||||||
|
a.send().receive().close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("provideArguments")
|
@MethodSource("provideArguments")
|
||||||
public void testGetDict(UpdateMode updateMode) {
|
public void testGetDict(UpdateMode updateMode) {
|
||||||
|
@ -3,14 +3,30 @@ package it.cavallium.dbengine.database.collections;
|
|||||||
import io.netty.buffer.ByteBufAllocator;
|
import io.netty.buffer.ByteBufAllocator;
|
||||||
import io.netty.buffer.PooledByteBufAllocator;
|
import io.netty.buffer.PooledByteBufAllocator;
|
||||||
import io.netty.buffer.Unpooled;
|
import io.netty.buffer.Unpooled;
|
||||||
|
import io.netty.buffer.api.BufferAllocator;
|
||||||
|
import io.netty.buffer.api.pool.PooledBufferAllocator;
|
||||||
import it.cavallium.dbengine.database.LLUtils;
|
import it.cavallium.dbengine.database.LLUtils;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import org.junit.jupiter.api.AfterAll;
|
||||||
import org.junit.jupiter.api.Assertions;
|
import org.junit.jupiter.api.Assertions;
|
||||||
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import static io.netty.buffer.Unpooled.*;
|
import static io.netty.buffer.Unpooled.*;
|
||||||
|
|
||||||
public class TestRanges {
|
public class TestRanges {
|
||||||
|
|
||||||
|
private static BufferAllocator alloc;
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void beforeAll() {
|
||||||
|
alloc = BufferAllocator.offHeapPooled();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterAll
|
||||||
|
public static void afterAll() {
|
||||||
|
alloc = BufferAllocator.offHeapPooled();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDirectBuffer() {
|
public void testDirectBuffer() {
|
||||||
Assertions.assertTrue(wrappedBuffer(Unpooled.directBuffer(10, 10), Unpooled.buffer(10, 10)).isDirect());
|
Assertions.assertTrue(wrappedBuffer(Unpooled.directBuffer(10, 10), Unpooled.buffer(10, 10)).isDirect());
|
||||||
@ -25,25 +41,31 @@ public class TestRanges {
|
|||||||
testNextRangeKey(new byte[] {0x00, 0x00, (byte) 0xFF});
|
testNextRangeKey(new byte[] {0x00, 0x00, (byte) 0xFF});
|
||||||
testNextRangeKey(new byte[] {0x00, 0x01, (byte) 0xFF});
|
testNextRangeKey(new byte[] {0x00, 0x01, (byte) 0xFF});
|
||||||
testNextRangeKey(new byte[] {0x00, (byte) 0xFF, (byte) 0xFF});
|
testNextRangeKey(new byte[] {0x00, (byte) 0xFF, (byte) 0xFF});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNextRangeKey2() {
|
||||||
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
|
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
|
||||||
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF});
|
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF});
|
||||||
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0});
|
testNextRangeKey(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testNextRangeKey(byte[] prefixKey) {
|
public void testNextRangeKey(byte[] prefixKey) {
|
||||||
|
byte[] firstRangeKey;
|
||||||
byte[] firstRangeKey = LLUtils.toArray(DatabaseMapDictionaryDeep.firstRangeKey(PooledByteBufAllocator.DEFAULT,
|
try (var firstRangeKeyBuf = DatabaseMapDictionaryDeep.firstRangeKey(alloc,
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(prefixKey)),
|
alloc.allocate(prefixKey.length).writeBytes(prefixKey).send(),
|
||||||
|
prefixKey.length, 7, 3).receive()) {
|
||||||
|
firstRangeKey = LLUtils.toArray(firstRangeKeyBuf);
|
||||||
|
}
|
||||||
|
byte[] nextRangeKey;
|
||||||
|
try (var nextRangeKeyBuf = DatabaseMapDictionaryDeep.nextRangeKey(alloc,
|
||||||
|
alloc.allocate(prefixKey.length).writeBytes(prefixKey).send(),
|
||||||
prefixKey.length,
|
prefixKey.length,
|
||||||
7,
|
7,
|
||||||
3
|
3
|
||||||
));
|
).receive()) {
|
||||||
byte[] nextRangeKey = LLUtils.toArray(DatabaseMapDictionaryDeep.nextRangeKey(PooledByteBufAllocator.DEFAULT,
|
nextRangeKey = LLUtils.toArray(nextRangeKeyBuf);
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(prefixKey)),
|
}
|
||||||
prefixKey.length,
|
|
||||||
7,
|
|
||||||
3
|
|
||||||
));
|
|
||||||
|
|
||||||
if (Arrays.equals(prefixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) {
|
if (Arrays.equals(prefixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) {
|
||||||
Assertions.assertArrayEquals(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0}, nextRangeKey);
|
Assertions.assertArrayEquals(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0}, nextRangeKey);
|
||||||
@ -78,6 +100,9 @@ public class TestRanges {
|
|||||||
testNextRangeKeyWithSuffix(new byte[] {0x00, 0x00, (byte) 0xFF}, new byte[] {0x00, 0x00, (byte) 0xFF});
|
testNextRangeKeyWithSuffix(new byte[] {0x00, 0x00, (byte) 0xFF}, new byte[] {0x00, 0x00, (byte) 0xFF});
|
||||||
testNextRangeKeyWithSuffix(new byte[] {0x00, 0x01, (byte) 0xFF}, new byte[] {0x00, 0x01, (byte) 0xFF});
|
testNextRangeKeyWithSuffix(new byte[] {0x00, 0x01, (byte) 0xFF}, new byte[] {0x00, 0x01, (byte) 0xFF});
|
||||||
testNextRangeKeyWithSuffix(new byte[] {0x00, (byte) 0xFF, (byte) 0xFF}, new byte[] {0x00, (byte) 0xFF, (byte) 0xFF});
|
testNextRangeKeyWithSuffix(new byte[] {0x00, (byte) 0xFF, (byte) 0xFF}, new byte[] {0x00, (byte) 0xFF, (byte) 0xFF});
|
||||||
|
}
|
||||||
|
@Test
|
||||||
|
public void testNextRangeKeyWithSuffix2() {
|
||||||
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
|
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
|
||||||
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF}, new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF});
|
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF}, new byte[] {(byte) 0xFF, (byte) 0, (byte) 0xFF});
|
||||||
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0}, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0});
|
testNextRangeKeyWithSuffix(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0}, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0});
|
||||||
@ -85,20 +110,24 @@ public class TestRanges {
|
|||||||
|
|
||||||
public void testNextRangeKeyWithSuffix(byte[] prefixKey, byte[] suffixKey) {
|
public void testNextRangeKeyWithSuffix(byte[] prefixKey, byte[] suffixKey) {
|
||||||
|
|
||||||
byte[] firstRangeKey = LLUtils.toArray(DatabaseMapDictionaryDeep.firstRangeKey(ByteBufAllocator.DEFAULT,
|
byte[] firstRangeKey;
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(prefixKey)),
|
try (var firstRangeKeyBuf = DatabaseMapDictionaryDeep.firstRangeKey(alloc,
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(suffixKey)),
|
alloc.allocate(prefixKey.length).writeBytes(prefixKey).send(),
|
||||||
|
alloc.allocate(suffixKey.length).writeBytes(suffixKey).send(),
|
||||||
prefixKey.length,
|
prefixKey.length,
|
||||||
3,
|
3,
|
||||||
7
|
7
|
||||||
));
|
).receive()) {
|
||||||
byte[] nextRangeKey = LLUtils.toArray(DatabaseMapDictionaryDeep.nextRangeKey(ByteBufAllocator.DEFAULT,
|
firstRangeKey = LLUtils.toArray(firstRangeKeyBuf);
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(prefixKey)),
|
}
|
||||||
LLUtils.convertToDirectByteBuf(PooledByteBufAllocator.DEFAULT, wrappedBuffer(suffixKey)),
|
try (var nextRangeKeyBuf = DatabaseMapDictionaryDeep.nextRangeKey(alloc,
|
||||||
|
alloc.allocate(prefixKey.length).writeBytes(prefixKey).send(),
|
||||||
|
alloc.allocate(suffixKey.length).writeBytes(suffixKey).send(),
|
||||||
prefixKey.length,
|
prefixKey.length,
|
||||||
3,
|
3,
|
||||||
7
|
7
|
||||||
));
|
).receive()) {
|
||||||
|
byte[] nextRangeKey = LLUtils.toArray(nextRangeKeyBuf);
|
||||||
|
|
||||||
if (Arrays.equals(prefixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}) && Arrays.equals(suffixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) {
|
if (Arrays.equals(prefixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}) && Arrays.equals(suffixKey, new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) {
|
||||||
Assertions.assertArrayEquals(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0}, nextRangeKey);
|
Assertions.assertArrayEquals(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0}, nextRangeKey);
|
||||||
@ -128,4 +157,5 @@ public class TestRanges {
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user