Clean database code
This commit is contained in:
parent
cdb65b31f3
commit
95afa6f9dd
@ -67,7 +67,7 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
|
|||||||
return getMulti(snapshot, keys, false);
|
return getMulti(snapshot, keys, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
Flux<Send<LLEntry>> putMulti(Flux<Send<LLEntry>> entries, boolean getOldValues);
|
Mono<Void> putMulti(Flux<Send<LLEntry>> entries);
|
||||||
|
|
||||||
<K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
|
<K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
|
||||||
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Buffer> updateFunction);
|
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Buffer> updateFunction);
|
||||||
@ -111,11 +111,7 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
|
|||||||
.flatMap(entriesReplacer)
|
.flatMap(entriesReplacer)
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return this
|
return this.putMulti(this.getRange(null, range, existsAlmostCertainly).flatMap(entriesReplacer));
|
||||||
.putMulti(this
|
|
||||||
.getRange(null, range, existsAlmostCertainly)
|
|
||||||
.flatMap(entriesReplacer), false)
|
|
||||||
.then();
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -391,7 +391,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
|
|||||||
sink.error(e);
|
sink.error(e);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
return dictionary.putMulti(serializedEntries, false).then();
|
return dictionary.putMulti(serializedEntries);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
package it.cavallium.dbengine.database.collections;
|
package it.cavallium.dbengine.database.collections;
|
||||||
|
|
||||||
import io.net5.buffer.api.Buffer;
|
|
||||||
import it.cavallium.dbengine.client.CompositeSnapshot;
|
import it.cavallium.dbengine.client.CompositeSnapshot;
|
||||||
import it.cavallium.dbengine.database.Delta;
|
import it.cavallium.dbengine.database.Delta;
|
||||||
import it.cavallium.dbengine.database.LLUtils;
|
import it.cavallium.dbengine.database.LLUtils;
|
||||||
@ -81,8 +80,10 @@ public interface DatabaseStageMap<T, U, US extends DatabaseStage<U>> extends
|
|||||||
default Mono<Delta<U>> updateValueAndGetDelta(T key,
|
default Mono<Delta<U>> updateValueAndGetDelta(T key,
|
||||||
boolean existsAlmostCertainly,
|
boolean existsAlmostCertainly,
|
||||||
SerializationFunction<@Nullable U, @Nullable U> updater) {
|
SerializationFunction<@Nullable U, @Nullable U> updater) {
|
||||||
return LLUtils.usingResource(this.at(null, key).single(),
|
var stageMono = this.at(null, key).single();
|
||||||
stage -> stage.updateAndGetDelta(updater, existsAlmostCertainly), true);
|
return stageMono.flatMap(stage -> stage
|
||||||
|
.updateAndGetDelta(updater, existsAlmostCertainly)
|
||||||
|
.doFinally(s -> stage.close()));
|
||||||
}
|
}
|
||||||
|
|
||||||
default Mono<Delta<U>> updateValueAndGetDelta(T key, SerializationFunction<@Nullable U, @Nullable U> updater) {
|
default Mono<Delta<U>> updateValueAndGetDelta(T key, SerializationFunction<@Nullable U, @Nullable U> updater) {
|
||||||
|
@ -38,6 +38,7 @@ import java.util.Map;
|
|||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.CompletionException;
|
||||||
import java.util.concurrent.ForkJoinPool;
|
import java.util.concurrent.ForkJoinPool;
|
||||||
import java.util.concurrent.ForkJoinTask;
|
import java.util.concurrent.ForkJoinTask;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
@ -68,7 +69,6 @@ import reactor.core.scheduler.Scheduler;
|
|||||||
import reactor.core.scheduler.Schedulers;
|
import reactor.core.scheduler.Schedulers;
|
||||||
import reactor.util.function.Tuple2;
|
import reactor.util.function.Tuple2;
|
||||||
import reactor.util.function.Tuple3;
|
import reactor.util.function.Tuple3;
|
||||||
import reactor.util.function.Tuple4;
|
|
||||||
import reactor.util.function.Tuples;
|
import reactor.util.function.Tuples;
|
||||||
|
|
||||||
public class LLLocalDictionary implements LLDictionary {
|
public class LLLocalDictionary implements LLDictionary {
|
||||||
@ -128,7 +128,6 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
private final Scheduler dbScheduler;
|
private final Scheduler dbScheduler;
|
||||||
private final Function<LLSnapshot, Snapshot> snapshotResolver;
|
private final Function<LLSnapshot, Snapshot> snapshotResolver;
|
||||||
private final UpdateMode updateMode;
|
private final UpdateMode updateMode;
|
||||||
private final DatabaseOptions databaseOptions;
|
|
||||||
private final boolean nettyDirect;
|
private final boolean nettyDirect;
|
||||||
private final BufferAllocator alloc;
|
private final BufferAllocator alloc;
|
||||||
|
|
||||||
@ -165,7 +164,6 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
this.dbScheduler = dbScheduler;
|
this.dbScheduler = dbScheduler;
|
||||||
this.snapshotResolver = snapshotResolver;
|
this.snapshotResolver = snapshotResolver;
|
||||||
this.updateMode = updateMode;
|
this.updateMode = updateMode;
|
||||||
this.databaseOptions = databaseOptions;
|
|
||||||
alloc = allocator;
|
alloc = allocator;
|
||||||
this.nettyDirect = databaseOptions.allowNettyDirect() && alloc.getAllocationType() == OFF_HEAP;
|
this.nettyDirect = databaseOptions.allowNettyDirect() && alloc.getAllocationType() == OFF_HEAP;
|
||||||
var meterRegistry = db.getMeterRegistry();
|
var meterRegistry = db.getMeterRegistry();
|
||||||
@ -350,6 +348,9 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
});
|
});
|
||||||
assert result != null;
|
assert result != null;
|
||||||
sink.next(!result);
|
sink.next(!result);
|
||||||
|
} catch (RocksDBException ex) {
|
||||||
|
sink.error(new RocksDBException("Failed to read range " + LLUtils.toStringSafe(range)
|
||||||
|
+ ": " + ex.getMessage()));
|
||||||
} finally {
|
} finally {
|
||||||
endedContains.increment();
|
endedContains.increment();
|
||||||
}
|
}
|
||||||
@ -397,39 +398,43 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
// Obtain the previous value from the database
|
// Obtain the previous value from the database
|
||||||
var previousDataMono = this.getPreviousData(keyMono, resultType, false);
|
var previousDataMono = this.getPreviousData(keyMono, resultType, false);
|
||||||
// Write the new entry to the database
|
// Write the new entry to the database
|
||||||
var putMono = entryMono
|
Mono<Send<Buffer>> putMono = entryMono
|
||||||
.publishOn(dbScheduler)
|
.publishOn(dbScheduler)
|
||||||
.<Void>handle((entry, sink) -> {
|
.handle((entry, sink) -> {
|
||||||
try (var key = entry.getKey().receive()) {
|
try (var key = entry.getKey().receive()) {
|
||||||
try (var value = entry.getValue().receive()) {
|
try (var value = entry.getValue().receive()) {
|
||||||
assert key.isAccessible();
|
assert key.isAccessible();
|
||||||
assert value.isAccessible();
|
assert value.isAccessible();
|
||||||
|
if (logger.isTraceEnabled(MARKER_ROCKSDB)) {
|
||||||
var varargs = new Supplier<?>[]{() -> toStringSafe(key), () -> toStringSafe(value)};
|
var varargs = new Supplier<?>[]{() -> toStringSafe(key), () -> toStringSafe(value)};
|
||||||
logger.trace(MARKER_ROCKSDB, "Writing {}: {}", varargs);
|
logger.trace(MARKER_ROCKSDB, "Writing {}: {}", varargs);
|
||||||
|
}
|
||||||
|
startedPut.increment();
|
||||||
|
try {
|
||||||
|
putTime.recordCallable(() -> {
|
||||||
db.put(EMPTY_WRITE_OPTIONS, key, value);
|
db.put(EMPTY_WRITE_OPTIONS, key, value);
|
||||||
|
return null;
|
||||||
|
});
|
||||||
|
} catch (RocksDBException ex) {
|
||||||
|
sink.error(new RocksDBException("Failed to write: " + ex.getMessage()));
|
||||||
|
return;
|
||||||
|
} catch (Exception ex) {
|
||||||
|
sink.error(ex);
|
||||||
|
return;
|
||||||
|
} finally {
|
||||||
|
endedPut.increment();
|
||||||
|
}
|
||||||
sink.complete();
|
sink.complete();
|
||||||
}
|
}
|
||||||
} catch (Throwable ex) {
|
|
||||||
sink.error(ex);
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
// Read the previous data, then write the new data, then return the previous data
|
// Read the previous data, then write the new data, then return the previous data
|
||||||
return Flux
|
return Flux.concat(previousDataMono, putMono).singleOrEmpty();
|
||||||
.concat(previousDataMono, putMono.then(Mono.empty()))
|
|
||||||
.singleOrEmpty()
|
|
||||||
.onErrorMap(cause -> new IOException("Failed to write", cause))
|
|
||||||
.elapsed()
|
|
||||||
.map(tuple -> {
|
|
||||||
putTime.record(tuple.getT1(), TimeUnit.MILLISECONDS);
|
|
||||||
return tuple.getT2();
|
|
||||||
})
|
|
||||||
.doFirst(startedPut::increment)
|
|
||||||
.doFinally(s -> endedPut.increment());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<UpdateMode> getUpdateMode() {
|
public Mono<UpdateMode> getUpdateMode() {
|
||||||
return Mono.fromSupplier(() -> updateMode);
|
return Mono.just(updateMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("DuplicatedCode")
|
@SuppressWarnings("DuplicatedCode")
|
||||||
@ -438,12 +443,14 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
SerializationFunction<@Nullable Send<Buffer>, @Nullable Buffer> updater,
|
SerializationFunction<@Nullable Send<Buffer>, @Nullable Buffer> updater,
|
||||||
UpdateReturnMode updateReturnMode,
|
UpdateReturnMode updateReturnMode,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return Mono.usingWhen(keyMono, keySend -> runOnDb(() -> {
|
return keyMono
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
.publishOn(dbScheduler)
|
||||||
throw new UnsupportedOperationException("Called update in a nonblocking thread");
|
.handle((keySend, sink) -> {
|
||||||
}
|
try (keySend) {
|
||||||
|
assert !Schedulers.isInNonBlockingThread() : "Called update in a nonblocking thread";
|
||||||
if (updateMode == UpdateMode.DISALLOW) {
|
if (updateMode == UpdateMode.DISALLOW) {
|
||||||
throw new UnsupportedOperationException("update() is disallowed");
|
sink.error(new UnsupportedOperationException("update() is disallowed"));
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
UpdateAtomicResultMode returnMode = switch (updateReturnMode) {
|
UpdateAtomicResultMode returnMode = switch (updateReturnMode) {
|
||||||
case NOTHING -> UpdateAtomicResultMode.NOTHING;
|
case NOTHING -> UpdateAtomicResultMode.NOTHING;
|
||||||
@ -459,13 +466,20 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
endedUpdates.increment();
|
endedUpdates.increment();
|
||||||
}
|
}
|
||||||
assert result != null;
|
assert result != null;
|
||||||
return switch (updateReturnMode) {
|
var previous = switch (updateReturnMode) {
|
||||||
case NOTHING -> null;
|
case NOTHING -> null;
|
||||||
case GET_NEW_VALUE -> ((UpdateAtomicResultCurrent) result).current();
|
case GET_NEW_VALUE -> ((UpdateAtomicResultCurrent) result).current();
|
||||||
case GET_OLD_VALUE -> ((UpdateAtomicResultPrevious) result).previous();
|
case GET_OLD_VALUE -> ((UpdateAtomicResultPrevious) result).previous();
|
||||||
};
|
};
|
||||||
}).onErrorMap(cause -> new IOException("Failed to read or write", cause)),
|
if (previous != null) {
|
||||||
keySend -> Mono.fromRunnable(keySend::close));
|
sink.next(previous);
|
||||||
|
} else {
|
||||||
|
sink.complete();
|
||||||
|
}
|
||||||
|
} catch (Exception ex) {
|
||||||
|
sink.error(ex);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("DuplicatedCode")
|
@SuppressWarnings("DuplicatedCode")
|
||||||
@ -473,19 +487,21 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
public Mono<Send<LLDelta>> updateAndGetDelta(Mono<Send<Buffer>> keyMono,
|
public Mono<Send<LLDelta>> updateAndGetDelta(Mono<Send<Buffer>> keyMono,
|
||||||
SerializationFunction<@Nullable Send<Buffer>, @Nullable Buffer> updater,
|
SerializationFunction<@Nullable Send<Buffer>, @Nullable Buffer> updater,
|
||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return Mono.usingWhen(keyMono, keySend -> runOnDb(() -> {
|
return keyMono
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
.publishOn(dbScheduler)
|
||||||
keySend.close();
|
.handle((keySend, sink) -> {
|
||||||
throw new UnsupportedOperationException("Called update in a nonblocking thread");
|
try (keySend) {
|
||||||
}
|
assert !Schedulers.isInNonBlockingThread() : "Called update in a nonblocking thread";
|
||||||
if (updateMode == UpdateMode.DISALLOW) {
|
if (updateMode == UpdateMode.DISALLOW) {
|
||||||
keySend.close();
|
sink.error(new UnsupportedOperationException("update() is disallowed"));
|
||||||
throw new UnsupportedOperationException("update() is disallowed");
|
return;
|
||||||
}
|
}
|
||||||
if (updateMode == UpdateMode.ALLOW && !db.supportsTransactions()) {
|
if (updateMode == UpdateMode.ALLOW && !db.supportsTransactions()) {
|
||||||
throw new UnsupportedOperationException("update() is disallowed because the database doesn't support"
|
sink.error(new UnsupportedOperationException("update() is disallowed because the database doesn't support"
|
||||||
+ "safe atomic operations");
|
+ "safe atomic operations"));
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
UpdateAtomicResult result;
|
UpdateAtomicResult result;
|
||||||
startedUpdates.increment();
|
startedUpdates.increment();
|
||||||
try {
|
try {
|
||||||
@ -495,36 +511,41 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
endedUpdates.increment();
|
endedUpdates.increment();
|
||||||
}
|
}
|
||||||
assert result != null;
|
assert result != null;
|
||||||
return ((UpdateAtomicResultDelta) result).delta();
|
sink.next(((UpdateAtomicResultDelta) result).delta());
|
||||||
}).onErrorMap(cause -> new IOException("Failed to read or write", cause)),
|
} catch (Exception ex) {
|
||||||
keySend -> Mono.fromRunnable(keySend::close));
|
sink.error(ex);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Send<Buffer>> remove(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
|
public Mono<Send<Buffer>> remove(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
|
||||||
return Mono.usingWhen(keyMono,
|
// Obtain the previous value from the database
|
||||||
keySend -> this
|
Mono<Send<Buffer>> previousDataMono = this.getPreviousData(keyMono, resultType, true);
|
||||||
.getPreviousData(keyMono, resultType, true)
|
// Delete the value from the database
|
||||||
.concatWith(this
|
Mono<Send<Buffer>> removeMono = keyMono
|
||||||
.<Send<Buffer>>runOnDb(() -> {
|
.publishOn(dbScheduler)
|
||||||
|
.handle((keySend, sink) -> {
|
||||||
try (var key = keySend.receive()) {
|
try (var key = keySend.receive()) {
|
||||||
if (logger.isTraceEnabled()) {
|
logger.trace(MARKER_ROCKSDB, "Deleting {}", () -> toStringSafe(key));
|
||||||
logger.trace(MARKER_ROCKSDB, "Deleting {}", toStringSafe(key));
|
startedRemove.increment();
|
||||||
|
try {
|
||||||
|
removeTime.recordCallable(() -> {
|
||||||
db.delete(EMPTY_WRITE_OPTIONS, key);
|
db.delete(EMPTY_WRITE_OPTIONS, key);
|
||||||
} else {
|
|
||||||
db.delete(EMPTY_WRITE_OPTIONS, key);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return null;
|
return null;
|
||||||
})
|
});
|
||||||
.onErrorMap(cause -> new IOException("Failed to delete", cause))
|
} finally {
|
||||||
)
|
endedRemove.increment();
|
||||||
.singleOrEmpty(),
|
}
|
||||||
keySend -> Mono.fromRunnable(keySend::close)
|
sink.complete();
|
||||||
).elapsed().map(tuple -> {
|
} catch (RocksDBException ex) {
|
||||||
removeTime.record(tuple.getT1(), TimeUnit.MILLISECONDS);
|
sink.error(new RocksDBException("Failed to delete: " + ex.getMessage()));
|
||||||
return tuple.getT2();
|
} catch (Exception ex) {
|
||||||
}).doFirst(startedRemove::increment).doFinally(s -> endedRemove.increment());
|
sink.error(ex);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
// Read the previous data, then delete the data, then return the previous data
|
||||||
|
return Flux.concat(previousDataMono, removeMono).singleOrEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Mono<Send<Buffer>> getPreviousData(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType,
|
private Mono<Send<Buffer>> getPreviousData(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType,
|
||||||
@ -534,28 +555,22 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
.containsKey(null, keyMono)
|
.containsKey(null, keyMono)
|
||||||
.single()
|
.single()
|
||||||
.map((Boolean bool) -> LLUtils.booleanToResponseByteBuffer(alloc, bool));
|
.map((Boolean bool) -> LLUtils.booleanToResponseByteBuffer(alloc, bool));
|
||||||
case PREVIOUS_VALUE -> Mono.usingWhen(
|
case PREVIOUS_VALUE -> keyMono
|
||||||
keyMono,
|
.publishOn(dbScheduler)
|
||||||
keySend -> this
|
.handle((keySend, sink) -> {
|
||||||
.runOnDb(() -> {
|
|
||||||
try (var key = keySend.receive()) {
|
try (var key = keySend.receive()) {
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
assert !Schedulers.isInNonBlockingThread() : "Called getPreviousData in a nonblocking thread";
|
||||||
throw new UnsupportedOperationException("Called getPreviousData in a nonblocking thread");
|
|
||||||
}
|
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
|
||||||
var keyString = toStringSafe(key);
|
|
||||||
var result = db.get(EMPTY_READ_OPTIONS, key, existsAlmostCertainly);
|
var result = db.get(EMPTY_READ_OPTIONS, key, existsAlmostCertainly);
|
||||||
logger.trace(MARKER_ROCKSDB, "Reading {}: {}", keyString, toStringSafe(result));
|
logger.trace(MARKER_ROCKSDB, "Reading {}: {}", () -> toStringSafe(key), () -> toStringSafe(result));
|
||||||
return result == null ? null : result.send();
|
if (result == null) {
|
||||||
|
sink.complete();
|
||||||
} else {
|
} else {
|
||||||
var result = db.get(EMPTY_READ_OPTIONS, key, existsAlmostCertainly);
|
sink.next(result.send());
|
||||||
return result == null ? null : result.send();
|
|
||||||
}
|
}
|
||||||
|
} catch (Exception ex) {
|
||||||
|
sink.error(ex);
|
||||||
}
|
}
|
||||||
})
|
});
|
||||||
.onErrorMap(cause -> new IOException("Failed to read ", cause)),
|
|
||||||
keySend -> Mono.fromRunnable(keySend::close));
|
|
||||||
case VOID -> Mono.empty();
|
case VOID -> Mono.empty();
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
@ -566,15 +581,14 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return keys
|
return keys
|
||||||
.buffer(MULTI_GET_WINDOW)
|
.buffer(MULTI_GET_WINDOW)
|
||||||
.flatMapSequential(keysWindow -> runOnDb(() -> {
|
.publishOn(dbScheduler)
|
||||||
|
.<ArrayList<Optional<Buffer>>>handle((keysWindow, sink) -> {
|
||||||
List<Buffer> keyBufsWindow = new ArrayList<>(keysWindow.size());
|
List<Buffer> keyBufsWindow = new ArrayList<>(keysWindow.size());
|
||||||
for (Send<Buffer> bufferSend : keysWindow) {
|
for (Send<Buffer> bufferSend : keysWindow) {
|
||||||
keyBufsWindow.add(bufferSend.receive());
|
keyBufsWindow.add(bufferSend.receive());
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
assert !Schedulers.isInNonBlockingThread() : "Called getMulti in a nonblocking thread";
|
||||||
throw new UnsupportedOperationException("Called getMulti in a nonblocking thread");
|
|
||||||
}
|
|
||||||
var readOptions = Objects.requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
|
var readOptions = Objects.requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
|
||||||
List<byte[]> results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
|
List<byte[]> results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
|
||||||
var mappedResults = new ArrayList<Optional<Buffer>>(results.size());
|
var mappedResults = new ArrayList<Optional<Buffer>>(results.size());
|
||||||
@ -591,48 +605,30 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
mappedResults.add(valueOpt);
|
mappedResults.add(valueOpt);
|
||||||
}
|
}
|
||||||
return mappedResults;
|
sink.next(mappedResults);
|
||||||
|
} catch (RocksDBException ex) {
|
||||||
|
sink.error(new RocksDBException("Failed to read keys: " + ex.getMessage()));
|
||||||
} finally {
|
} finally {
|
||||||
for (Buffer buffer : keyBufsWindow) {
|
for (Buffer buffer : keyBufsWindow) {
|
||||||
buffer.close();
|
buffer.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
.flatMapIterable(list -> list)
|
.flatMapIterable(list -> list);
|
||||||
.onErrorMap(cause -> new IOException("Failed to read keys", cause))
|
|
||||||
.doAfterTerminate(() -> keysWindow.forEach(Send::close)), 2); // Max concurrency is 2 to read data while preparing the next segment;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<Send<LLEntry>> putMulti(Flux<Send<LLEntry>> entries, boolean getOldValues) {
|
public Mono<Void> putMulti(Flux<Send<LLEntry>> entries) {
|
||||||
return entries
|
return entries
|
||||||
.buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
|
.buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
|
||||||
.flatMapSequential(ew -> Mono
|
.publishOn(dbScheduler)
|
||||||
.<List<Send<LLEntry>>>fromCallable(() -> {
|
.handle((entriesWindowList, sink) -> {
|
||||||
var entriesWindow = new ArrayList<LLEntry>(ew.size());
|
var entriesWindow = new ArrayList<LLEntry>(entriesWindowList.size());
|
||||||
for (Send<LLEntry> entrySend : ew) {
|
for (Send<LLEntry> entrySend : entriesWindowList) {
|
||||||
entriesWindow.add(entrySend.receive());
|
entriesWindow.add(entrySend.receive());
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
assert !Schedulers.isInNonBlockingThread() : "Called putMulti in a nonblocking thread";
|
||||||
throw new UnsupportedOperationException("Called putMulti in a nonblocking thread");
|
|
||||||
}
|
|
||||||
ArrayList<Send<LLEntry>> oldValues;
|
|
||||||
if (getOldValues) {
|
|
||||||
oldValues = new ArrayList<>(entriesWindow.size());
|
|
||||||
try (var readOptions = resolveSnapshot(null)) {
|
|
||||||
for (LLEntry entry : entriesWindow) {
|
|
||||||
try (var key = entry.getKey().receive()) {
|
|
||||||
Buffer oldValue = db.get(readOptions, key, false);
|
|
||||||
if (oldValue != null) {
|
|
||||||
oldValues.add(LLEntry.of(key, oldValue).send());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
oldValues = null;
|
|
||||||
}
|
|
||||||
if (USE_WRITE_BATCHES_IN_PUT_MULTI) {
|
if (USE_WRITE_BATCHES_IN_PUT_MULTI) {
|
||||||
var batch = new CappedWriteBatch(db,
|
var batch = new CappedWriteBatch(db,
|
||||||
alloc,
|
alloc,
|
||||||
@ -661,14 +657,16 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
db.put(EMPTY_WRITE_OPTIONS, entry.getKeyUnsafe(), entry.getValueUnsafe());
|
db.put(EMPTY_WRITE_OPTIONS, entry.getKeyUnsafe(), entry.getValueUnsafe());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return oldValues;
|
sink.complete();
|
||||||
|
} catch (RocksDBException ex) {
|
||||||
|
sink.error(new RocksDBException("Failed to write: " + ex.getMessage()));
|
||||||
} finally {
|
} finally {
|
||||||
for (LLEntry llEntry : entriesWindow) {
|
for (LLEntry llEntry : entriesWindow) {
|
||||||
llEntry.close();
|
llEntry.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}).subscribeOn(dbScheduler), 2) // Max concurrency is 2 to read data while preparing the next segment
|
})
|
||||||
.flatMapIterable(oldValuesList -> oldValuesList);
|
.then();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -895,11 +893,8 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
ro.setReadaheadSize(32 * 1024);
|
ro.setReadaheadSize(32 * 1024);
|
||||||
}
|
}
|
||||||
ro.setVerifyChecksums(true);
|
ro.setVerifyChecksums(true);
|
||||||
var rocksIteratorTuple = getRocksIterator(alloc,
|
try (var rocksIteratorTuple = getRocksIterator(nettyDirect, ro, range, db)) {
|
||||||
nettyDirect, ro, range, db
|
var rocksIterator = rocksIteratorTuple.iterator();
|
||||||
);
|
|
||||||
try {
|
|
||||||
try (var rocksIterator = rocksIteratorTuple.getT1()) {
|
|
||||||
rocksIterator.seekToFirst();
|
rocksIterator.seekToFirst();
|
||||||
rocksIterator.status();
|
rocksIterator.status();
|
||||||
while (rocksIterator.isValid() && !sink.isCancelled()) {
|
while (rocksIterator.isValid() && !sink.isCancelled()) {
|
||||||
@ -915,11 +910,6 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
rocksIterator.next();
|
rocksIterator.next();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
rocksIteratorTuple.getT2().close();
|
|
||||||
rocksIteratorTuple.getT3().close();
|
|
||||||
rocksIteratorTuple.getT4().close();
|
|
||||||
}
|
|
||||||
sink.complete();
|
sink.complete();
|
||||||
} catch (Throwable ex) {
|
} catch (Throwable ex) {
|
||||||
sink.error(ex);
|
sink.error(ex);
|
||||||
@ -1271,6 +1261,9 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method should not modify or move the writerIndex/readerIndex of the key
|
||||||
|
*/
|
||||||
private static ReleasableSlice setIterateBound(boolean allowNettyDirect,
|
private static ReleasableSlice setIterateBound(boolean allowNettyDirect,
|
||||||
ReadOptions readOpts, IterateBound boundType, Buffer key) {
|
ReadOptions readOpts, IterateBound boundType, Buffer key) {
|
||||||
requireNonNull(key);
|
requireNonNull(key);
|
||||||
@ -1281,46 +1274,36 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
assert keyInternalByteBuffer.position() == 0;
|
assert keyInternalByteBuffer.position() == 0;
|
||||||
slice = new DirectSlice(keyInternalByteBuffer, key.readableBytes());
|
slice = new DirectSlice(keyInternalByteBuffer, key.readableBytes());
|
||||||
assert slice.size() == key.readableBytes();
|
assert slice.size() == key.readableBytes();
|
||||||
assert slice.compare(new Slice(LLUtils.toArray(key))) == 0;
|
|
||||||
if (boundType == IterateBound.LOWER) {
|
|
||||||
readOpts.setIterateLowerBound(slice);
|
|
||||||
} else {
|
|
||||||
readOpts.setIterateUpperBound(slice);
|
|
||||||
}
|
|
||||||
return new ReleasableSliceImpl(slice, null, key);
|
|
||||||
} else {
|
} else {
|
||||||
slice = new Slice(requireNonNull(LLUtils.toArray(key)));
|
slice = new Slice(requireNonNull(LLUtils.toArray(key)));
|
||||||
|
}
|
||||||
if (boundType == IterateBound.LOWER) {
|
if (boundType == IterateBound.LOWER) {
|
||||||
readOpts.setIterateLowerBound(slice);
|
readOpts.setIterateLowerBound(slice);
|
||||||
} else {
|
} else {
|
||||||
readOpts.setIterateUpperBound(slice);
|
readOpts.setIterateUpperBound(slice);
|
||||||
}
|
}
|
||||||
return new ReleasableSliceImpl(slice, null, null);
|
return new ReleasableSliceImplWithRelease(slice);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ReleasableSlice emptyReleasableSlice() {
|
private static ReleasableSlice emptyReleasableSlice() {
|
||||||
var arr = new byte[0];
|
var arr = new byte[0];
|
||||||
|
|
||||||
return new SimpleSliceWithoutRelease(new Slice(arr), null, arr);
|
return new ReleasableSliceImplWithoutRelease(new Slice(arr));
|
||||||
}
|
}
|
||||||
|
|
||||||
public record SimpleSliceWithoutRelease(AbstractSlice<?> slice, @Nullable Buffer byteBuf,
|
/**
|
||||||
@Nullable Object additionalData) implements ReleasableSlice {}
|
* This method should not modify or move the writerIndex/readerIndex of the key
|
||||||
|
*/
|
||||||
|
public record ReleasableSliceImplWithoutRelease(AbstractSlice<?> slice) implements ReleasableSlice {}
|
||||||
|
|
||||||
public record ReleasableSliceImpl(AbstractSlice<?> slice, @Nullable Buffer byteBuf,
|
/**
|
||||||
@Nullable Object additionalData) implements ReleasableSlice {
|
* This class should not modify or move the writerIndex/readerIndex of the key
|
||||||
|
*/
|
||||||
|
public record ReleasableSliceImplWithRelease(AbstractSlice<?> slice) implements ReleasableSlice {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
slice.clear();
|
|
||||||
slice.close();
|
slice.close();
|
||||||
if (byteBuf != null) {
|
|
||||||
byteBuf.close();
|
|
||||||
}
|
|
||||||
if (additionalData instanceof ByteBuffer bb && bb.isDirect()) {
|
|
||||||
PlatformDependent.freeDirectBuffer(bb);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1745,15 +1728,16 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method should not modify or move the writerIndex/readerIndex of the buffers inside the range
|
||||||
|
*/
|
||||||
@NotNull
|
@NotNull
|
||||||
public static Tuple4<RocksIterator, ReleasableSlice, ReleasableSlice, SafeCloseable> getRocksIterator(BufferAllocator alloc,
|
public static RocksIteratorTuple getRocksIterator(
|
||||||
boolean allowNettyDirect,
|
boolean allowNettyDirect,
|
||||||
ReadOptions readOptions,
|
ReadOptions readOptions,
|
||||||
LLRange range,
|
LLRange range,
|
||||||
RocksDBColumn db) {
|
RocksDBColumn db) {
|
||||||
if (Schedulers.isInNonBlockingThread()) {
|
assert !Schedulers.isInNonBlockingThread() : "Called getRocksIterator in a nonblocking thread";
|
||||||
throw new UnsupportedOperationException("Called getRocksIterator in a nonblocking thread");
|
|
||||||
}
|
|
||||||
ReleasableSlice sliceMin;
|
ReleasableSlice sliceMin;
|
||||||
ReleasableSlice sliceMax;
|
ReleasableSlice sliceMax;
|
||||||
if (range.hasMin()) {
|
if (range.hasMin()) {
|
||||||
@ -1775,6 +1759,6 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
seekTo = () -> {};
|
seekTo = () -> {};
|
||||||
rocksIterator.seekToFirst();
|
rocksIterator.seekToFirst();
|
||||||
}
|
}
|
||||||
return Tuples.of(rocksIterator, sliceMin, sliceMax, seekTo);
|
return new RocksIteratorTuple(rocksIterator, sliceMin, sliceMax, seekTo);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -91,10 +91,10 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
|
|||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range));
|
||||||
}
|
}
|
||||||
return LLLocalDictionary.getRocksIterator(db.getAllocator(), allowNettyDirect, readOptions, range, db);
|
return LLLocalDictionary.getRocksIterator(allowNettyDirect, readOptions, range, db);
|
||||||
}, (tuple, sink) -> {
|
}, (tuple, sink) -> {
|
||||||
try {
|
try {
|
||||||
var rocksIterator = tuple.getT1();
|
var rocksIterator = tuple.iterator();
|
||||||
ObjectArrayList<T> values = new ObjectArrayList<>();
|
ObjectArrayList<T> values = new ObjectArrayList<>();
|
||||||
Buffer firstGroupKey = null;
|
Buffer firstGroupKey = null;
|
||||||
try {
|
try {
|
||||||
@ -155,13 +155,7 @@ public abstract class LLLocalGroupedReactiveRocksIterator<T> extends
|
|||||||
sink.error(ex);
|
sink.error(ex);
|
||||||
}
|
}
|
||||||
return tuple;
|
return tuple;
|
||||||
}, tuple -> {
|
}, RocksIteratorTuple::close);
|
||||||
var rocksIterator = tuple.getT1();
|
|
||||||
rocksIterator.close();
|
|
||||||
tuple.getT2().close();
|
|
||||||
tuple.getT3().close();
|
|
||||||
tuple.getT4().close();
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
|
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
|
||||||
|
@ -3,7 +3,6 @@ package it.cavallium.dbengine.database.disk;
|
|||||||
import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB;
|
import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB;
|
||||||
|
|
||||||
import io.net5.buffer.api.Buffer;
|
import io.net5.buffer.api.Buffer;
|
||||||
import io.net5.buffer.api.BufferAllocator;
|
|
||||||
import io.net5.buffer.api.Drop;
|
import io.net5.buffer.api.Drop;
|
||||||
import io.net5.buffer.api.Owned;
|
import io.net5.buffer.api.Owned;
|
||||||
import io.net5.buffer.api.Send;
|
import io.net5.buffer.api.Send;
|
||||||
@ -12,10 +11,7 @@ import it.cavallium.dbengine.database.LLRange;
|
|||||||
import it.cavallium.dbengine.database.LLUtils;
|
import it.cavallium.dbengine.database.LLUtils;
|
||||||
import org.apache.logging.log4j.LogManager;
|
import org.apache.logging.log4j.LogManager;
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.jetbrains.annotations.Nullable;
|
|
||||||
import org.rocksdb.ColumnFamilyHandle;
|
|
||||||
import org.rocksdb.ReadOptions;
|
import org.rocksdb.ReadOptions;
|
||||||
import org.rocksdb.RocksDB;
|
|
||||||
import org.rocksdb.RocksDBException;
|
import org.rocksdb.RocksDBException;
|
||||||
import reactor.core.publisher.Flux;
|
import reactor.core.publisher.Flux;
|
||||||
|
|
||||||
@ -27,8 +23,8 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
@Override
|
@Override
|
||||||
public void drop(LLLocalKeyPrefixReactiveRocksIterator obj) {
|
public void drop(LLLocalKeyPrefixReactiveRocksIterator obj) {
|
||||||
try {
|
try {
|
||||||
if (obj.range != null) {
|
if (obj.rangeShared != null) {
|
||||||
obj.range.close();
|
obj.rangeShared.close();
|
||||||
}
|
}
|
||||||
} catch (Throwable ex) {
|
} catch (Throwable ex) {
|
||||||
logger.error("Failed to close range", ex);
|
logger.error("Failed to close range", ex);
|
||||||
@ -57,7 +53,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
|
|
||||||
private final RocksDBColumn db;
|
private final RocksDBColumn db;
|
||||||
private final int prefixLength;
|
private final int prefixLength;
|
||||||
private LLRange range;
|
private LLRange rangeShared;
|
||||||
private final boolean allowNettyDirect;
|
private final boolean allowNettyDirect;
|
||||||
private ReadOptions readOptions;
|
private ReadOptions readOptions;
|
||||||
private final boolean canFillCache;
|
private final boolean canFillCache;
|
||||||
@ -72,7 +68,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
try (range) {
|
try (range) {
|
||||||
this.db = db;
|
this.db = db;
|
||||||
this.prefixLength = prefixLength;
|
this.prefixLength = prefixLength;
|
||||||
this.range = range.receive();
|
this.rangeShared = range.receive();
|
||||||
this.allowNettyDirect = allowNettyDirect;
|
this.allowNettyDirect = allowNettyDirect;
|
||||||
this.readOptions = readOptions;
|
this.readOptions = readOptions;
|
||||||
this.canFillCache = canFillCache;
|
this.canFillCache = canFillCache;
|
||||||
@ -81,22 +77,19 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
|
|
||||||
|
|
||||||
public Flux<Send<Buffer>> flux() {
|
public Flux<Send<Buffer>> flux() {
|
||||||
return Flux.using(
|
return Flux.generate(() -> {
|
||||||
() -> range.copy().send(),
|
|
||||||
rangeSend -> Flux
|
|
||||||
.generate(() -> {
|
|
||||||
var readOptions = new ReadOptions(this.readOptions);
|
var readOptions = new ReadOptions(this.readOptions);
|
||||||
if (!range.hasMin() || !range.hasMax()) {
|
if (!rangeShared.hasMin() || !rangeShared.hasMax()) {
|
||||||
readOptions.setReadaheadSize(32 * 1024); // 32KiB
|
readOptions.setReadaheadSize(32 * 1024); // 32KiB
|
||||||
readOptions.setFillCache(canFillCache);
|
readOptions.setFillCache(canFillCache);
|
||||||
}
|
}
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
return LLLocalDictionary.getRocksIterator(db.getAllocator(), allowNettyDirect, readOptions, range, db);
|
return LLLocalDictionary.getRocksIterator(allowNettyDirect, readOptions, rangeShared, db);
|
||||||
}, (tuple, sink) -> {
|
}, (tuple, sink) -> {
|
||||||
try {
|
try {
|
||||||
var rocksIterator = tuple.getT1();
|
var rocksIterator = tuple.iterator();
|
||||||
rocksIterator.status();
|
rocksIterator.status();
|
||||||
Buffer firstGroupKey = null;
|
Buffer firstGroupKey = null;
|
||||||
try {
|
try {
|
||||||
@ -110,8 +103,12 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
try (key) {
|
try (key) {
|
||||||
if (firstGroupKey == null) {
|
if (firstGroupKey == null) {
|
||||||
firstGroupKey = key.copy();
|
firstGroupKey = key.copy();
|
||||||
} else if (!LLUtils.equals(firstGroupKey, firstGroupKey.readerOffset(), key, key.readerOffset(),
|
} else if (!LLUtils.equals(firstGroupKey,
|
||||||
prefixLength)) {
|
firstGroupKey.readerOffset(),
|
||||||
|
key,
|
||||||
|
key.readerOffset(),
|
||||||
|
prefixLength
|
||||||
|
)) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
rocksIterator.next();
|
rocksIterator.next();
|
||||||
@ -127,7 +124,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB,
|
logger.trace(MARKER_ROCKSDB,
|
||||||
"Range {} is reading prefix {}",
|
"Range {} is reading prefix {}",
|
||||||
LLUtils.toStringSafe(range),
|
LLUtils.toStringSafe(rangeShared),
|
||||||
LLUtils.toStringSafe(groupKeyPrefix)
|
LLUtils.toStringSafe(groupKeyPrefix)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -135,7 +132,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
sink.next(groupKeyPrefix.send());
|
sink.next(groupKeyPrefix.send());
|
||||||
} else {
|
} else {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
sink.complete();
|
sink.complete();
|
||||||
}
|
}
|
||||||
@ -146,20 +143,12 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
}
|
}
|
||||||
} catch (RocksDBException ex) {
|
} catch (RocksDBException ex) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
sink.error(ex);
|
sink.error(ex);
|
||||||
}
|
}
|
||||||
return tuple;
|
return tuple;
|
||||||
}, tuple -> {
|
}, RocksIteratorTuple::close);
|
||||||
var rocksIterator = tuple.getT1();
|
|
||||||
rocksIterator.close();
|
|
||||||
tuple.getT2().close();
|
|
||||||
tuple.getT3().close();
|
|
||||||
tuple.getT4().close();
|
|
||||||
}),
|
|
||||||
resource -> resource.close()
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -169,7 +158,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Owned<LLLocalKeyPrefixReactiveRocksIterator> prepareSend() {
|
protected Owned<LLLocalKeyPrefixReactiveRocksIterator> prepareSend() {
|
||||||
var range = this.range.send();
|
var range = this.rangeShared.send();
|
||||||
var readOptions = new ReadOptions(this.readOptions);
|
var readOptions = new ReadOptions(this.readOptions);
|
||||||
return drop -> new LLLocalKeyPrefixReactiveRocksIterator(db,
|
return drop -> new LLLocalKeyPrefixReactiveRocksIterator(db,
|
||||||
prefixLength,
|
prefixLength,
|
||||||
@ -181,7 +170,7 @@ public class LLLocalKeyPrefixReactiveRocksIterator extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void makeInaccessible() {
|
protected void makeInaccessible() {
|
||||||
this.range = null;
|
this.rangeShared = null;
|
||||||
this.readOptions = null;
|
this.readOptions = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -25,8 +25,8 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
@Override
|
@Override
|
||||||
public void drop(LLLocalReactiveRocksIterator<?> obj) {
|
public void drop(LLLocalReactiveRocksIterator<?> obj) {
|
||||||
try {
|
try {
|
||||||
if (obj.range != null) {
|
if (obj.rangeShared != null) {
|
||||||
obj.range.close();
|
obj.rangeShared.close();
|
||||||
}
|
}
|
||||||
} catch (Throwable ex) {
|
} catch (Throwable ex) {
|
||||||
logger.error("Failed to close range", ex);
|
logger.error("Failed to close range", ex);
|
||||||
@ -54,7 +54,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
};
|
};
|
||||||
|
|
||||||
private final RocksDBColumn db;
|
private final RocksDBColumn db;
|
||||||
private LLRange range;
|
private LLRange rangeShared;
|
||||||
private final boolean allowNettyDirect;
|
private final boolean allowNettyDirect;
|
||||||
private ReadOptions readOptions;
|
private ReadOptions readOptions;
|
||||||
private final boolean readValues;
|
private final boolean readValues;
|
||||||
@ -66,27 +66,29 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
ReadOptions readOptions,
|
ReadOptions readOptions,
|
||||||
boolean readValues) {
|
boolean readValues) {
|
||||||
super((Drop<LLLocalReactiveRocksIterator<T>>) (Drop) DROP);
|
super((Drop<LLLocalReactiveRocksIterator<T>>) (Drop) DROP);
|
||||||
|
try (range) {
|
||||||
this.db = db;
|
this.db = db;
|
||||||
this.range = range.receive();
|
this.rangeShared = range.receive();
|
||||||
this.allowNettyDirect = allowNettyDirect;
|
this.allowNettyDirect = allowNettyDirect;
|
||||||
this.readOptions = readOptions;
|
this.readOptions = readOptions;
|
||||||
this.readValues = readValues;
|
this.readValues = readValues;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public final Flux<T> flux() {
|
public final Flux<T> flux() {
|
||||||
return Flux.generate(() -> {
|
return Flux.generate(() -> {
|
||||||
var readOptions = new ReadOptions(this.readOptions);
|
var readOptions = new ReadOptions(this.readOptions);
|
||||||
if (!range.hasMin() || !range.hasMax()) {
|
if (!rangeShared.hasMin() || !rangeShared.hasMax()) {
|
||||||
readOptions.setReadaheadSize(32 * 1024); // 32KiB
|
readOptions.setReadaheadSize(32 * 1024); // 32KiB
|
||||||
readOptions.setFillCache(false);
|
readOptions.setFillCache(false);
|
||||||
}
|
}
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} started", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
return getRocksIterator(db.getAllocator(), allowNettyDirect, readOptions, range, db);
|
return getRocksIterator(allowNettyDirect, readOptions, rangeShared, db);
|
||||||
}, (tuple, sink) -> {
|
}, (tuple, sink) -> {
|
||||||
try {
|
try {
|
||||||
var rocksIterator = tuple.getT1();
|
var rocksIterator = tuple.iterator();
|
||||||
rocksIterator.status();
|
rocksIterator.status();
|
||||||
if (rocksIterator.isValid()) {
|
if (rocksIterator.isValid()) {
|
||||||
Buffer key;
|
Buffer key;
|
||||||
@ -110,7 +112,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB,
|
logger.trace(MARKER_ROCKSDB,
|
||||||
"Range {} is reading {}: {}",
|
"Range {} is reading {}: {}",
|
||||||
LLUtils.toStringSafe(range),
|
LLUtils.toStringSafe(rangeShared),
|
||||||
LLUtils.toStringSafe(key),
|
LLUtils.toStringSafe(key),
|
||||||
LLUtils.toStringSafe(value)
|
LLUtils.toStringSafe(value)
|
||||||
);
|
);
|
||||||
@ -128,24 +130,18 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} ended", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
sink.complete();
|
sink.complete();
|
||||||
}
|
}
|
||||||
} catch (RocksDBException ex) {
|
} catch (RocksDBException ex) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(range));
|
logger.trace(MARKER_ROCKSDB, "Range {} failed", LLUtils.toStringSafe(rangeShared));
|
||||||
}
|
}
|
||||||
sink.error(ex);
|
sink.error(ex);
|
||||||
}
|
}
|
||||||
return tuple;
|
return tuple;
|
||||||
}, tuple -> {
|
}, RocksIteratorTuple::close);
|
||||||
var rocksIterator = tuple.getT1();
|
|
||||||
rocksIterator.close();
|
|
||||||
tuple.getT2().close();
|
|
||||||
tuple.getT3().close();
|
|
||||||
tuple.getT4().close();
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
|
public abstract T getEntry(@Nullable Send<Buffer> key, @Nullable Send<Buffer> value);
|
||||||
@ -157,7 +153,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Owned<LLLocalReactiveRocksIterator<T>> prepareSend() {
|
protected Owned<LLLocalReactiveRocksIterator<T>> prepareSend() {
|
||||||
var range = this.range.send();
|
var range = this.rangeShared.send();
|
||||||
var readOptions = new ReadOptions(this.readOptions);
|
var readOptions = new ReadOptions(this.readOptions);
|
||||||
return drop -> new LLLocalReactiveRocksIterator<>(db, range, allowNettyDirect, readOptions, readValues) {
|
return drop -> new LLLocalReactiveRocksIterator<>(db, range, allowNettyDirect, readOptions, readValues) {
|
||||||
@Override
|
@Override
|
||||||
@ -168,7 +164,7 @@ public abstract class LLLocalReactiveRocksIterator<T> extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void makeInaccessible() {
|
protected void makeInaccessible() {
|
||||||
this.range = null;
|
this.rangeShared = null;
|
||||||
this.readOptions = null;
|
this.readOptions = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -10,10 +10,4 @@ public interface ReleasableSlice extends SafeCloseable {
|
|||||||
default void close() {
|
default void close() {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
AbstractSlice<?> slice();
|
|
||||||
|
|
||||||
Buffer byteBuf();
|
|
||||||
|
|
||||||
Object additionalData();
|
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,18 @@
|
|||||||
|
package it.cavallium.dbengine.database.disk;
|
||||||
|
|
||||||
|
import it.cavallium.dbengine.database.SafeCloseable;
|
||||||
|
import org.jetbrains.annotations.NotNull;
|
||||||
|
import org.rocksdb.RocksIterator;
|
||||||
|
|
||||||
|
public record RocksIteratorTuple(@NotNull RocksIterator iterator, @NotNull ReleasableSlice sliceMin,
|
||||||
|
@NotNull ReleasableSlice sliceMax, @NotNull SafeCloseable seekTo) implements
|
||||||
|
SafeCloseable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
iterator.close();
|
||||||
|
sliceMin.close();
|
||||||
|
sliceMax.close();
|
||||||
|
seekTo.close();
|
||||||
|
}
|
||||||
|
}
|
@ -279,19 +279,16 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<Send<LLEntry>> putMulti(Flux<Send<LLEntry>> entries, boolean getOldValues) {
|
public Mono<Void> putMulti(Flux<Send<LLEntry>> entries) {
|
||||||
return entries.handle((entryToReceive, sink) -> {
|
return entries.doOnNext(entryToReceive -> {
|
||||||
try (var entry = entryToReceive.receive()) {
|
try (var entry = entryToReceive.receive()) {
|
||||||
try (var key = entry.getKey().receive()) {
|
try (var key = entry.getKey().receive()) {
|
||||||
try (var val = entry.getValue().receive()) {
|
try (var val = entry.getValue().receive()) {
|
||||||
var oldValue = mainDb.put(k(key.copy().send()), k(val.send()));
|
mainDb.put(k(key.copy().send()), k(val.send()));
|
||||||
if (oldValue != null && getOldValues) {
|
|
||||||
sink.next(LLEntry.of(key.send(), kk(oldValue)).send());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}).then();
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -3,7 +3,8 @@ package it.cavallium.dbengine;
|
|||||||
import static it.cavallium.dbengine.DbTestUtils.destroyAllocator;
|
import static it.cavallium.dbengine.DbTestUtils.destroyAllocator;
|
||||||
import static it.cavallium.dbengine.DbTestUtils.ensureNoLeaks;
|
import static it.cavallium.dbengine.DbTestUtils.ensureNoLeaks;
|
||||||
import static it.cavallium.dbengine.DbTestUtils.newAllocator;
|
import static it.cavallium.dbengine.DbTestUtils.newAllocator;
|
||||||
import static it.cavallium.dbengine.SyncUtils.*;
|
import static it.cavallium.dbengine.SyncUtils.run;
|
||||||
|
import static it.cavallium.dbengine.SyncUtils.runVoid;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
|
||||||
import io.net5.buffer.api.Buffer;
|
import io.net5.buffer.api.Buffer;
|
||||||
@ -30,9 +31,7 @@ import org.junit.jupiter.api.Test;
|
|||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
import org.junit.jupiter.params.provider.Arguments;
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
import org.junit.jupiter.params.provider.MethodSource;
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
import reactor.core.publisher.Flux;
|
|
||||||
import reactor.core.publisher.Mono;
|
import reactor.core.publisher.Mono;
|
||||||
import reactor.core.scheduler.Schedulers;
|
|
||||||
|
|
||||||
public abstract class TestLLDictionary {
|
public abstract class TestLLDictionary {
|
||||||
|
|
||||||
@ -211,17 +210,17 @@ public abstract class TestLLDictionary {
|
|||||||
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
long afterSize;
|
long afterSize;
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, true).then()
|
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, true).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(0, afterSize - beforeSize);
|
assertEquals(0, afterSize - beforeSize);
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, false).then()
|
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode, false).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(0, afterSize - beforeSize);
|
assertEquals(0, afterSize - beforeSize);
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).then()
|
dict.update(keyEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(0, afterSize - beforeSize);
|
assertEquals(0, afterSize - beforeSize);
|
||||||
@ -236,17 +235,17 @@ public abstract class TestLLDictionary {
|
|||||||
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
var beforeSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
long afterSize;
|
long afterSize;
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, true).then()
|
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, true).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(expected, afterSize - beforeSize);
|
assertEquals(expected, afterSize - beforeSize);
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, false).then()
|
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode, false).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(expected, afterSize - beforeSize);
|
assertEquals(expected, afterSize - beforeSize);
|
||||||
runVoid(updateMode == UpdateMode.DISALLOW,
|
runVoid(updateMode == UpdateMode.DISALLOW,
|
||||||
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).then()
|
dict.update(keyNonEx, old -> fromString("test-value"), updateReturnMode).doOnNext(Send::close).then()
|
||||||
);
|
);
|
||||||
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
afterSize = run(dict.sizeRange(null, RANGE_ALL, false));
|
||||||
assertEquals(expected, afterSize - beforeSize);
|
assertEquals(expected, afterSize - beforeSize);
|
||||||
|
Loading…
Reference in New Issue
Block a user