CavalliumDBEngine/src/main/java/it/cavallium/dbengine/database/disk/LLLocalDictionary.java

1507 lines
52 KiB
Java
Raw Normal View History

2020-12-07 22:15:18 +01:00
package it.cavallium.dbengine.database.disk;
2022-03-16 13:47:56 +01:00
import static io.netty5.buffer.api.StandardAllocationTypes.OFF_HEAP;
2022-05-12 19:14:27 +02:00
import static it.cavallium.dbengine.database.LLUtils.ALLOW_STATIC_OPTIONS;
2021-09-10 12:13:52 +02:00
import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB;
2021-08-29 23:18:03 +02:00
import static it.cavallium.dbengine.database.LLUtils.fromByteArray;
2022-04-15 16:49:01 +02:00
import static it.cavallium.dbengine.database.LLUtils.isBoundedRange;
2022-03-16 19:19:26 +01:00
import static it.cavallium.dbengine.database.LLUtils.isReadOnlyDirect;
import static it.cavallium.dbengine.database.LLUtils.toStringSafe;
import static it.cavallium.dbengine.database.disk.UpdateAtomicResultMode.DELTA;
2021-08-29 23:18:03 +02:00
import static java.util.Objects.requireNonNull;
2021-12-30 17:28:06 +01:00
import io.micrometer.core.instrument.Counter;
import io.micrometer.core.instrument.Timer;
2022-03-16 13:47:56 +01:00
import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.BufferAllocator;
2022-03-16 19:19:26 +01:00
import io.netty5.buffer.api.ReadableComponent;
2022-05-12 19:14:27 +02:00
import io.netty5.buffer.api.Resource;
2022-03-16 13:47:56 +01:00
import io.netty5.buffer.api.Send;
import it.cavallium.dbengine.client.BadBlock;
2022-03-02 12:34:30 +01:00
import it.cavallium.dbengine.database.ColumnUtils;
2021-08-29 23:18:03 +02:00
import it.cavallium.dbengine.database.LLDelta;
2021-01-17 18:31:25 +01:00
import it.cavallium.dbengine.database.LLDictionary;
import it.cavallium.dbengine.database.LLDictionaryResultType;
2021-08-28 22:42:51 +02:00
import it.cavallium.dbengine.database.LLEntry;
2021-01-30 00:24:55 +01:00
import it.cavallium.dbengine.database.LLRange;
2021-01-17 18:31:25 +01:00
import it.cavallium.dbengine.database.LLSnapshot;
import it.cavallium.dbengine.database.LLUtils;
2021-09-01 00:01:56 +02:00
import it.cavallium.dbengine.database.SafeCloseable;
2021-02-13 01:31:24 +01:00
import it.cavallium.dbengine.database.UpdateMode;
2021-05-08 03:09:00 +02:00
import it.cavallium.dbengine.database.UpdateReturnMode;
2022-05-12 19:14:27 +02:00
import it.cavallium.dbengine.database.disk.rocksdb.RocksObj;
2021-11-08 10:49:59 +01:00
import it.cavallium.dbengine.database.serialization.KVSerializationFunction;
2022-03-02 12:34:30 +01:00
import it.cavallium.dbengine.rpc.current.data.DatabaseOptions;
2020-12-07 22:15:18 +01:00
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
2021-07-17 11:52:08 +02:00
import java.util.Optional;
2021-03-18 19:53:32 +01:00
import java.util.concurrent.Callable;
import java.util.concurrent.ForkJoinPool;
import java.util.concurrent.ForkJoinTask;
2020-12-07 22:15:18 +01:00
import java.util.function.Function;
2021-03-18 19:53:32 +01:00
import java.util.stream.IntStream;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
2021-12-27 17:34:44 +01:00
import org.apache.logging.log4j.util.Supplier;
2020-12-07 22:15:18 +01:00
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
2022-05-12 19:14:27 +02:00
import org.rocksdb.AbstractNativeReference;
import org.rocksdb.AbstractSlice;
2020-12-07 22:15:18 +01:00
import org.rocksdb.ColumnFamilyHandle;
2021-03-20 12:41:11 +01:00
import org.rocksdb.CompactRangeOptions;
import org.rocksdb.DirectSlice;
2020-12-07 22:15:18 +01:00
import org.rocksdb.FlushOptions;
import org.rocksdb.ReadOptions;
import org.rocksdb.RocksDBException;
2021-03-13 19:01:36 +01:00
import org.rocksdb.Slice;
2020-12-07 22:15:18 +01:00
import org.rocksdb.Snapshot;
2021-05-02 19:18:15 +02:00
import org.rocksdb.WriteBatch;
2020-12-07 22:15:18 +01:00
import org.rocksdb.WriteOptions;
2021-01-30 00:24:55 +01:00
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
2021-07-17 11:52:08 +02:00
import reactor.util.function.Tuple2;
2021-04-03 19:09:06 +02:00
import reactor.util.function.Tuple3;
import reactor.util.function.Tuples;
2020-12-07 22:15:18 +01:00
public class LLLocalDictionary implements LLDictionary {
protected static final Logger logger = LogManager.getLogger(LLLocalDictionary.class);
2021-03-18 19:53:32 +01:00
private static final boolean USE_CURRENT_FASTSIZE_FOR_OLD_SNAPSHOTS = false;
2020-12-07 22:15:18 +01:00
static final int RESERVED_WRITE_BATCH_SIZE = 2 * 1024 * 1024; // 2MiB
static final long MAX_WRITE_BATCH_SIZE = 1024L * 1024L * 1024L; // 1GiB
static final int CAPPED_WRITE_BATCH_CAP = 50000; // 50K operations
static final int MULTI_GET_WINDOW = 16;
2022-05-12 19:14:27 +02:00
private static final RocksObj<ReadOptions> EMPTY_READ_OPTIONS = LLUtils.ALLOW_STATIC_OPTIONS ? new RocksObj<>(new ReadOptions()) : null;
2022-03-24 21:14:17 +01:00
static final boolean PREFER_AUTO_SEEK_BOUND = false;
/**
2021-08-16 10:36:54 +02:00
* It used to be false,
* now it's true to avoid crashes during iterations on completely corrupted files
*/
2022-04-15 16:49:01 +02:00
static final boolean VERIFY_CHECKSUMS_WHEN_NOT_NEEDED = !LLUtils.FORCE_DISABLE_CHECKSUM_VERIFICATION;
2021-05-03 02:45:29 +02:00
/**
* Default: true. Use false to debug problems with windowing.
*/
static final boolean USE_WINDOW_IN_SET_RANGE = true;
2021-05-02 19:18:15 +02:00
/**
* Default: true. Use false to debug problems with write batches.
*/
2021-05-03 12:29:15 +02:00
static final boolean USE_WRITE_BATCHES_IN_PUT_MULTI = true;
/**
* Default: true. Use false to debug problems with write batches.
*/
static final boolean USE_WRITE_BATCHES_IN_SET_RANGE = true;
2021-05-02 19:18:15 +02:00
/**
* Default: true. Use false to debug problems with capped write batches.
*/
2021-05-03 02:45:29 +02:00
static final boolean USE_CAPPED_WRITE_BATCH_IN_SET_RANGE = true;
2021-05-03 12:29:15 +02:00
/**
* Default: true. Use false to debug problems with write batches deletes.
*/
static final boolean USE_WRITE_BATCH_IN_SET_RANGE_DELETE = false;
2021-03-18 19:53:32 +01:00
static final boolean PARALLEL_EXACT_SIZE = true;
2020-12-07 22:15:18 +01:00
2021-01-30 00:24:55 +01:00
private static final byte[] FIRST_KEY = new byte[]{};
/**
* 1KiB dummy buffer, write only, used for debugging purposes
*/
private static final ByteBuffer DUMMY_WRITE_ONLY_BYTE_BUFFER = ByteBuffer.allocateDirect(1024);
2021-10-20 01:51:34 +02:00
private final RocksDBColumn db;
2022-05-12 19:14:27 +02:00
private final RocksObj<ColumnFamilyHandle> cfh;
2020-12-07 22:15:18 +01:00
private final String databaseName;
2021-06-26 02:35:33 +02:00
private final String columnName;
2022-04-05 13:58:12 +02:00
private final Scheduler dbWScheduler;
private final Scheduler dbRScheduler;
2022-05-12 19:14:27 +02:00
private final Function<LLSnapshot, RocksObj<Snapshot>> snapshotResolver;
2021-02-13 01:31:24 +01:00
private final UpdateMode updateMode;
2021-10-30 12:39:56 +02:00
private final boolean nettyDirect;
private final BufferAllocator alloc;
2020-12-07 22:15:18 +01:00
2021-12-30 17:28:06 +01:00
private final Counter startedUpdates;
private final Counter endedUpdates;
private final Timer updateTime;
private final Counter startedGet;
private final Counter endedGet;
private final Timer getTime;
private final Counter startedContains;
private final Counter endedContains;
private final Timer containsTime;
private final Counter startedPut;
private final Counter endedPut;
private final Timer putTime;
private final Counter startedRemove;
private final Counter endedRemove;
private final Timer removeTime;
2022-04-05 13:58:12 +02:00
public LLLocalDictionary(BufferAllocator allocator,
2021-10-20 01:51:34 +02:00
@NotNull RocksDBColumn db,
2020-12-07 22:15:18 +01:00
String databaseName,
2021-06-26 02:35:33 +02:00
String columnName,
2022-04-05 13:58:12 +02:00
Scheduler dbWScheduler,
Scheduler dbRScheduler,
2022-05-12 19:14:27 +02:00
Function<LLSnapshot, RocksObj<Snapshot>> snapshotResolver,
2021-06-29 23:31:02 +02:00
UpdateMode updateMode,
DatabaseOptions databaseOptions) {
2021-08-29 23:18:03 +02:00
requireNonNull(db);
2020-12-07 22:15:18 +01:00
this.db = db;
2021-10-20 01:51:34 +02:00
this.cfh = db.getColumnFamilyHandle();
2020-12-07 22:15:18 +01:00
this.databaseName = databaseName;
2021-06-26 02:35:33 +02:00
this.columnName = columnName;
2022-04-05 13:58:12 +02:00
this.dbWScheduler = dbWScheduler;
this.dbRScheduler = dbRScheduler;
2020-12-07 22:15:18 +01:00
this.snapshotResolver = snapshotResolver;
2021-02-13 01:31:24 +01:00
this.updateMode = updateMode;
2021-05-03 21:41:51 +02:00
alloc = allocator;
2021-10-30 12:39:56 +02:00
this.nettyDirect = databaseOptions.allowNettyDirect() && alloc.getAllocationType() == OFF_HEAP;
2021-12-30 17:28:06 +01:00
var meterRegistry = db.getMeterRegistry();
this.startedGet = meterRegistry.counter("db.read.map.get.started.counter", "db.name", databaseName, "db.column", columnName);
this.endedGet = meterRegistry.counter("db.read.map.get.ended.counter", "db.name", databaseName, "db.column", columnName);
this.getTime = Timer
2021-12-30 18:20:56 +01:00
.builder("db.read.map.get.timer")
2021-12-30 17:28:06 +01:00
.publishPercentiles(0.2, 0.5, 0.95)
.publishPercentileHistogram()
.tags("db.name", databaseName, "db.column", columnName)
.register(meterRegistry);
2021-12-30 18:20:56 +01:00
this.startedContains = meterRegistry.counter("db.read.map.contains.started.counter", "db.name", databaseName, "db.column", columnName);
this.endedContains = meterRegistry.counter("db.read.map.contains.ended.counter", "db.name", databaseName, "db.column", columnName);
2021-12-30 17:28:06 +01:00
this.containsTime = Timer
2021-12-30 18:20:56 +01:00
.builder("db.read.map.contains.timer")
2021-12-30 17:28:06 +01:00
.publishPercentiles(0.2, 0.5, 0.95)
.publishPercentileHistogram()
.tags("db.name", databaseName, "db.column", columnName)
.register(meterRegistry);
this.startedUpdates = meterRegistry.counter("db.write.map.update.started.counter", "db.name", databaseName, "db.column", columnName);
this.endedUpdates = meterRegistry.counter("db.write.map.update.ended.counter", "db.name", databaseName, "db.column", columnName);
this.updateTime = Timer
.builder("db.write.map.update.timer")
.publishPercentiles(0.2, 0.5, 0.95)
.publishPercentileHistogram()
.tags("db.name", databaseName, "db.column", columnName)
.register(meterRegistry);
this.startedPut = meterRegistry.counter("db.write.map.put.started.counter", "db.name", databaseName, "db.column", columnName);
this.endedPut = meterRegistry.counter("db.write.map.put.ended.counter", "db.name", databaseName, "db.column", columnName);
this.putTime = Timer
.builder("db.write.map.put.timer")
.publishPercentiles(0.2, 0.5, 0.95)
.publishPercentileHistogram()
.tags("db.name", databaseName, "db.column", columnName)
.register(meterRegistry);
this.startedRemove = meterRegistry.counter("db.write.map.remove.started.counter", "db.name", databaseName, "db.column", columnName);
this.endedRemove = meterRegistry.counter("db.write.map.remove.ended.counter", "db.name", databaseName, "db.column", columnName);
this.removeTime = Timer
.builder("db.write.map.remove.timer")
.publishPercentiles(0.2, 0.5, 0.95)
.publishPercentileHistogram()
.tags("db.name", databaseName, "db.column", columnName)
.register(meterRegistry);
2020-12-07 22:15:18 +01:00
}
@Override
public String getDatabaseName() {
return databaseName;
}
2021-06-26 02:35:33 +02:00
public String getColumnName() {
return columnName;
}
2022-05-10 16:57:41 +02:00
@NotNull
2022-05-12 19:14:27 +02:00
private RocksObj<ReadOptions> generateReadOptionsOrStatic(LLSnapshot snapshot) {
var resolved = generateReadOptions(snapshot != null ? snapshotResolver.apply(snapshot) : null, true);
if (resolved != null) {
return resolved;
} else {
return new RocksObj<>(new ReadOptions());
}
2020-12-07 22:15:18 +01:00
}
2022-05-10 16:57:41 +02:00
@Nullable
2022-05-12 19:14:27 +02:00
private RocksObj<ReadOptions> generateReadOptionsOrNull(LLSnapshot snapshot) {
2022-05-10 16:57:41 +02:00
return generateReadOptions(snapshot != null ? snapshotResolver.apply(snapshot) : null, false);
}
2022-05-11 00:29:42 +02:00
@NotNull
2022-05-12 19:14:27 +02:00
private RocksObj<ReadOptions> generateReadOptionsOrNew(LLSnapshot snapshot) {
2022-05-11 00:29:42 +02:00
var result = generateReadOptions(snapshot != null ? snapshotResolver.apply(snapshot) : null, false);
if (result != null) {
return result;
} else {
2022-05-12 19:14:27 +02:00
return new RocksObj<>(new ReadOptions());
2022-05-11 00:29:42 +02:00
}
}
2022-05-12 19:14:27 +02:00
private RocksObj<ReadOptions> generateReadOptions(RocksObj<Snapshot> snapshot, boolean orStaticOpts) {
2020-12-07 22:15:18 +01:00
if (snapshot != null) {
2022-05-12 19:14:27 +02:00
return new RocksObj<>(new ReadOptions().setSnapshot(snapshot.v()));
} else if (ALLOW_STATIC_OPTIONS && orStaticOpts) {
2020-12-07 22:15:18 +01:00
return EMPTY_READ_OPTIONS;
2022-05-10 16:57:41 +02:00
} else {
return null;
2020-12-07 22:15:18 +01:00
}
}
@Override
2021-08-29 23:18:03 +02:00
public BufferAllocator getAllocator() {
return alloc;
}
2022-04-05 13:58:12 +02:00
private <T> @NotNull Mono<T> runOnDb(boolean write, Callable<@Nullable T> callable) {
return Mono.fromCallable(callable).subscribeOn(write ? dbWScheduler : dbRScheduler);
}
2021-05-12 01:25:59 +02:00
@Override
2022-03-24 21:14:17 +01:00
public Mono<Send<Buffer>> get(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
return keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbRScheduler)
.<Send<Buffer>>handle((keySend, sink) -> {
2021-08-29 23:18:03 +02:00
try (var key = keySend.receive()) {
logger.trace(MARKER_ROCKSDB, "Reading {}", () -> toStringSafe(key));
2021-08-29 23:18:03 +02:00
try {
2022-05-10 16:57:41 +02:00
var readOptions = generateReadOptionsOrStatic(snapshot);
2021-12-30 17:28:06 +01:00
Buffer result;
startedGet.increment();
try {
2022-03-02 18:33:58 +01:00
result = getTime.recordCallable(() -> db.get(readOptions, key));
2021-12-30 17:28:06 +01:00
} finally {
endedGet.increment();
2022-05-10 16:57:41 +02:00
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
}
2021-12-30 17:28:06 +01:00
}
logger.trace(MARKER_ROCKSDB, "Read {}: {}", () -> toStringSafe(key), () -> toStringSafe(result));
if (result != null) {
sink.next(result.send());
2021-08-29 23:18:03 +02:00
} else {
sink.complete();
2021-08-29 23:18:03 +02:00
}
} catch (RocksDBException ex) {
sink.error(new IOException("Failed to read " + toStringSafe(key) + ": " + ex.getMessage()));
2021-08-29 23:18:03 +02:00
} catch (Exception ex) {
sink.error(ex);
2021-05-12 01:25:59 +02:00
}
}
});
}
2020-12-07 22:15:18 +01:00
@Override
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fillCache) {
2022-04-05 13:58:12 +02:00
return rangeMono.publishOn(dbRScheduler).handle((rangeSend, sink) -> {
try (var range = rangeSend.receive()) {
assert !Schedulers.isInNonBlockingThread() : "Called isRangeEmpty in a nonblocking thread";
startedContains.increment();
try {
2022-02-02 23:29:11 +01:00
Boolean isRangeEmpty = containsTime.recordCallable(() -> {
if (range.isSingle()) {
return !containsKey(snapshot, range.getSingleUnsafe());
} else {
// Temporary resources to release after finished
2020-12-07 22:15:18 +01:00
2022-05-10 16:57:41 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(generateReadOptionsOrNull(snapshot),
2022-04-15 16:49:01 +02:00
true,
isBoundedRange(range),
true
)) {
2022-05-12 19:14:27 +02:00
readOpts.v().setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
readOpts.v().setFillCache(fillCache);
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
2022-03-24 21:14:17 +01:00
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
2022-03-16 19:19:26 +01:00
if (nettyDirect && isReadOnlyDirect(range.getMinUnsafe())) {
var seekBuf = ((ReadableComponent) range.getMinUnsafe()).readableBuffer();
rocksIterator.seek(seekBuf);
} else {
var seekArray = LLUtils.toArray(range.getMinUnsafe());
rocksIterator.seek(seekArray);
}
} else {
rocksIterator.seekToFirst();
}
2022-02-02 23:29:11 +01:00
return !rocksIterator.isValid();
}
2021-12-30 17:28:06 +01:00
}
}
});
2022-02-02 23:29:11 +01:00
assert isRangeEmpty != null;
sink.next(isRangeEmpty);
2022-01-26 19:03:51 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to read range " + LLUtils.toStringSafe(range)
+ ": " + ex.getMessage()));
} finally {
endedContains.increment();
}
} catch (Throwable ex) {
sink.error(ex);
}
});
2021-05-12 01:25:59 +02:00
}
private boolean containsKey(@Nullable LLSnapshot snapshot, Buffer key) throws RocksDBException {
2021-12-30 17:28:06 +01:00
startedContains.increment();
try {
var result = containsTime.recordCallable(() -> {
2022-05-10 16:57:41 +02:00
var readOptions = generateReadOptionsOrStatic(snapshot);
try {
return db.exists(readOptions, key);
} finally {
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
}
}
2021-12-30 17:28:06 +01:00
});
assert result != null;
return result;
} catch (RocksDBException | RuntimeException e) {
throw e;
} catch (Exception ex) {
throw new RuntimeException(ex);
} finally {
endedContains.increment();
}
2021-01-30 00:24:55 +01:00
}
@Override
public Mono<Send<Buffer>> put(Mono<Send<Buffer>> keyMono, Mono<Send<Buffer>> valueMono,
2021-08-16 10:36:54 +02:00
LLDictionaryResultType resultType) {
// Zip the entry to write to the database
var entryMono = Mono.zip(keyMono, valueMono, Map::entry);
// Obtain the previous value from the database
2022-03-24 21:14:17 +01:00
var previousDataMono = this.getPreviousData(keyMono, resultType);
// Write the new entry to the database
2022-01-26 19:03:51 +01:00
Mono<Send<Buffer>> putMono = entryMono
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 19:03:51 +01:00
.handle((entry, sink) -> {
try (var key = entry.getKey().receive()) {
try (var value = entry.getValue().receive()) {
2022-01-26 21:18:43 +01:00
put(key, value);
sink.complete();
2022-01-26 21:18:43 +01:00
} catch (RocksDBException ex) {
sink.error(ex);
}
}
});
// Read the previous data, then write the new data, then return the previous data
2022-01-26 19:03:51 +01:00
return Flux.concat(previousDataMono, putMono).singleOrEmpty();
2020-12-07 22:15:18 +01:00
}
2022-01-26 21:18:43 +01:00
private void put(Buffer key, Buffer value) throws RocksDBException {
assert key.isAccessible();
assert value.isAccessible();
if (logger.isTraceEnabled(MARKER_ROCKSDB)) {
var varargs = new Supplier<?>[]{() -> toStringSafe(key), () -> toStringSafe(value)};
logger.trace(MARKER_ROCKSDB, "Writing {}: {}", varargs);
}
startedPut.increment();
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
2022-01-26 21:18:43 +01:00
putTime.recordCallable(() -> {
2022-05-12 19:14:27 +02:00
db.put(writeOptions, key, value);
2022-01-26 21:18:43 +01:00
return null;
});
} catch (RocksDBException ex) {
throw new RocksDBException("Failed to write: " + ex.getMessage());
} catch (RuntimeException ex) {
throw ex;
} catch (Exception ex) {
throw new RuntimeException("Failed to write", ex);
} finally {
endedPut.increment();
}
}
2021-05-02 19:18:15 +02:00
@Override
public Mono<UpdateMode> getUpdateMode() {
2022-01-26 19:03:51 +01:00
return Mono.just(updateMode);
2021-05-02 19:18:15 +02:00
}
2021-05-08 03:09:00 +02:00
@SuppressWarnings("DuplicatedCode")
2021-02-06 19:21:31 +01:00
@Override
2021-08-29 23:18:03 +02:00
public Mono<Send<Buffer>> update(Mono<Send<Buffer>> keyMono,
BinarySerializationFunction updater,
2022-03-02 18:33:58 +01:00
UpdateReturnMode updateReturnMode) {
2022-01-26 19:03:51 +01:00
return keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 19:03:51 +01:00
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
2022-01-26 19:03:51 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called update in a nonblocking thread";
if (updateMode == UpdateMode.DISALLOW) {
sink.error(new UnsupportedOperationException("update() is disallowed"));
return;
}
UpdateAtomicResultMode returnMode = switch (updateReturnMode) {
case NOTHING -> UpdateAtomicResultMode.NOTHING;
case GET_NEW_VALUE -> UpdateAtomicResultMode.CURRENT;
case GET_OLD_VALUE -> UpdateAtomicResultMode.PREVIOUS;
};
UpdateAtomicResult result;
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrStatic(null);
2022-01-26 19:03:51 +01:00
startedUpdates.increment();
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
result = updateTime.recordCallable(() ->
db.updateAtomic(readOptions, writeOptions, key, updater, returnMode));
2022-01-26 19:03:51 +01:00
} finally {
endedUpdates.increment();
2022-05-12 19:14:27 +02:00
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
}
2022-01-26 19:03:51 +01:00
}
assert result != null;
var previous = switch (updateReturnMode) {
case NOTHING -> null;
case GET_NEW_VALUE -> ((UpdateAtomicResultCurrent) result).current();
case GET_OLD_VALUE -> ((UpdateAtomicResultPrevious) result).previous();
};
if (previous != null) {
sink.next(previous);
} else {
sink.complete();
}
} catch (Exception ex) {
sink.error(ex);
2021-12-30 17:28:06 +01:00
}
2022-01-26 19:03:51 +01:00
});
2021-10-17 19:52:43 +02:00
}
2021-10-20 01:51:34 +02:00
@SuppressWarnings("DuplicatedCode")
2021-05-08 03:09:00 +02:00
@Override
2021-09-08 00:22:39 +02:00
public Mono<Send<LLDelta>> updateAndGetDelta(Mono<Send<Buffer>> keyMono,
BinarySerializationFunction updater) {
2022-01-26 19:03:51 +01:00
return keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 19:03:51 +01:00
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
2022-01-26 19:03:51 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called update in a nonblocking thread";
if (updateMode == UpdateMode.DISALLOW) {
sink.error(new UnsupportedOperationException("update() is disallowed"));
return;
}
if (updateMode == UpdateMode.ALLOW && !db.supportsTransactions()) {
sink.error(new UnsupportedOperationException("update() is disallowed because the database doesn't support"
+ "safe atomic operations"));
return;
}
UpdateAtomicResult result;
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrStatic(null);
2022-01-26 19:03:51 +01:00
startedUpdates.increment();
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
result = updateTime.recordCallable(() ->
2022-05-12 19:14:27 +02:00
db.updateAtomic(readOptions, writeOptions, key, updater, DELTA));
2022-01-26 19:03:51 +01:00
} finally {
endedUpdates.increment();
2022-05-12 19:14:27 +02:00
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
}
2022-01-26 19:03:51 +01:00
}
assert result != null;
sink.next(((UpdateAtomicResultDelta) result).delta());
} catch (Exception ex) {
sink.error(ex);
2021-12-30 17:28:06 +01:00
}
2022-01-26 19:03:51 +01:00
});
}
2020-12-07 22:15:18 +01:00
@Override
2021-08-29 23:18:03 +02:00
public Mono<Send<Buffer>> remove(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
2022-01-26 19:03:51 +01:00
// Obtain the previous value from the database
2022-03-24 21:14:17 +01:00
Mono<Send<Buffer>> previousDataMono = this.getPreviousData(keyMono, resultType);
2022-01-26 19:03:51 +01:00
// Delete the value from the database
Mono<Send<Buffer>> removeMono = keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 19:03:51 +01:00
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
logger.trace(MARKER_ROCKSDB, "Deleting {}", () -> toStringSafe(key));
startedRemove.increment();
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
2022-01-26 19:03:51 +01:00
removeTime.recordCallable(() -> {
2022-05-12 19:14:27 +02:00
db.delete(writeOptions, key);
2022-01-26 19:03:51 +01:00
return null;
});
} finally {
endedRemove.increment();
}
sink.complete();
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to delete: " + ex.getMessage()));
} catch (Exception ex) {
sink.error(ex);
}
});
// Read the previous data, then delete the data, then return the previous data
return Flux.concat(previousDataMono, removeMono).singleOrEmpty();
2021-01-30 00:24:55 +01:00
}
2020-12-07 22:15:18 +01:00
2022-03-24 21:14:17 +01:00
private Mono<Send<Buffer>> getPreviousData(Mono<Send<Buffer>> keyMono, LLDictionaryResultType resultType) {
2021-08-31 15:50:11 +02:00
return switch (resultType) {
2022-01-26 21:30:08 +01:00
case PREVIOUS_VALUE_EXISTENCE -> keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbRScheduler)
2022-01-26 21:30:08 +01:00
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
var contained = containsKey(null, key);
sink.next(LLUtils.booleanToResponseByteBuffer(alloc, contained));
} catch (RocksDBException ex) {
sink.error(ex);
}
});
2022-01-26 19:03:51 +01:00
case PREVIOUS_VALUE -> keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbRScheduler)
2022-01-26 19:03:51 +01:00
.handle((keySend, sink) -> {
try (var key = keySend.receive()) {
assert !Schedulers.isInNonBlockingThread() : "Called getPreviousData in a nonblocking thread";
2022-05-12 19:14:27 +02:00
Buffer result;
var readOptions = generateReadOptionsOrStatic(null);
try {
result = db.get(readOptions, key);
} finally {
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
}
}
logger.trace(MARKER_ROCKSDB, "Read {}: {}", () -> toStringSafe(key), () -> toStringSafe(result));
2022-01-26 19:03:51 +01:00
if (result == null) {
sink.complete();
} else {
sink.next(result.send());
}
} catch (Exception ex) {
sink.error(ex);
}
});
2021-08-31 15:50:11 +02:00
case VOID -> Mono.empty();
};
}
@Override
2022-03-24 21:14:17 +01:00
public Flux<Optional<Buffer>> getMulti(@Nullable LLSnapshot snapshot, Flux<Send<Buffer>> keys) {
return keys
.buffer(MULTI_GET_WINDOW)
2022-04-05 13:58:12 +02:00
.publishOn(dbRScheduler)
2022-01-26 19:03:51 +01:00
.<ArrayList<Optional<Buffer>>>handle((keysWindow, sink) -> {
2021-11-08 10:49:59 +01:00
List<Buffer> keyBufsWindow = new ArrayList<>(keysWindow.size());
for (Send<Buffer> bufferSend : keysWindow) {
keyBufsWindow.add(bufferSend.receive());
2021-08-29 23:18:03 +02:00
}
2021-11-08 10:49:59 +01:00
try {
2022-01-26 19:03:51 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called getMulti in a nonblocking thread";
2022-05-10 16:57:41 +02:00
ArrayList<Optional<Buffer>> mappedResults;
var readOptions = generateReadOptionsOrStatic(snapshot);
try {
List<byte[]> results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
2022-05-11 00:29:42 +02:00
mappedResults = new ArrayList<>(results.size());
2022-05-10 16:57:41 +02:00
for (int i = 0; i < results.size(); i++) {
byte[] val = results.get(i);
Optional<Buffer> valueOpt;
if (val != null) {
// free memory
results.set(i, null);
valueOpt = Optional.of(LLUtils.fromByteArray(alloc, val));
} else {
valueOpt = Optional.empty();
}
mappedResults.add(valueOpt);
}
} finally {
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
2021-08-29 23:18:03 +02:00
}
2021-11-08 10:49:59 +01:00
}
2022-01-26 19:03:51 +01:00
sink.next(mappedResults);
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to read keys: " + ex.getMessage()));
2021-11-08 10:49:59 +01:00
} finally {
for (Buffer buffer : keyBufsWindow) {
buffer.close();
2021-08-29 23:18:03 +02:00
}
2021-11-08 10:49:59 +01:00
}
})
2022-01-26 19:03:51 +01:00
.flatMapIterable(list -> list);
2021-08-29 23:18:03 +02:00
}
@Override
2022-01-26 19:03:51 +01:00
public Mono<Void> putMulti(Flux<Send<LLEntry>> entries) {
2021-08-29 23:18:03 +02:00
return entries
.buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 19:03:51 +01:00
.handle((entriesWindowList, sink) -> {
var entriesWindow = new ArrayList<LLEntry>(entriesWindowList.size());
for (Send<LLEntry> entrySend : entriesWindowList) {
entriesWindow.add(entrySend.receive());
}
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
2022-01-26 19:03:51 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called putMulti in a nonblocking thread";
if (USE_WRITE_BATCHES_IN_PUT_MULTI) {
2022-05-12 19:14:27 +02:00
try (var batch = new CappedWriteBatch(db,
2022-01-26 19:03:51 +01:00
alloc,
CAPPED_WRITE_BATCH_CAP,
RESERVED_WRITE_BATCH_SIZE,
MAX_WRITE_BATCH_SIZE,
2022-05-12 19:14:27 +02:00
writeOptions
)) {
for (LLEntry entry : entriesWindow) {
var k = entry.getKey();
var v = entry.getValue();
if (nettyDirect) {
batch.put(cfh.v(), k, v);
} else {
try (var key = k.receive()) {
try (var value = v.receive()) {
batch.put(cfh.v(), LLUtils.toArray(key), LLUtils.toArray(value));
}
2021-08-29 23:18:03 +02:00
}
2021-10-17 19:52:43 +02:00
}
2021-05-02 19:18:15 +02:00
}
2022-05-12 19:14:27 +02:00
batch.writeToDbAndClose();
2021-08-29 23:18:03 +02:00
}
2022-01-26 19:03:51 +01:00
} else {
for (LLEntry entry : entriesWindow) {
2022-05-12 19:14:27 +02:00
db.put(writeOptions, entry.getKeyUnsafe(), entry.getValueUnsafe());
2022-01-26 19:03:51 +01:00
}
}
2022-04-12 00:25:18 +02:00
sink.next(true);
2022-01-26 19:03:51 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to write: " + ex.getMessage()));
} finally {
for (LLEntry llEntry : entriesWindow) {
llEntry.close();
}
}
})
.then();
}
2021-07-17 11:52:08 +02:00
@Override
2021-11-08 10:49:59 +01:00
public <K> Flux<Boolean> updateMulti(Flux<K> keys, Flux<Send<Buffer>> serializedKeys,
2021-11-08 16:33:41 +01:00
KVSerializationFunction<K, @Nullable Send<Buffer>, @Nullable Buffer> updateFunction) {
2021-11-08 10:49:59 +01:00
return Flux.zip(keys, serializedKeys)
2021-07-17 11:52:08 +02:00
.buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
2022-04-05 13:58:12 +02:00
.flatMapSequential(ew -> this.<List<Boolean>>runOnDb(true, () -> {
2021-11-08 10:49:59 +01:00
List<Tuple2<K, Buffer>> entriesWindow = new ArrayList<>(ew.size());
for (Tuple2<K, Send<Buffer>> tuple : ew) {
entriesWindow.add(tuple.mapT2(Send::receive));
2021-08-29 23:18:03 +02:00
}
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called updateMulti in a nonblocking thread");
}
2021-08-29 23:18:03 +02:00
List<Buffer> keyBufsWindow = new ArrayList<>(entriesWindow.size());
2021-11-08 10:49:59 +01:00
for (Tuple2<K, Buffer> objects : entriesWindow) {
keyBufsWindow.add(objects.getT2());
2021-08-29 23:18:03 +02:00
}
2021-11-08 10:49:59 +01:00
ArrayList<Tuple3<K, Send<Buffer>, Optional<Send<Buffer>>>> mappedInputs;
2021-10-17 19:52:43 +02:00
{
2022-05-10 16:57:41 +02:00
var readOptions = generateReadOptionsOrStatic(null);
try {
var inputs = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow));
mappedInputs = new ArrayList<>(inputs.size());
for (int i = 0; i < inputs.size(); i++) {
var val = inputs.get(i);
if (val != null) {
inputs.set(i, null);
mappedInputs.add(Tuples.of(
entriesWindow.get(i).getT1(),
keyBufsWindow.get(i).send(),
Optional.of(fromByteArray(alloc, val).send())
));
} else {
mappedInputs.add(Tuples.of(
entriesWindow.get(i).getT1(),
keyBufsWindow.get(i).send(),
Optional.empty()
));
}
}
} finally {
if (readOptions != EMPTY_READ_OPTIONS) {
readOptions.close();
2021-10-17 19:52:43 +02:00
}
2021-08-29 23:18:03 +02:00
}
}
2021-11-08 16:33:41 +01:00
var updatedValuesToWrite = new ArrayList<Buffer>(mappedInputs.size());
2021-11-08 10:49:59 +01:00
var valueChangedResult = new ArrayList<Boolean>(mappedInputs.size());
2021-08-29 23:18:03 +02:00
try {
2021-10-17 19:52:43 +02:00
for (var mappedInput : mappedInputs) {
2021-11-08 16:33:41 +01:00
var updatedValue = updateFunction.apply(mappedInput.getT1(), mappedInput.getT2());
try {
if (updatedValue != null) {
try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) {
valueChangedResult.add(!LLUtils.equals(t3, updatedValue));
2021-11-08 10:49:59 +01:00
}
2021-11-08 16:33:41 +01:00
updatedValuesToWrite.add(updatedValue);
2021-11-08 10:49:59 +01:00
} else {
try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) {
valueChangedResult.add(!LLUtils.equals(t3, null));
}
updatedValuesToWrite.add(null);
2021-07-17 11:52:08 +02:00
}
2021-11-08 16:33:41 +01:00
} catch (Throwable t) {
if (updatedValue != null) {
updatedValue.close();
}
throw t;
2021-08-29 23:18:03 +02:00
}
}
2021-10-17 19:52:43 +02:00
} finally {
for (var mappedInput : mappedInputs) {
mappedInput.getT3().ifPresent(Send::close);
2021-08-29 23:18:03 +02:00
}
2021-10-17 19:52:43 +02:00
}
2021-07-17 11:52:08 +02:00
2021-10-17 19:52:43 +02:00
if (USE_WRITE_BATCHES_IN_PUT_MULTI) {
2022-05-12 19:14:27 +02:00
try (var batch = new CappedWriteBatch(db,
2021-10-17 19:52:43 +02:00
alloc,
CAPPED_WRITE_BATCH_CAP,
RESERVED_WRITE_BATCH_SIZE,
MAX_WRITE_BATCH_SIZE,
2022-05-12 19:14:27 +02:00
writeOptions
)) {
int i = 0;
for (Tuple2<K, Buffer> entry : entriesWindow) {
try (var valueToWrite = updatedValuesToWrite.get(i)) {
if (valueToWrite == null) {
batch.delete(cfh.v(), entry.getT2().send());
} else {
batch.put(cfh.v(), entry.getT2().send(), valueToWrite.send());
}
2021-11-08 16:33:41 +01:00
}
2022-05-12 19:14:27 +02:00
i++;
2021-07-17 11:52:08 +02:00
}
2022-05-12 19:14:27 +02:00
batch.writeToDbAndClose();
2021-10-17 19:52:43 +02:00
}
} else {
int i = 0;
2021-11-08 10:49:59 +01:00
for (Tuple2<K, Buffer> entry : entriesWindow) {
2022-05-12 19:14:27 +02:00
db.put(writeOptions, entry.getT2(), updatedValuesToWrite.get(i));
2021-10-17 19:52:43 +02:00
i++;
2021-08-29 23:18:03 +02:00
}
}
2021-10-17 19:52:43 +02:00
return valueChangedResult;
2021-08-29 23:18:03 +02:00
} finally {
2021-11-08 10:49:59 +01:00
for (Tuple2<K, Buffer> tuple : entriesWindow) {
tuple.getT2().close();
2021-08-29 23:18:03 +02:00
}
}
2022-01-26 14:22:54 +01:00
}).flatMapIterable(list -> list), /* Max concurrency is 2 to update data while preparing the next segment */ 2);
2021-07-17 11:52:08 +02:00
}
2020-12-07 22:15:18 +01:00
@Override
2022-03-24 23:56:23 +01:00
public Flux<Send<LLEntry>> getRange(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
boolean reverse,
boolean smallRange) {
2022-01-26 19:56:51 +01:00
return rangeMono.flatMapMany(rangeSend -> {
try (var range = rangeSend.receive()) {
if (range.isSingle()) {
var rangeSingleMono = rangeMono.map(r -> r.receive().getSingle());
2022-03-24 21:14:17 +01:00
return getRangeSingle(snapshot, rangeSingleMono);
2022-01-26 19:56:51 +01:00
} else {
2022-03-24 23:56:23 +01:00
return getRangeMulti(snapshot, rangeMono, reverse, smallRange);
2022-01-26 19:56:51 +01:00
}
}
});
}
@Override
2021-08-29 23:18:03 +02:00
public Flux<List<Send<LLEntry>>> getRangeGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
2022-03-24 23:56:23 +01:00
int prefixLength,
boolean smallRange) {
2022-01-26 19:56:51 +01:00
return rangeMono.flatMapMany(rangeSend -> {
try (var range = rangeSend.receive()) {
if (range.isSingle()) {
var rangeSingleMono = rangeMono.map(r -> r.receive().getSingle());
2022-03-24 21:14:17 +01:00
return getRangeSingle(snapshot, rangeSingleMono).map(List::of);
2022-01-26 19:56:51 +01:00
} else {
2022-03-24 23:56:23 +01:00
return getRangeMultiGrouped(snapshot, rangeMono, prefixLength, smallRange);
2022-01-26 19:56:51 +01:00
}
}
});
2021-01-30 00:24:55 +01:00
}
2022-03-24 21:14:17 +01:00
private Flux<Send<LLEntry>> getRangeSingle(LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
2021-08-29 23:18:03 +02:00
return Mono
2022-03-24 21:14:17 +01:00
.zip(keyMono, this.get(snapshot, keyMono))
2021-08-29 23:18:03 +02:00
.map(result -> LLEntry.of(result.getT1(), result.getT2()).send())
2022-01-26 14:22:54 +01:00
.flux();
}
2022-03-24 23:56:23 +01:00
private Flux<Send<LLEntry>> getRangeMulti(LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
boolean reverse,
boolean smallRange) {
2022-01-26 19:56:51 +01:00
Mono<LLLocalEntryReactiveRocksIterator> iteratorMono = rangeMono.map(rangeSend -> {
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrNull(snapshot);
2022-05-11 00:29:42 +02:00
return new LLLocalEntryReactiveRocksIterator(db, rangeSend, nettyDirect, readOptions, reverse, smallRange);
2022-01-26 19:56:51 +01:00
});
return Flux.usingWhen(iteratorMono,
2022-04-05 13:58:12 +02:00
iterator -> iterator.flux().subscribeOn(dbRScheduler, false),
2022-01-26 19:56:51 +01:00
iterator -> Mono.fromRunnable(iterator::close)
);
}
2021-10-19 00:22:05 +02:00
private Flux<List<Send<LLEntry>>> getRangeMultiGrouped(LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono,
2022-03-24 23:56:23 +01:00
int prefixLength, boolean smallRange) {
2022-01-26 19:56:51 +01:00
Mono<LLLocalGroupedEntryReactiveRocksIterator> iteratorMono = rangeMono.map(rangeSend -> {
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrNull(snapshot);
2022-03-24 23:56:23 +01:00
return new LLLocalGroupedEntryReactiveRocksIterator(db,
prefixLength,
rangeSend,
nettyDirect,
2022-05-11 00:29:42 +02:00
readOptions,
2022-03-24 23:56:23 +01:00
smallRange
);
2022-01-26 19:56:51 +01:00
});
return Flux.usingWhen(
iteratorMono,
2022-04-05 13:58:12 +02:00
iterator -> iterator.flux().subscribeOn(dbRScheduler, false),
2022-01-26 19:56:51 +01:00
iterator -> Mono.fromRunnable(iterator::close)
);
2021-01-30 00:24:55 +01:00
}
@Override
2022-03-24 23:56:23 +01:00
public Flux<Send<Buffer>> getRangeKeys(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
boolean reverse,
boolean smallRange) {
2022-01-26 19:56:51 +01:00
return rangeMono.flatMapMany(rangeSend -> {
try (var range = rangeSend.receive()) {
if (range.isSingle()) {
return this.getRangeKeysSingle(snapshot, rangeMono.map(r -> r.receive().getSingle()));
} else {
2022-03-24 23:56:23 +01:00
return this.getRangeKeysMulti(snapshot, rangeMono, reverse, smallRange);
2022-01-26 19:56:51 +01:00
}
}
});
}
@Override
2021-08-29 23:18:03 +02:00
public Flux<List<Send<Buffer>>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
2022-03-24 23:56:23 +01:00
int prefixLength,
boolean smallRange) {
2022-01-26 19:56:51 +01:00
Mono<LLLocalGroupedKeyReactiveRocksIterator> iteratorMono = rangeMono.map(rangeSend -> {
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrNull(snapshot);
2022-03-24 23:56:23 +01:00
return new LLLocalGroupedKeyReactiveRocksIterator(db,
prefixLength,
rangeSend,
nettyDirect,
2022-05-11 00:29:42 +02:00
readOptions,
2022-03-24 23:56:23 +01:00
smallRange
);
2022-01-26 19:56:51 +01:00
});
return Flux.usingWhen(iteratorMono,
2022-04-05 13:58:12 +02:00
iterator -> iterator.flux().subscribeOn(dbRScheduler, false),
2022-01-26 19:56:51 +01:00
iterator -> Mono.fromRunnable(iterator::close)
);
2021-03-14 13:24:46 +01:00
}
@Override
2021-08-29 23:18:03 +02:00
public Flux<BadBlock> badBlocks(Mono<Send<LLRange>> rangeMono) {
return Flux.usingWhen(rangeMono,
2021-08-29 23:18:03 +02:00
rangeSend -> Flux
.<BadBlock>create(sink -> {
2021-08-29 23:18:03 +02:00
var range = rangeSend.receive();
sink.onDispose(range::close);
2022-05-10 16:57:41 +02:00
try (var ro = LLUtils.generateCustomReadOptions(null,
2022-04-15 16:49:01 +02:00
false,
isBoundedRange(range),
false
)) {
2022-05-12 19:14:27 +02:00
ro.v().setFillCache(false);
if (!range.isSingle()) {
2022-03-20 14:33:27 +01:00
if (LLUtils.MANUAL_READAHEAD) {
2022-05-12 19:14:27 +02:00
ro.v().setReadaheadSize(32 * 1024);
2022-03-20 14:33:27 +01:00
}
}
2022-05-12 19:14:27 +02:00
ro.v().setVerifyChecksums(true);
try (var rocksIterator = db.newRocksIterator(nettyDirect, ro, range, false)) {
rocksIterator.seekToFirst();
while (rocksIterator.isValid() && !sink.isCancelled()) {
try {
rocksIterator.key(DUMMY_WRITE_ONLY_BYTE_BUFFER);
rocksIterator.value(DUMMY_WRITE_ONLY_BYTE_BUFFER);
rocksIterator.next();
} catch (RocksDBException ex) {
sink.next(new BadBlock(databaseName, ColumnUtils.special(columnName), null, ex));
}
}
}
sink.complete();
} catch (Throwable ex) {
sink.error(ex);
}
})
2022-04-05 13:58:12 +02:00
.subscribeOn(dbRScheduler),
2021-08-29 23:18:03 +02:00
rangeSend -> Mono.fromRunnable(rangeSend::close)
);
}
2021-03-14 13:24:46 +01:00
@Override
2021-10-19 00:22:05 +02:00
public Flux<Send<Buffer>> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono,
2022-03-24 23:56:23 +01:00
int prefixLength, boolean smallRange) {
2022-01-26 19:56:51 +01:00
Mono<LLLocalKeyPrefixReactiveRocksIterator> iteratorMono = rangeMono.map(range -> {
2022-05-12 19:14:27 +02:00
var readOptions = generateReadOptionsOrNull(snapshot);
2022-05-11 00:29:42 +02:00
return new LLLocalKeyPrefixReactiveRocksIterator(db,
prefixLength,
range,
nettyDirect,
readOptions,
true,
2022-03-24 23:56:23 +01:00
smallRange
);
2022-01-26 19:56:51 +01:00
});
return Flux.usingWhen(iteratorMono,
2022-04-05 13:58:12 +02:00
iterator -> iterator.flux().subscribeOn(dbRScheduler),
2022-01-26 19:56:51 +01:00
iterator -> Mono.fromRunnable(iterator::close)
);
}
2021-08-29 23:18:03 +02:00
private Flux<Send<Buffer>> getRangeKeysSingle(LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
2022-01-26 19:56:51 +01:00
return keyMono
2022-04-05 13:58:12 +02:00
.publishOn(dbRScheduler)
2022-01-26 19:56:51 +01:00
.<Send<Buffer>>handle((keySend, sink) -> {
try (var key = keySend.receive()) {
if (containsKey(snapshot, key)) {
sink.next(key.send());
} else {
sink.complete();
}
} catch (Throwable ex) {
sink.error(ex);
}
})
.flux();
2020-12-07 22:15:18 +01:00
}
2022-05-12 19:14:27 +02:00
private record RocksObjTuple<T extends AbstractNativeReference, U extends Resource<?>>(RocksObj<T> t1, U t2) implements SafeCloseable {
@Override
public void close() {
//noinspection EmptyTryBlock
try (t1; t2) {}
}
}
2022-03-24 23:56:23 +01:00
private Flux<Send<Buffer>> getRangeKeysMulti(LLSnapshot snapshot,
Mono<Send<LLRange>> rangeMono,
boolean reverse,
boolean smallRange) {
2022-05-12 19:14:27 +02:00
Mono<RocksObjTuple<ReadOptions, LLLocalKeyReactiveRocksIterator>> iteratorMono = rangeMono.map(range -> {
var readOptions = generateReadOptionsOrNull(snapshot);
var it = new LLLocalKeyReactiveRocksIterator(db, range, nettyDirect, readOptions, reverse, smallRange);
return new RocksObjTuple<>(readOptions, it);
2022-01-26 19:56:51 +01:00
});
return Flux.usingWhen(iteratorMono,
2022-05-12 19:14:27 +02:00
t -> t.t2().flux().subscribeOn(dbRScheduler, false),
t -> Mono.fromRunnable(t::close)
);
}
2020-12-07 22:15:18 +01:00
@Override
2022-03-24 23:56:23 +01:00
public Mono<Void> setRange(Mono<Send<LLRange>> rangeMono, Flux<Send<LLEntry>> entries, boolean smallRange) {
2022-01-26 21:18:43 +01:00
if (USE_WINDOW_IN_SET_RANGE) {
return rangeMono
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-04-12 00:25:18 +02:00
.<Boolean>handle((rangeSend, sink) -> {
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions());
var range = rangeSend.receive()) {
2022-01-26 21:18:43 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called setRange in a nonblocking thread";
if (!USE_WRITE_BATCH_IN_SET_RANGE_DELETE || !USE_WRITE_BATCHES_IN_SET_RANGE) {
2022-05-10 16:57:41 +02:00
try (var opts = LLUtils.generateCustomReadOptions(null,
2022-04-15 16:49:01 +02:00
true,
isBoundedRange(range),
smallRange
)) {
2022-01-26 21:18:43 +01:00
SafeCloseable seekTo;
2022-05-12 19:14:27 +02:00
try (var it = db.newIterator(opts, range.getMinUnsafe(), range.getMaxUnsafe())) {
2022-03-24 21:14:17 +01:00
if (!PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
2022-05-12 19:14:27 +02:00
it.seekTo(range.getMinUnsafe());
2022-01-26 21:18:43 +01:00
} else {
seekTo = null;
it.seekToFirst();
2021-06-19 21:55:20 +02:00
}
2022-05-12 19:14:27 +02:00
while (it.isValid()) {
db.delete(writeOptions, it.key());
it.next();
}
}
2022-01-26 21:18:43 +01:00
}
} else if (USE_CAPPED_WRITE_BATCH_IN_SET_RANGE) {
try (var batch = new CappedWriteBatch(db,
alloc,
CAPPED_WRITE_BATCH_CAP,
RESERVED_WRITE_BATCH_SIZE,
MAX_WRITE_BATCH_SIZE,
2022-05-12 19:14:27 +02:00
writeOptions
2022-01-26 21:18:43 +01:00
)) {
if (range.isSingle()) {
2022-05-12 19:14:27 +02:00
batch.delete(cfh.v(), range.getSingle());
2022-01-26 21:18:43 +01:00
} else {
deleteSmallRangeWriteBatch(batch, range.copy().send());
}
batch.writeToDbAndClose();
}
} else {
try (var batch = new WriteBatch(RESERVED_WRITE_BATCH_SIZE)) {
if (range.isSingle()) {
2022-05-12 19:14:27 +02:00
batch.delete(cfh.v(), LLUtils.toArray(range.getSingleUnsafe()));
2022-01-26 21:18:43 +01:00
} else {
deleteSmallRangeWriteBatch(batch, range.copy().send());
}
2022-05-12 19:14:27 +02:00
db.write(writeOptions, batch);
2022-01-26 21:18:43 +01:00
batch.clear();
}
}
2022-04-12 00:25:18 +02:00
sink.next(true);
2022-01-26 21:18:43 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to set a range: " + ex.getMessage()));
}
})
.thenMany(entries.window(MULTI_GET_WINDOW))
.flatMap(keysWindowFlux -> keysWindowFlux
.collectList()
.flatMap(entriesListSend -> this
2022-04-05 13:58:12 +02:00
.<Void>runOnDb(true, () -> {
2022-01-26 21:18:43 +01:00
List<LLEntry> entriesList = new ArrayList<>(entriesListSend.size());
for (Send<LLEntry> entrySend : entriesListSend) {
entriesList.add(entrySend.receive());
}
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions())) {
2022-01-26 21:18:43 +01:00
if (!USE_WRITE_BATCHES_IN_SET_RANGE) {
for (LLEntry entry : entriesList) {
assert entry.isAccessible();
2022-05-12 19:14:27 +02:00
db.put(writeOptions, entry.getKeyUnsafe(), entry.getValueUnsafe());
2022-01-26 21:18:43 +01:00
}
} else if (USE_CAPPED_WRITE_BATCH_IN_SET_RANGE) {
try (var batch = new CappedWriteBatch(db,
alloc,
CAPPED_WRITE_BATCH_CAP,
RESERVED_WRITE_BATCH_SIZE,
MAX_WRITE_BATCH_SIZE,
2022-05-12 19:14:27 +02:00
writeOptions
2022-01-26 21:18:43 +01:00
)) {
for (LLEntry entry : entriesList) {
assert entry.isAccessible();
if (nettyDirect) {
2022-05-12 19:14:27 +02:00
batch.put(cfh.v(), entry.getKey(), entry.getValue());
} else {
2022-05-12 19:14:27 +02:00
batch.put(cfh.v(),
2022-01-26 21:18:43 +01:00
LLUtils.toArray(entry.getKeyUnsafe()),
LLUtils.toArray(entry.getValueUnsafe())
);
2021-05-12 01:25:59 +02:00
}
2021-05-02 19:18:15 +02:00
}
2022-01-26 21:18:43 +01:00
batch.writeToDbAndClose();
}
} else {
try (var batch = new WriteBatch(RESERVED_WRITE_BATCH_SIZE)) {
for (LLEntry entry : entriesList) {
assert entry.isAccessible();
2022-05-12 19:14:27 +02:00
batch.put(cfh.v(), LLUtils.toArray(entry.getKeyUnsafe()),
2022-01-26 21:18:43 +01:00
LLUtils.toArray(entry.getValueUnsafe()));
}
2022-05-12 19:14:27 +02:00
db.write(writeOptions, batch);
2022-01-26 21:18:43 +01:00
batch.clear();
}
}
return null;
} finally {
for (LLEntry entry : entriesList) {
assert entry.isAccessible();
entry.close();
}
}
})
)
)
.then()
.onErrorMap(cause -> new IOException("Failed to write range", cause));
} else {
if (USE_WRITE_BATCHES_IN_SET_RANGE) {
return Mono.error(() -> new UnsupportedOperationException(
"Can't use write batches in setRange without window. Please fix the parameters"));
}
var deleteMono = this
2022-03-24 23:56:23 +01:00
.getRange(null, rangeMono, false, smallRange)
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 21:18:43 +01:00
.handle((oldValueSend, sink) -> {
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions());
var oldValue = oldValueSend.receive()) {
db.delete(writeOptions, oldValue.getKeyUnsafe());
2022-04-12 00:25:18 +02:00
sink.next(true);
2022-01-26 21:18:43 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to write range: " + ex.getMessage()));
}
2022-01-26 21:18:43 +01:00
})
.then(Mono.<Void>empty());
var putMono = entries
2022-04-05 13:58:12 +02:00
.publishOn(dbWScheduler)
2022-01-26 21:18:43 +01:00
.handle((entrySend, sink) -> {
try (var entry = entrySend.receive()) {
if (entry.getKeyUnsafe() != null && entry.getValueUnsafe() != null) {
this.put(entry.getKeyUnsafe(), entry.getValueUnsafe());
}
2022-04-12 00:25:18 +02:00
sink.next(true);
2022-01-26 21:18:43 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to write range: " + ex.getMessage()));
}
})
.then(Mono.<Void>empty());
return deleteMono.then(putMono);
}
2021-03-14 03:13:19 +01:00
}
//todo: this is broken, check why. (is this still true?)
2021-08-29 23:18:03 +02:00
private void deleteSmallRangeWriteBatch(CappedWriteBatch writeBatch, Send<LLRange> rangeToReceive)
2021-03-20 12:41:11 +01:00
throws RocksDBException {
2022-05-12 19:14:27 +02:00
try (var range = rangeToReceive.receive();
var readOpts = generateReadOptionsOrNew(null)) {
readOpts.v().setFillCache(false);
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2021-05-03 02:57:08 +02:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
2021-05-03 02:57:08 +02:00
}
2022-05-12 19:14:27 +02:00
while (rocksIterator.isValid()) {
writeBatch.delete(cfh.v(), LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send());
rocksIterator.next();
2021-05-03 02:57:08 +02:00
}
2021-03-20 12:41:11 +01:00
}
}
}
2021-08-29 23:18:03 +02:00
private void deleteSmallRangeWriteBatch(WriteBatch writeBatch, Send<LLRange> rangeToReceive)
2021-05-02 19:18:15 +02:00
throws RocksDBException {
2021-08-29 23:18:03 +02:00
try (var range = rangeToReceive.receive()) {
2022-05-11 00:29:42 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(null, true, isBoundedRange(range), true)) {
2022-05-12 19:14:27 +02:00
readOpts.v().setFillCache(false);
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2021-05-03 02:57:08 +02:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
2021-05-03 02:57:08 +02:00
}
2022-05-12 19:14:27 +02:00
while (rocksIterator.isValid()) {
writeBatch.delete(cfh.v(), rocksIterator.key());
rocksIterator.next();
2021-05-03 02:57:08 +02:00
}
}
}
2021-05-02 19:18:15 +02:00
}
}
2021-01-30 00:24:55 +01:00
public Mono<Void> clear() {
return Mono
.<Void>fromCallable(() -> {
2022-01-26 21:18:43 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called clear in a nonblocking thread";
2021-10-17 19:52:43 +02:00
boolean shouldCompactLater = false;
2022-05-12 19:14:27 +02:00
try (var writeOptions = new RocksObj<>(new WriteOptions());
var readOpts = LLUtils.generateCustomReadOptions(null, false, false, false)) {
readOpts.v().setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
2021-06-27 16:52:45 +02:00
// readOpts.setIgnoreRangeDeletions(true);
2022-05-12 19:14:27 +02:00
readOpts.v().setFillCache(false);
2022-03-20 14:33:27 +01:00
if (LLUtils.MANUAL_READAHEAD) {
2022-05-12 19:14:27 +02:00
readOpts.v().setReadaheadSize(32 * 1024); // 32KiB
2022-03-20 14:33:27 +01:00
}
2021-06-27 16:52:45 +02:00
try (CappedWriteBatch writeBatch = new CappedWriteBatch(db,
2021-09-01 00:01:56 +02:00
alloc,
2021-06-27 16:52:45 +02:00
CAPPED_WRITE_BATCH_CAP,
RESERVED_WRITE_BATCH_SIZE,
MAX_WRITE_BATCH_SIZE,
2022-05-12 19:14:27 +02:00
writeOptions
2021-06-27 16:52:45 +02:00
)) {
byte[] firstDeletedKey = null;
byte[] lastDeletedKey = null;
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(readOpts, null, null)) {
2021-10-20 01:51:34 +02:00
// If the database supports transactions, delete each key one by one
if (db.supportsTransactions()) {
2021-10-17 19:52:43 +02:00
rocksIterator.seekToFirst();
while (rocksIterator.isValid()) {
2022-05-12 19:14:27 +02:00
writeBatch.delete(cfh.v(), rocksIterator.key());
2021-10-17 19:52:43 +02:00
rocksIterator.next();
}
} else {
rocksIterator.seekToLast();
2021-06-27 16:52:45 +02:00
2021-10-17 19:52:43 +02:00
if (rocksIterator.isValid()) {
firstDeletedKey = FIRST_KEY;
lastDeletedKey = rocksIterator.key();
2022-05-12 19:14:27 +02:00
writeBatch.deleteRange(cfh.v(), FIRST_KEY, rocksIterator.key());
writeBatch.delete(cfh.v(), rocksIterator.key());
2021-10-17 19:52:43 +02:00
shouldCompactLater = true;
}
2021-06-27 16:52:45 +02:00
}
2021-03-14 03:13:19 +01:00
}
2020-12-07 22:15:18 +01:00
2021-06-27 16:52:45 +02:00
writeBatch.writeToDbAndClose();
2020-12-07 22:15:18 +01:00
2021-10-17 19:52:43 +02:00
if (shouldCompactLater) {
// Compact range
2021-10-20 01:51:34 +02:00
db.suggestCompactRange();
if (lastDeletedKey != null) {
2022-05-12 19:14:27 +02:00
try (var cro = new RocksObj<>(new CompactRangeOptions()
2021-10-20 01:51:34 +02:00
.setAllowWriteStall(false)
.setExclusiveManualCompaction(false)
2022-05-12 19:14:27 +02:00
.setChangeLevel(false))) {
2022-05-11 00:29:42 +02:00
db.compactRange(firstDeletedKey, lastDeletedKey, cro);
}
2021-10-17 19:52:43 +02:00
}
2021-06-27 16:52:45 +02:00
}
2020-12-07 22:15:18 +01:00
2022-05-12 19:14:27 +02:00
try (var fo = new RocksObj<>(new FlushOptions().setWaitForFlush(true).setAllowWriteStall(true))) {
2022-05-11 00:29:42 +02:00
db.flush(fo);
}
2021-06-27 16:52:45 +02:00
db.flushWal(true);
}
return null;
2020-12-07 22:15:18 +01:00
}
2021-01-30 00:24:55 +01:00
})
.onErrorMap(cause -> new IOException("Failed to clear", cause))
2022-04-05 13:58:12 +02:00
.subscribeOn(dbWScheduler);
2020-12-07 22:15:18 +01:00
}
@Override
2021-08-29 23:18:03 +02:00
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fast) {
2022-04-05 13:58:12 +02:00
return rangeMono.publishOn(dbRScheduler).handle((rangeSend, sink) -> {
2021-10-19 00:22:05 +02:00
try (var range = rangeSend.receive()) {
2022-01-26 21:18:43 +01:00
assert !Schedulers.isInNonBlockingThread() : "Called sizeRange in a nonblocking thread";
2021-10-19 00:22:05 +02:00
if (range.isAll()) {
2022-01-26 21:18:43 +01:00
sink.next(fast ? fastSizeAll(snapshot) : exactSizeAll(snapshot));
2021-10-19 00:22:05 +02:00
} else {
2022-05-11 00:29:42 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(generateReadOptionsOrNull(snapshot),
false,
isBoundedRange(range),
false
)) {
2022-05-12 19:14:27 +02:00
readOpts.v().setFillCache(false);
readOpts.v().setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
if (fast) {
readOpts.v().setIgnoreRangeDeletions(true);
2021-10-19 00:22:05 +02:00
}
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2021-10-16 01:49:41 +02:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
2021-10-19 00:22:05 +02:00
}
2022-05-12 19:14:27 +02:00
long i = 0;
while (rocksIterator.isValid()) {
rocksIterator.next();
i++;
2021-10-16 01:49:41 +02:00
}
2022-05-12 19:14:27 +02:00
sink.next(i);
2021-08-29 23:18:03 +02:00
}
2021-10-19 00:22:05 +02:00
}
}
2022-01-26 21:18:43 +01:00
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to get size of range: " + ex.getMessage()));
2021-10-19 00:22:05 +02:00
}
2022-01-26 21:18:43 +01:00
});
2021-05-12 01:25:59 +02:00
}
@Override
2021-08-29 23:18:03 +02:00
public Mono<Send<LLEntry>> getOne(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
2022-04-05 13:58:12 +02:00
return rangeMono.publishOn(dbRScheduler).handle((rangeSend, sink) -> {
2022-01-26 21:18:43 +01:00
try (var range = rangeSend.receive()) {
assert !Schedulers.isInNonBlockingThread() : "Called getOne in a nonblocking thread";
2022-05-10 16:57:41 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(generateReadOptionsOrNull(snapshot), true, true, true)) {
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2022-01-26 21:18:43 +01:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
}
2022-05-12 19:14:27 +02:00
if (rocksIterator.isValid()) {
try (var key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key)) {
try (var value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value)) {
sink.next(LLEntry.of(key.send(), value.send()).send());
2022-01-26 21:18:43 +01:00
}
2021-03-14 13:08:03 +01:00
}
2022-05-12 19:14:27 +02:00
} else {
sink.complete();
2021-03-14 13:08:03 +01:00
}
}
2022-01-26 21:18:43 +01:00
}
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to get one entry: " + ex.getMessage()));
}
});
}
@Override
2021-08-29 23:18:03 +02:00
public Mono<Send<Buffer>> getOneKey(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
2022-04-05 13:58:12 +02:00
return rangeMono.publishOn(dbRScheduler).handle((rangeSend, sink) -> {
2022-01-26 21:18:43 +01:00
try (var range = rangeSend.receive()) {
assert !Schedulers.isInNonBlockingThread() : "Called getOneKey in a nonblocking thread";
2022-05-10 16:57:41 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(generateReadOptionsOrNull(snapshot), true, true, true)) {
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2022-01-26 21:18:43 +01:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
}
2022-05-12 19:14:27 +02:00
if (rocksIterator.isValid()) {
sink.next(LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send());
} else {
sink.complete();
}
}
2022-01-26 21:18:43 +01:00
}
} catch (RocksDBException ex) {
sink.error(new RocksDBException("Failed to get one key: " + ex.getMessage()));
}
});
}
private long fastSizeAll(@Nullable LLSnapshot snapshot) throws RocksDBException {
2022-05-11 00:29:42 +02:00
try (var rocksdbSnapshot = generateReadOptionsOrNew(snapshot)) {
2022-05-12 19:14:27 +02:00
if (USE_CURRENT_FASTSIZE_FOR_OLD_SNAPSHOTS || rocksdbSnapshot.v().snapshot() == null) {
2021-06-19 21:55:20 +02:00
try {
2021-10-20 01:51:34 +02:00
return db.getLongProperty("rocksdb.estimate-num-keys");
2021-06-19 21:55:20 +02:00
} catch (RocksDBException e) {
2021-09-10 12:13:52 +02:00
logger.error(MARKER_ROCKSDB, "Failed to get RocksDB estimated keys count property", e);
2021-06-19 21:55:20 +02:00
return 0;
}
} else if (PARALLEL_EXACT_SIZE) {
return exactSizeAll(snapshot);
} else {
2022-05-12 19:14:27 +02:00
rocksdbSnapshot.v().setFillCache(false);
rocksdbSnapshot.v().setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
rocksdbSnapshot.v().setIgnoreRangeDeletions(true);
2021-06-19 21:55:20 +02:00
long count = 0;
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(rocksdbSnapshot, null, null)) {
rocksIterator.seekToFirst();
2021-06-19 21:55:20 +02:00
// If it's a fast size of a snapshot, count only up to 100'000 elements
while (rocksIterator.isValid() && count < 100_000) {
2021-06-19 21:55:20 +02:00
count++;
rocksIterator.next();
2021-06-19 21:55:20 +02:00
}
return count;
2020-12-07 22:15:18 +01:00
}
}
}
}
2021-01-30 00:24:55 +01:00
private long exactSizeAll(@Nullable LLSnapshot snapshot) {
if (Schedulers.isInNonBlockingThread()) {
throw new UnsupportedOperationException("Called exactSizeAll in a nonblocking thread");
}
2022-05-10 16:57:41 +02:00
try (var readOpts = LLUtils.generateCustomReadOptions(generateReadOptionsOrNull(snapshot), false, false, false)) {
2022-03-20 14:33:27 +01:00
if (LLUtils.MANUAL_READAHEAD) {
2022-05-12 19:14:27 +02:00
readOpts.v().setReadaheadSize(128 * 1024); // 128KiB
2022-03-20 14:33:27 +01:00
}
2022-05-12 19:14:27 +02:00
readOpts.v().setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
2021-06-19 21:55:20 +02:00
if (PARALLEL_EXACT_SIZE) {
var commonPool = ForkJoinPool.commonPool();
var futures = IntStream
.range(-1, LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS.length)
.mapToObj(idx -> Pair.of(idx == -1 ? new byte[0] : LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS[idx],
idx + 1 >= LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS.length ? null
: LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS[idx + 1]
))
.map(range -> (Callable<Long>) () -> {
long partialCount = 0;
2022-05-12 19:14:27 +02:00
try (var rangeReadOpts = new RocksObj<>(new ReadOptions(readOpts.v()))) {
2021-06-19 21:55:20 +02:00
Slice sliceBegin;
if (range.getKey() != null) {
sliceBegin = new Slice(range.getKey());
} else {
sliceBegin = null;
}
Slice sliceEnd;
if (range.getValue() != null) {
sliceEnd = new Slice(range.getValue());
} else {
sliceEnd = null;
}
try {
if (sliceBegin != null) {
2022-05-12 19:14:27 +02:00
rangeReadOpts.v().setIterateLowerBound(sliceBegin);
2021-06-19 21:55:20 +02:00
}
2022-05-11 20:32:56 +02:00
if (sliceEnd != null) {
2022-05-12 19:14:27 +02:00
rangeReadOpts.v().setIterateUpperBound(sliceEnd);
2021-06-19 21:55:20 +02:00
}
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(rangeReadOpts, null, null)) {
rocksIterator.seekToFirst();
while (rocksIterator.isValid()) {
2021-06-19 21:55:20 +02:00
partialCount++;
rocksIterator.next();
2021-06-19 21:55:20 +02:00
}
return partialCount;
}
} finally {
if (sliceBegin != null) {
sliceBegin.close();
}
if (sliceEnd != null) {
sliceEnd.close();
}
2021-03-18 19:53:32 +01:00
}
}
2021-06-19 21:55:20 +02:00
})
.map(commonPool::submit)
.toList();
2021-06-19 21:55:20 +02:00
long count = 0;
for (ForkJoinTask<Long> future : futures) {
count += future.join();
2021-03-18 19:53:32 +01:00
}
return count;
2021-06-19 21:55:20 +02:00
} else {
long count = 0;
2022-05-12 19:14:27 +02:00
try (var rocksIterator = db.newIterator(readOpts, null, null)) {
rocksIterator.seekToFirst();
while (rocksIterator.isValid()) {
2021-06-19 21:55:20 +02:00
count++;
rocksIterator.next();
2021-06-19 21:55:20 +02:00
}
return count;
} catch (RocksDBException ex) {
throw new IllegalStateException("Failed to read exact size", ex);
2021-06-19 21:55:20 +02:00
}
2021-03-18 19:53:32 +01:00
}
2020-12-07 22:15:18 +01:00
}
}
@Override
2021-08-29 23:18:03 +02:00
public Mono<Send<LLEntry>> removeOne(Mono<Send<LLRange>> rangeMono) {
2022-04-05 13:58:12 +02:00
return rangeMono.publishOn(dbWScheduler).handle((rangeSend, sink) -> {
2022-01-26 21:18:43 +01:00
try (var range = rangeSend.receive()) {
assert !Schedulers.isInNonBlockingThread() : "Called removeOne in a nonblocking thread";
2022-05-12 19:14:27 +02:00
try (var readOpts = new RocksObj<>(new ReadOptions());
var writeOpts = new RocksObj<>(new WriteOptions())) {
try (var rocksIterator = db.newIterator(readOpts, range.getMinUnsafe(), range.getMaxUnsafe())) {
if (!LLLocalDictionary.PREFER_AUTO_SEEK_BOUND && range.hasMin()) {
rocksIterator.seekTo(range.getMinUnsafe());
2022-01-26 21:18:43 +01:00
} else {
2022-05-12 19:14:27 +02:00
rocksIterator.seekToFirst();
}
2022-05-12 19:14:27 +02:00
if (!rocksIterator.isValid()) {
sink.complete();
return;
2021-01-30 00:24:55 +01:00
}
2022-05-12 19:14:27 +02:00
Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key);
Buffer value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value);
db.delete(writeOpts, key);
sink.next(LLEntry.of(key, value).send());
}
2022-01-26 21:18:43 +01:00
}
} catch (RocksDBException ex) {
sink.error(ex);
}
});
2020-12-07 22:15:18 +01:00
}
2021-04-03 19:09:06 +02:00
2020-12-07 22:15:18 +01:00
}