2020-12-07 22:15:18 +01:00
|
|
|
package it.cavallium.dbengine.database.disk;
|
|
|
|
|
2021-01-17 18:31:25 +01:00
|
|
|
import it.cavallium.dbengine.database.LLDictionary;
|
|
|
|
import it.cavallium.dbengine.database.LLDictionaryResultType;
|
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-02-13 01:31:24 +01:00
|
|
|
import it.cavallium.dbengine.database.UpdateMode;
|
2021-02-06 19:21:31 +01:00
|
|
|
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
2021-02-01 02:21:53 +01:00
|
|
|
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.io.IOException;
|
2021-02-01 02:21:53 +01:00
|
|
|
import java.util.ArrayList;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Arrays;
|
2021-02-01 02:21:53 +01:00
|
|
|
import java.util.List;
|
2020-12-07 22:15:18 +01:00
|
|
|
import java.util.Map;
|
|
|
|
import java.util.Map.Entry;
|
|
|
|
import java.util.Objects;
|
2021-02-06 19:21:31 +01: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;
|
2021-02-13 00:18:57 +01:00
|
|
|
import java.util.concurrent.locks.StampedLock;
|
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.Collectors;
|
|
|
|
import java.util.stream.IntStream;
|
|
|
|
import org.apache.commons.lang3.tuple.Pair;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.jetbrains.annotations.NotNull;
|
|
|
|
import org.jetbrains.annotations.Nullable;
|
|
|
|
import org.rocksdb.ColumnFamilyHandle;
|
2021-03-20 12:41:11 +01:00
|
|
|
import org.rocksdb.CompactRangeOptions;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.rocksdb.FlushOptions;
|
|
|
|
import org.rocksdb.Holder;
|
|
|
|
import org.rocksdb.ReadOptions;
|
|
|
|
import org.rocksdb.RocksDB;
|
|
|
|
import org.rocksdb.RocksDBException;
|
|
|
|
import org.rocksdb.RocksIterator;
|
2021-03-13 19:01:36 +01:00
|
|
|
import org.rocksdb.Slice;
|
2020-12-07 22:15:18 +01:00
|
|
|
import org.rocksdb.Snapshot;
|
|
|
|
import org.rocksdb.WriteOptions;
|
|
|
|
import org.warp.commonutils.concurrency.atomicity.NotAtomic;
|
2021-02-06 19:21:31 +01:00
|
|
|
import org.warp.commonutils.locks.Striped;
|
2021-02-26 14:06:16 +01:00
|
|
|
import org.warp.commonutils.log.Logger;
|
|
|
|
import org.warp.commonutils.log.LoggerFactory;
|
2021-01-30 00:24:55 +01:00
|
|
|
import reactor.core.publisher.Flux;
|
|
|
|
import reactor.core.publisher.Mono;
|
2021-02-01 02:21:53 +01:00
|
|
|
import reactor.core.scheduler.Scheduler;
|
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
|
|
|
|
|
|
|
@NotAtomic
|
|
|
|
public class LLLocalDictionary implements LLDictionary {
|
|
|
|
|
2021-02-02 15:36:11 +01:00
|
|
|
protected static final Logger logger = LoggerFactory.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
|
2021-02-01 02:21:53 +01:00
|
|
|
static final int MULTI_GET_WINDOW = 500;
|
2020-12-07 22:15:18 +01:00
|
|
|
static final WriteOptions BATCH_WRITE_OPTIONS = new WriteOptions().setLowPri(true);
|
2021-03-14 19:38:20 +01:00
|
|
|
static final boolean PREFER_SEEK_TO_FIRST = false;
|
|
|
|
static final boolean VERIFY_CHECKSUMS_WHEN_NOT_NEEDED = false;
|
|
|
|
public static final boolean DEBUG_PREFIXES_WHEN_ASSERTIONS_ARE_ENABLED = true;
|
2021-03-18 19:53:32 +01:00
|
|
|
static final boolean PARALLEL_EXACT_SIZE = true;
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-02-15 00:15:42 +01:00
|
|
|
private static final int STRIPES = 512;
|
2021-01-30 00:24:55 +01:00
|
|
|
private static final byte[] FIRST_KEY = new byte[]{};
|
2020-12-07 22:15:18 +01:00
|
|
|
private static final byte[] NO_DATA = new byte[0];
|
|
|
|
private static final ReadOptions EMPTY_READ_OPTIONS = new ReadOptions();
|
|
|
|
private final RocksDB db;
|
|
|
|
private final ColumnFamilyHandle cfh;
|
|
|
|
private final String databaseName;
|
2021-02-01 02:21:53 +01:00
|
|
|
private final Scheduler dbScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
private final Function<LLSnapshot, Snapshot> snapshotResolver;
|
2021-02-13 00:18:57 +01:00
|
|
|
private final Striped<StampedLock> itemsLock = Striped.readWriteStampedLock(STRIPES);
|
2021-02-13 01:31:24 +01:00
|
|
|
private final UpdateMode updateMode;
|
2020-12-07 22:15:18 +01:00
|
|
|
|
|
|
|
public LLLocalDictionary(@NotNull RocksDB db,
|
|
|
|
@NotNull ColumnFamilyHandle columnFamilyHandle,
|
|
|
|
String databaseName,
|
2021-02-01 02:21:53 +01:00
|
|
|
Scheduler dbScheduler,
|
2021-02-13 01:31:24 +01:00
|
|
|
Function<LLSnapshot, Snapshot> snapshotResolver,
|
|
|
|
UpdateMode updateMode) {
|
2020-12-07 22:15:18 +01:00
|
|
|
Objects.requireNonNull(db);
|
|
|
|
this.db = db;
|
|
|
|
Objects.requireNonNull(columnFamilyHandle);
|
|
|
|
this.cfh = columnFamilyHandle;
|
|
|
|
this.databaseName = databaseName;
|
2021-02-01 02:21:53 +01:00
|
|
|
this.dbScheduler = dbScheduler;
|
2020-12-07 22:15:18 +01:00
|
|
|
this.snapshotResolver = snapshotResolver;
|
2021-02-13 01:31:24 +01:00
|
|
|
this.updateMode = updateMode;
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public String getDatabaseName() {
|
|
|
|
return databaseName;
|
|
|
|
}
|
|
|
|
|
|
|
|
private ReadOptions resolveSnapshot(LLSnapshot snapshot) {
|
|
|
|
if (snapshot != null) {
|
|
|
|
return getReadOptions(snapshotResolver.apply(snapshot));
|
|
|
|
} else {
|
|
|
|
return EMPTY_READ_OPTIONS;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private ReadOptions getReadOptions(Snapshot snapshot) {
|
|
|
|
if (snapshot != null) {
|
|
|
|
return new ReadOptions().setSnapshot(snapshot);
|
|
|
|
} else {
|
|
|
|
return EMPTY_READ_OPTIONS;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-06 19:21:31 +01:00
|
|
|
private int getLockIndex(byte[] key) {
|
2021-02-11 01:09:15 +01:00
|
|
|
return Math.abs(Arrays.hashCode(key) % STRIPES);
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private IntArrayList getLockIndices(List<byte[]> keys) {
|
|
|
|
var list = new IntArrayList(keys.size());
|
|
|
|
for (byte[] key : keys) {
|
|
|
|
list.add(getLockIndex(key));
|
|
|
|
}
|
|
|
|
return list;
|
|
|
|
}
|
|
|
|
|
|
|
|
private IntArrayList getLockIndicesEntries(List<Entry<byte[], byte[]>> keys) {
|
|
|
|
var list = new IntArrayList(keys.size());
|
|
|
|
for (Entry<byte[], byte[]> key : keys) {
|
|
|
|
list.add(getLockIndex(key.getKey()));
|
|
|
|
}
|
|
|
|
return list;
|
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-03-18 16:19:41 +01:00
|
|
|
public Mono<byte[]> get(@Nullable LLSnapshot snapshot, byte[] key, boolean existsAlmostCertainly) {
|
2021-01-30 00:24:55 +01:00
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.readLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
try {
|
|
|
|
logger.trace("Reading {}", key);
|
2021-03-18 16:19:41 +01:00
|
|
|
Holder<byte[]> data = existsAlmostCertainly ? null : new Holder<>();
|
|
|
|
if (existsAlmostCertainly || db.keyMayExist(cfh, resolveSnapshot(snapshot), key, data)) {
|
|
|
|
if (!existsAlmostCertainly && data.getValue() != null) {
|
2021-02-06 19:21:31 +01:00
|
|
|
return data.getValue();
|
|
|
|
} else {
|
|
|
|
return db.get(cfh, resolveSnapshot(snapshot), key);
|
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
} else {
|
2021-02-06 19:21:31 +01:00
|
|
|
return null;
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlockRead(stamp);
|
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(key), cause))
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 00:26:58 +01:00
|
|
|
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, LLRange range) {
|
2021-01-30 00:24:55 +01:00
|
|
|
if (range.isSingle()) {
|
|
|
|
return containsKey(snapshot, range.getSingle()).map(contains -> !contains);
|
|
|
|
} else {
|
|
|
|
return containsRange(snapshot, range).map(contains -> !contains);
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
public Mono<Boolean> containsRange(@Nullable LLSnapshot snapshot, LLRange range) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-13 19:01:36 +01:00
|
|
|
var readOpts = resolveSnapshot(snapshot);
|
2021-03-14 19:38:20 +01:00
|
|
|
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
2021-03-14 13:08:03 +01:00
|
|
|
readOpts.setFillCache(false);
|
2021-03-13 19:01:36 +01:00
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
2021-03-18 19:53:32 +01:00
|
|
|
try (RocksIterator rocksIterator = db.newIterator(cfh, readOpts)) {
|
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
|
|
|
return rocksIterator.isValid();
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to read range " + range.toString(), cause))
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
private Mono<Boolean> containsKey(@Nullable LLSnapshot snapshot, byte[] key) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.readLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
try {
|
|
|
|
int size = RocksDB.NOT_FOUND;
|
|
|
|
Holder<byte[]> data = new Holder<>();
|
|
|
|
if (db.keyMayExist(cfh, resolveSnapshot(snapshot), key, data)) {
|
|
|
|
if (data.getValue() != null) {
|
|
|
|
size = data.getValue().length;
|
|
|
|
} else {
|
|
|
|
size = db.get(cfh, resolveSnapshot(snapshot), key, NO_DATA);
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
return size != RocksDB.NOT_FOUND;
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlockRead(stamp);
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(key), cause))
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public Mono<byte[]> put(byte[] key, byte[] value, LLDictionaryResultType resultType) {
|
2021-02-02 00:09:46 +01:00
|
|
|
return getPrevValue(key, resultType)
|
|
|
|
.concatWith(Mono
|
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.writeLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
try {
|
|
|
|
logger.trace("Writing {}: {}", key, value);
|
|
|
|
db.put(cfh, key, value);
|
|
|
|
return null;
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlockWrite(stamp);
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
2021-02-02 00:09:46 +01:00
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to write " + Arrays.toString(key), cause))
|
2021-02-02 00:09:46 +01:00
|
|
|
.subscribeOn(dbScheduler)
|
|
|
|
.then(Mono.empty())
|
|
|
|
).singleOrEmpty();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-06 19:21:31 +01:00
|
|
|
@Override
|
2021-03-18 16:19:41 +01:00
|
|
|
public Mono<Boolean> update(byte[] key,
|
|
|
|
Function<Optional<byte[]>, Optional<byte[]>> value,
|
|
|
|
boolean existsAlmostCertainly) {
|
2021-02-06 19:21:31 +01:00
|
|
|
return Mono
|
2021-02-09 14:14:38 +01:00
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.DISALLOW) throw new UnsupportedOperationException("update() is disallowed");
|
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.readLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
try {
|
2021-02-11 10:43:03 +01:00
|
|
|
logger.trace("Reading {}", key);
|
2021-02-13 00:18:57 +01:00
|
|
|
while (true) {
|
2021-02-13 02:16:24 +01:00
|
|
|
boolean changed = false;
|
2021-02-13 00:18:57 +01:00
|
|
|
Optional<byte[]> prevData;
|
2021-03-18 16:19:41 +01:00
|
|
|
var prevDataHolder = existsAlmostCertainly ? null : new Holder<byte[]>();
|
|
|
|
if (existsAlmostCertainly || db.keyMayExist(cfh, key, prevDataHolder)) {
|
|
|
|
if (!existsAlmostCertainly && prevDataHolder.getValue() != null) {
|
2021-02-13 00:18:57 +01:00
|
|
|
prevData = Optional.ofNullable(prevDataHolder.getValue());
|
|
|
|
} else {
|
|
|
|
prevData = Optional.ofNullable(db.get(cfh, key));
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
} else {
|
2021-02-13 00:18:57 +01:00
|
|
|
prevData = Optional.empty();
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
|
2021-02-13 00:18:57 +01:00
|
|
|
Optional<byte[]> newData = value.apply(prevData);
|
|
|
|
if (prevData.isPresent() && newData.isEmpty()) {
|
2021-04-03 19:09:06 +02:00
|
|
|
//noinspection DuplicatedCode
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
var ws = lock.tryConvertToWriteLock(stamp);
|
2021-02-13 02:16:24 +01:00
|
|
|
if (ws != 0) {
|
|
|
|
stamp = ws;
|
|
|
|
} else {
|
2021-02-13 01:31:24 +01:00
|
|
|
lock.unlockRead(stamp);
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.writeLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
continue;
|
|
|
|
}
|
2021-02-13 00:18:57 +01:00
|
|
|
}
|
2021-02-11 10:43:03 +01:00
|
|
|
logger.trace("Deleting {}", key);
|
2021-02-09 14:14:38 +01:00
|
|
|
changed = true;
|
2021-02-06 19:21:31 +01:00
|
|
|
db.delete(cfh, key);
|
2021-02-13 00:18:57 +01:00
|
|
|
} else if (newData.isPresent()
|
|
|
|
&& (prevData.isEmpty() || !Arrays.equals(prevData.get(), newData.get()))) {
|
2021-04-03 19:09:06 +02:00
|
|
|
//noinspection DuplicatedCode
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
var ws = lock.tryConvertToWriteLock(stamp);
|
2021-02-13 02:16:24 +01:00
|
|
|
if (ws != 0) {
|
|
|
|
stamp = ws;
|
|
|
|
} else {
|
2021-02-13 01:31:24 +01:00
|
|
|
lock.unlockRead(stamp);
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.writeLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
continue;
|
|
|
|
}
|
2021-02-13 00:18:57 +01:00
|
|
|
}
|
2021-02-11 10:43:03 +01:00
|
|
|
logger.trace("Writing {}: {}", key, newData.get());
|
2021-02-09 14:14:38 +01:00
|
|
|
changed = true;
|
2021-02-06 19:21:31 +01:00
|
|
|
db.put(cfh, key, newData.get());
|
|
|
|
}
|
2021-02-13 00:18:57 +01:00
|
|
|
return changed;
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlock(stamp);
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to read or write " + Arrays.toString(key), cause))
|
2021-02-06 19:21:31 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-01-30 00:24:55 +01:00
|
|
|
public Mono<byte[]> remove(byte[] key, LLDictionaryResultType resultType) {
|
2021-02-02 00:09:46 +01:00
|
|
|
return getPrevValue(key, resultType)
|
|
|
|
.concatWith(Mono
|
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamp = lock.writeLock();
|
2021-02-13 01:31:24 +01:00
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
try {
|
|
|
|
db.delete(cfh, key);
|
|
|
|
return null;
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlockWrite(stamp);
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
2021-02-02 00:09:46 +01:00
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to delete " + Arrays.toString(key), cause))
|
2021-02-02 00:09:46 +01:00
|
|
|
.subscribeOn(dbScheduler)
|
|
|
|
.then(Mono.empty())
|
|
|
|
).singleOrEmpty();
|
2021-01-30 00:35:03 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private Mono<byte[]> getPrevValue(byte[] key, LLDictionaryResultType resultType) {
|
2021-03-14 13:08:03 +01:00
|
|
|
switch (resultType) {
|
|
|
|
case VALUE_CHANGED:
|
|
|
|
return containsKey(null, key).single().map(LLUtils::booleanToResponse);
|
|
|
|
case PREVIOUS_VALUE:
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
|
|
|
StampedLock lock;
|
|
|
|
long stamp;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock = itemsLock.getAt(getLockIndex(key));
|
|
|
|
|
|
|
|
stamp = lock.readLock();
|
|
|
|
} else {
|
|
|
|
lock = null;
|
|
|
|
stamp = 0;
|
|
|
|
}
|
|
|
|
try {
|
|
|
|
logger.trace("Reading {}", key);
|
|
|
|
var data = new Holder<byte[]>();
|
|
|
|
if (db.keyMayExist(cfh, key, data)) {
|
|
|
|
if (data.getValue() != null) {
|
|
|
|
return data.getValue();
|
2021-02-02 00:09:46 +01:00
|
|
|
} else {
|
2021-03-14 13:08:03 +01:00
|
|
|
return db.get(cfh, key);
|
2021-02-13 01:31:24 +01:00
|
|
|
}
|
2021-03-14 13:08:03 +01:00
|
|
|
} else {
|
|
|
|
return null;
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
2021-03-14 13:08:03 +01:00
|
|
|
} finally {
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
lock.unlockRead(stamp);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.onErrorMap(cause -> new IOException("Failed to read " + Arrays.toString(key), cause))
|
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
case VOID:
|
|
|
|
return Mono.empty();
|
|
|
|
default:
|
|
|
|
return Mono.error(new IllegalStateException("Unexpected value: " + resultType));
|
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
@Override
|
2021-03-18 16:19:41 +01:00
|
|
|
public Flux<Entry<byte[], byte[]>> getMulti(@Nullable LLSnapshot snapshot,
|
|
|
|
Flux<byte[]> keys,
|
|
|
|
boolean existsAlmostCertainly) {
|
2021-02-01 02:21:53 +01:00
|
|
|
return keys
|
|
|
|
.window(MULTI_GET_WINDOW)
|
|
|
|
.flatMap(keysWindowFlux -> keysWindowFlux.collectList()
|
|
|
|
.flatMapMany(keysWindow -> Mono
|
2021-02-03 14:37:02 +01:00
|
|
|
.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
Iterable<StampedLock> locks;
|
|
|
|
ArrayList<Long> stamps;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
locks = itemsLock.bulkGetAt(getLockIndices(keysWindow));
|
|
|
|
stamps = new ArrayList<>();
|
|
|
|
for (var lock : locks) {
|
2021-02-13 02:16:24 +01:00
|
|
|
|
|
|
|
stamps.add(lock.readLock());
|
2021-02-13 01:31:24 +01:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
locks = null;
|
|
|
|
stamps = null;
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
try {
|
|
|
|
var handlesArray = new ColumnFamilyHandle[keysWindow.size()];
|
|
|
|
Arrays.fill(handlesArray, cfh);
|
|
|
|
var handles = ObjectArrayList.wrap(handlesArray, handlesArray.length);
|
|
|
|
var results = db.multiGetAsList(resolveSnapshot(snapshot), handles, keysWindow);
|
|
|
|
var mappedResults = new ArrayList<Entry<byte[], byte[]>>(results.size());
|
|
|
|
for (int i = 0; i < results.size(); i++) {
|
|
|
|
var val = results.get(i);
|
|
|
|
if (val != null) {
|
|
|
|
results.set(i, null);
|
|
|
|
mappedResults.add(Map.entry(keysWindow.get(i), val));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return mappedResults;
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
int index = 0;
|
|
|
|
for (var lock : locks) {
|
|
|
|
lock.unlockRead(stamps.get(index));
|
|
|
|
index++;
|
|
|
|
}
|
2021-02-01 02:21:53 +01:00
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
})
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler)
|
2021-02-03 14:37:02 +01:00
|
|
|
.flatMapMany(Flux::fromIterable)
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to read keys "
|
|
|
|
+ Arrays.deepToString(keysWindow.toArray(byte[][]::new)), cause))
|
2021-01-30 00:24:55 +01:00
|
|
|
)
|
2021-03-04 22:01:50 +01:00
|
|
|
);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-01 02:21:53 +01:00
|
|
|
@Override
|
|
|
|
public Flux<Entry<byte[], byte[]>> putMulti(Flux<Entry<byte[], byte[]>> entries, boolean getOldValues) {
|
|
|
|
return entries
|
|
|
|
.window(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP))
|
|
|
|
.flatMap(Flux::collectList)
|
|
|
|
.flatMap(entriesWindow -> this
|
2021-03-18 16:19:41 +01:00
|
|
|
.getMulti(null, Flux.fromIterable(entriesWindow).map(Entry::getKey), false)
|
2021-02-02 00:09:46 +01:00
|
|
|
.publishOn(dbScheduler)
|
2021-02-01 02:21:53 +01:00
|
|
|
.concatWith(Mono.fromCallable(() -> {
|
2021-02-13 01:31:24 +01:00
|
|
|
Iterable<StampedLock> locks;
|
|
|
|
ArrayList<Long> stamps;
|
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
locks = itemsLock.bulkGetAt(getLockIndicesEntries(entriesWindow));
|
|
|
|
stamps = new ArrayList<>();
|
|
|
|
for (var lock : locks) {
|
2021-02-13 02:16:24 +01:00
|
|
|
stamps.add(lock.writeLock());
|
2021-02-13 01:31:24 +01:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
locks = null;
|
|
|
|
stamps = null;
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
|
|
|
try {
|
|
|
|
var batch = new CappedWriteBatch(db,
|
|
|
|
CAPPED_WRITE_BATCH_CAP,
|
|
|
|
RESERVED_WRITE_BATCH_SIZE,
|
|
|
|
MAX_WRITE_BATCH_SIZE,
|
|
|
|
BATCH_WRITE_OPTIONS
|
|
|
|
);
|
|
|
|
for (Entry<byte[], byte[]> entry : entriesWindow) {
|
|
|
|
batch.put(entry.getKey(), entry.getValue());
|
|
|
|
}
|
|
|
|
batch.writeToDbAndClose();
|
|
|
|
batch.close();
|
|
|
|
return null;
|
|
|
|
} finally {
|
2021-02-13 01:31:24 +01:00
|
|
|
if (updateMode == UpdateMode.ALLOW) {
|
|
|
|
int index = 0;
|
|
|
|
for (var lock : locks) {
|
|
|
|
lock.unlockWrite(stamps.get(index));
|
|
|
|
index++;
|
|
|
|
}
|
2021-02-06 19:21:31 +01:00
|
|
|
}
|
2021-02-01 02:21:53 +01:00
|
|
|
}
|
|
|
|
})));
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-01-30 00:33:36 +01:00
|
|
|
@NotNull
|
2021-03-18 16:19:41 +01:00
|
|
|
private Mono<Entry<byte[], byte[]>> putEntryToWriteBatch(Entry<byte[], byte[]> newEntry,
|
2021-04-03 19:09:06 +02:00
|
|
|
boolean getOldValues, CappedWriteBatch writeBatch) {
|
2021-03-13 19:01:36 +01:00
|
|
|
Mono<byte[]> getOldValueMono;
|
|
|
|
if (getOldValues) {
|
2021-04-03 19:09:06 +02:00
|
|
|
getOldValueMono = get(null, newEntry.getKey(), false);
|
2021-03-13 19:01:36 +01:00
|
|
|
} else {
|
|
|
|
getOldValueMono = Mono.empty();
|
|
|
|
}
|
|
|
|
return getOldValueMono
|
|
|
|
.concatWith(Mono
|
|
|
|
.<byte[]>fromCallable(() -> {
|
|
|
|
writeBatch.put(cfh, newEntry.getKey(), newEntry.getValue());
|
|
|
|
return null;
|
|
|
|
})
|
|
|
|
.subscribeOn(dbScheduler)
|
|
|
|
)
|
|
|
|
.singleOrEmpty()
|
|
|
|
.map(oldValue -> Map.entry(newEntry.getKey(), oldValue));
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-03-18 16:19:41 +01:00
|
|
|
public Flux<Entry<byte[], byte[]>> getRange(@Nullable LLSnapshot snapshot,
|
|
|
|
LLRange range,
|
|
|
|
boolean existsAlmostCertainly) {
|
2021-03-14 13:08:03 +01:00
|
|
|
if (range.isSingle()) {
|
2021-03-18 16:19:41 +01:00
|
|
|
return getRangeSingle(snapshot, range.getMin(), existsAlmostCertainly);
|
2021-03-14 13:08:03 +01:00
|
|
|
} else {
|
|
|
|
return getRangeMulti(snapshot, range);
|
|
|
|
}
|
2021-02-02 00:09:46 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public Flux<List<Entry<byte[], byte[]>>> getRangeGrouped(@Nullable LLSnapshot snapshot,
|
|
|
|
LLRange range,
|
2021-03-18 16:19:41 +01:00
|
|
|
int prefixLength, boolean existsAlmostCertainly) {
|
2021-03-14 13:08:03 +01:00
|
|
|
if (range.isSingle()) {
|
2021-03-18 16:19:41 +01:00
|
|
|
return getRangeSingle(snapshot, range.getMin(), existsAlmostCertainly).map(List::of);
|
2021-03-14 13:08:03 +01:00
|
|
|
} else {
|
|
|
|
return getRangeMultiGrouped(snapshot, range, prefixLength);
|
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
|
2021-03-18 16:19:41 +01:00
|
|
|
private Flux<Entry<byte[],byte[]>> getRangeSingle(LLSnapshot snapshot, byte[] key, boolean existsAlmostCertainly) {
|
2021-01-30 20:16:14 +01:00
|
|
|
return this
|
2021-03-18 16:19:41 +01:00
|
|
|
.get(snapshot, key, existsAlmostCertainly)
|
2021-01-30 20:16:14 +01:00
|
|
|
.map(value -> Map.entry(key, value))
|
|
|
|
.flux();
|
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
private Flux<Entry<byte[],byte[]>> getRangeMulti(LLSnapshot snapshot, LLRange range) {
|
2021-03-14 23:06:46 +01:00
|
|
|
return new LLLocalEntryReactiveRocksIterator(db, cfh, range, resolveSnapshot(snapshot))
|
2021-03-14 13:08:03 +01:00
|
|
|
.flux()
|
|
|
|
.subscribeOn(dbScheduler);
|
2021-02-02 00:09:46 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private Flux<List<Entry<byte[],byte[]>>> getRangeMultiGrouped(LLSnapshot snapshot, LLRange range, int prefixLength) {
|
2021-03-14 23:06:46 +01:00
|
|
|
return new LLLocalGroupedEntryReactiveRocksIterator(db,
|
2021-03-14 13:08:03 +01:00
|
|
|
cfh,
|
|
|
|
prefixLength,
|
|
|
|
range,
|
|
|
|
resolveSnapshot(snapshot),
|
|
|
|
"getRangeMultiGrouped"
|
|
|
|
)
|
|
|
|
.flux()
|
|
|
|
.subscribeOn(dbScheduler);
|
2021-01-30 00:24:55 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 20:16:14 +01:00
|
|
|
@Override
|
|
|
|
public Flux<byte[]> getRangeKeys(@Nullable LLSnapshot snapshot, LLRange range) {
|
2021-03-14 13:08:03 +01:00
|
|
|
if (range.isSingle()) {
|
|
|
|
return getRangeKeysSingle(snapshot, range.getMin());
|
|
|
|
} else {
|
|
|
|
return getRangeKeysMulti(snapshot, range);
|
|
|
|
}
|
2021-02-02 00:09:46 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public Flux<List<byte[]>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot, LLRange range, int prefixLength) {
|
2021-03-14 23:06:46 +01:00
|
|
|
return new LLLocalGroupedKeyReactiveRocksIterator(db,
|
2021-03-14 03:13:19 +01:00
|
|
|
cfh,
|
|
|
|
prefixLength,
|
|
|
|
range,
|
2021-03-14 13:08:03 +01:00
|
|
|
resolveSnapshot(snapshot),
|
|
|
|
"getRangeKeysGrouped"
|
|
|
|
).flux().subscribeOn(dbScheduler);
|
2021-03-14 13:24:46 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public Flux<byte[]> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot, LLRange range, int prefixLength) {
|
2021-03-14 23:06:46 +01:00
|
|
|
return new LLLocalKeyPrefixReactiveRocksIterator(db,
|
2021-03-14 13:24:46 +01:00
|
|
|
cfh,
|
|
|
|
prefixLength,
|
|
|
|
range,
|
|
|
|
resolveSnapshot(snapshot),
|
2021-03-22 20:02:19 +01:00
|
|
|
true,
|
2021-03-14 13:24:46 +01:00
|
|
|
"getRangeKeysGrouped"
|
|
|
|
).flux().subscribeOn(dbScheduler);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
private Flux<byte[]> getRangeKeysSingle(LLSnapshot snapshot, byte[] key) {
|
2021-01-30 00:24:55 +01:00
|
|
|
return this
|
2021-01-30 20:16:14 +01:00
|
|
|
.containsKey(snapshot, key)
|
|
|
|
.filter(contains -> contains)
|
|
|
|
.map(contains -> key)
|
2021-01-30 00:24:55 +01:00
|
|
|
.flux();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 20:16:14 +01:00
|
|
|
private Flux<byte[]> getRangeKeysMulti(LLSnapshot snapshot, LLRange range) {
|
2021-03-14 23:06:46 +01:00
|
|
|
return new LLLocalKeyReactiveRocksIterator(db, cfh, range, resolveSnapshot(snapshot)).flux().subscribeOn(dbScheduler);
|
2021-01-30 20:16:14 +01:00
|
|
|
}
|
|
|
|
|
2020-12-07 22:15:18 +01:00
|
|
|
@Override
|
2021-01-30 00:24:55 +01:00
|
|
|
public Flux<Entry<byte[], byte[]>> setRange(LLRange range,
|
|
|
|
Flux<Entry<byte[], byte[]>> entries,
|
|
|
|
boolean getOldValues) {
|
2021-03-14 13:08:03 +01:00
|
|
|
return Flux
|
2021-03-14 03:13:19 +01:00
|
|
|
.usingWhen(
|
|
|
|
Mono
|
|
|
|
.fromCallable(() -> new CappedWriteBatch(db,
|
|
|
|
CAPPED_WRITE_BATCH_CAP,
|
|
|
|
RESERVED_WRITE_BATCH_SIZE,
|
|
|
|
MAX_WRITE_BATCH_SIZE,
|
|
|
|
BATCH_WRITE_OPTIONS)
|
|
|
|
)
|
|
|
|
.subscribeOn(dbScheduler),
|
|
|
|
writeBatch -> Mono
|
|
|
|
.fromCallable(() -> {
|
|
|
|
if (range.isSingle()) {
|
|
|
|
writeBatch.delete(cfh, range.getSingle());
|
|
|
|
} else {
|
2021-03-20 12:41:11 +01:00
|
|
|
deleteSmallRangeWriteBatch(writeBatch, range);
|
2021-03-14 03:13:19 +01:00
|
|
|
}
|
|
|
|
return null;
|
|
|
|
})
|
|
|
|
.subscribeOn(dbScheduler)
|
|
|
|
.thenMany(entries)
|
2021-04-03 19:09:06 +02:00
|
|
|
.flatMapSequential(newEntry -> putEntryToWriteBatch(newEntry, getOldValues, writeBatch)),
|
2021-03-14 03:13:19 +01:00
|
|
|
writeBatch -> Mono
|
|
|
|
.fromCallable(() -> {
|
|
|
|
try (writeBatch) {
|
|
|
|
writeBatch.writeToDbAndClose();
|
|
|
|
}
|
|
|
|
return null;
|
|
|
|
})
|
|
|
|
.subscribeOn(dbScheduler)
|
|
|
|
)
|
|
|
|
.subscribeOn(dbScheduler)
|
2021-03-14 13:08:03 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to write range", cause));
|
2021-03-14 03:13:19 +01:00
|
|
|
}
|
|
|
|
|
2021-03-20 12:41:11 +01:00
|
|
|
private void deleteSmallRangeWriteBatch(CappedWriteBatch writeBatch, LLRange range)
|
|
|
|
throws RocksDBException {
|
|
|
|
var readOpts = getReadOptions(null);
|
|
|
|
readOpts.setFillCache(false);
|
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
|
|
|
try (var rocksIterator = db.newIterator(cfh, readOpts)) {
|
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
|
|
|
while (rocksIterator.isValid()) {
|
|
|
|
writeBatch.delete(cfh, rocksIterator.key());
|
|
|
|
rocksIterator.next();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
public Mono<Void> clear() {
|
|
|
|
return Mono
|
|
|
|
.<Void>fromCallable(() -> {
|
2021-03-14 03:13:19 +01:00
|
|
|
var readOpts = getReadOptions(null);
|
2021-03-14 19:38:20 +01:00
|
|
|
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
2021-03-14 13:08:03 +01:00
|
|
|
|
2021-03-14 03:13:19 +01:00
|
|
|
// readOpts.setIgnoreRangeDeletions(true);
|
|
|
|
readOpts.setFillCache(false);
|
2021-03-22 20:02:19 +01:00
|
|
|
//readOpts.setReadaheadSize(2 * 1024 * 1024);
|
2021-03-14 03:13:19 +01:00
|
|
|
try (CappedWriteBatch writeBatch = new CappedWriteBatch(db,
|
2021-01-30 00:24:55 +01:00
|
|
|
CAPPED_WRITE_BATCH_CAP,
|
|
|
|
RESERVED_WRITE_BATCH_SIZE,
|
|
|
|
MAX_WRITE_BATCH_SIZE,
|
|
|
|
BATCH_WRITE_OPTIONS
|
|
|
|
)) {
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-03-20 12:41:11 +01:00
|
|
|
byte[] firstDeletedKey = null;
|
|
|
|
byte[] lastDeletedKey = null;
|
2021-03-14 03:13:19 +01:00
|
|
|
try (RocksIterator iter = db.newIterator(cfh, readOpts)) {
|
|
|
|
iter.seekToLast();
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-03-14 03:13:19 +01:00
|
|
|
if (iter.isValid()) {
|
2021-03-20 12:41:11 +01:00
|
|
|
firstDeletedKey = FIRST_KEY;
|
|
|
|
lastDeletedKey = iter.key();
|
2021-03-14 03:13:19 +01:00
|
|
|
writeBatch.deleteRange(cfh, FIRST_KEY, iter.key());
|
|
|
|
writeBatch.delete(cfh, iter.key());
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
writeBatch.writeToDbAndClose();
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-03-20 12:41:11 +01:00
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
// Compact range
|
2021-03-14 03:13:19 +01:00
|
|
|
db.suggestCompactRange(cfh);
|
2021-03-20 12:41:11 +01:00
|
|
|
if (firstDeletedKey != null && lastDeletedKey != null) {
|
2021-03-21 13:06:54 +01:00
|
|
|
db.compactRange(cfh,
|
|
|
|
firstDeletedKey,
|
|
|
|
lastDeletedKey,
|
|
|
|
new CompactRangeOptions()
|
|
|
|
.setAllowWriteStall(false)
|
|
|
|
.setExclusiveManualCompaction(false)
|
|
|
|
.setChangeLevel(false)
|
|
|
|
);
|
2021-03-20 12:41:11 +01:00
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
db.flush(new FlushOptions().setWaitForFlush(true).setAllowWriteStall(true), cfh);
|
|
|
|
db.flushWal(true);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
return null;
|
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to clear", cause))
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 00:24:55 +01:00
|
|
|
public Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, LLRange range, boolean fast) {
|
2021-03-14 13:08:03 +01:00
|
|
|
if (range.isAll()) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> fast ? fastSizeAll(snapshot) : exactSizeAll(snapshot))
|
|
|
|
.onErrorMap(IOException::new)
|
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
} else {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
|
|
|
var readOpts = resolveSnapshot(snapshot);
|
|
|
|
readOpts.setFillCache(false);
|
2021-03-14 19:38:20 +01:00
|
|
|
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
2021-03-14 13:08:03 +01:00
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
|
|
|
if (fast) {
|
|
|
|
readOpts.setIgnoreRangeDeletions(true);
|
|
|
|
|
|
|
|
}
|
2021-03-18 19:53:32 +01:00
|
|
|
try (var rocksIterator = db.newIterator(cfh, readOpts)) {
|
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
2021-03-14 13:08:03 +01:00
|
|
|
long i = 0;
|
2021-03-18 19:53:32 +01:00
|
|
|
while (rocksIterator.isValid()) {
|
|
|
|
rocksIterator.next();
|
2021-03-14 13:08:03 +01:00
|
|
|
i++;
|
|
|
|
}
|
|
|
|
return i;
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.onErrorMap(cause -> new IOException("Failed to get size of range "
|
|
|
|
+ range.toString(), cause))
|
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
|
2021-02-02 15:36:11 +01:00
|
|
|
@Override
|
|
|
|
public Mono<Entry<byte[], byte[]>> getOne(@Nullable LLSnapshot snapshot, LLRange range) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-13 19:01:36 +01:00
|
|
|
var readOpts = resolveSnapshot(snapshot);
|
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
|
|
|
try (var rocksIterator = db.newIterator(cfh, readOpts)) {
|
2021-03-18 19:53:32 +01:00
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
2021-02-02 15:36:11 +01:00
|
|
|
byte[] key;
|
|
|
|
if (rocksIterator.isValid()) {
|
|
|
|
key = rocksIterator.key();
|
|
|
|
return Map.entry(key, rocksIterator.value());
|
|
|
|
} else {
|
|
|
|
return null;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
|
|
public Mono<byte[]> getOneKey(@Nullable LLSnapshot snapshot, LLRange range) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-13 19:01:36 +01:00
|
|
|
var readOpts = resolveSnapshot(snapshot);
|
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
|
|
|
try (var rocksIterator = db.newIterator(cfh, readOpts)) {
|
2021-03-18 19:53:32 +01:00
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
2021-02-02 15:36:11 +01:00
|
|
|
byte[] key;
|
|
|
|
if (rocksIterator.isValid()) {
|
|
|
|
key = rocksIterator.key();
|
|
|
|
return key;
|
|
|
|
} else {
|
|
|
|
return null;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.subscribeOn(dbScheduler);
|
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
private long fastSizeAll(@Nullable LLSnapshot snapshot) {
|
2021-03-14 03:13:19 +01:00
|
|
|
var rocksdbSnapshot = resolveSnapshot(snapshot);
|
2020-12-07 22:15:18 +01:00
|
|
|
if (USE_CURRENT_FASTSIZE_FOR_OLD_SNAPSHOTS || rocksdbSnapshot.snapshot() == null) {
|
|
|
|
try {
|
|
|
|
return db.getLongProperty(cfh, "rocksdb.estimate-num-keys");
|
|
|
|
} catch (RocksDBException e) {
|
|
|
|
e.printStackTrace();
|
|
|
|
return 0;
|
|
|
|
}
|
2021-03-18 19:53:32 +01:00
|
|
|
} else if (PARALLEL_EXACT_SIZE) {
|
|
|
|
return exactSizeAll(snapshot);
|
2020-12-07 22:15:18 +01:00
|
|
|
} else {
|
2021-03-13 19:01:36 +01:00
|
|
|
rocksdbSnapshot.setFillCache(false);
|
2021-03-14 19:38:20 +01:00
|
|
|
rocksdbSnapshot.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
2021-03-18 19:53:32 +01:00
|
|
|
rocksdbSnapshot.setIgnoreRangeDeletions(true);
|
2020-12-07 22:15:18 +01:00
|
|
|
long count = 0;
|
|
|
|
try (RocksIterator iter = db.newIterator(cfh, rocksdbSnapshot)) {
|
|
|
|
iter.seekToFirst();
|
2021-03-18 19:53:32 +01:00
|
|
|
// If it's a fast size of a snapshot, count only up to 100'000 elements
|
|
|
|
while (iter.isValid() && count < 100_000) {
|
2020-12-07 22:15:18 +01:00
|
|
|
count++;
|
|
|
|
iter.next();
|
|
|
|
}
|
|
|
|
return count;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-30 00:24:55 +01:00
|
|
|
private long exactSizeAll(@Nullable LLSnapshot snapshot) {
|
2021-03-13 19:01:36 +01:00
|
|
|
var readOpts = resolveSnapshot(snapshot);
|
|
|
|
readOpts.setFillCache(false);
|
2021-03-22 20:02:19 +01:00
|
|
|
//readOpts.setReadaheadSize(2 * 1024 * 1024);
|
2021-03-14 19:38:20 +01:00
|
|
|
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
2021-03-14 03:13:19 +01:00
|
|
|
|
2021-03-18 19:53:32 +01: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 ? null : LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS[idx],
|
|
|
|
LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS.length >= idx + 1 ? null
|
|
|
|
: LLUtils.LEXICONOGRAPHIC_ITERATION_SEEKS[idx + 1]
|
|
|
|
))
|
|
|
|
.map(range -> (Callable<Long>) () -> {
|
|
|
|
long partialCount = 0;
|
|
|
|
var rangeReadOpts = new ReadOptions(readOpts);
|
|
|
|
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) {
|
|
|
|
rangeReadOpts.setIterateLowerBound(sliceBegin);
|
|
|
|
}
|
|
|
|
if (sliceBegin != null) {
|
|
|
|
rangeReadOpts.setIterateUpperBound(sliceEnd);
|
|
|
|
}
|
|
|
|
try (RocksIterator iter = db.newIterator(cfh, rangeReadOpts)) {
|
|
|
|
iter.seekToFirst();
|
|
|
|
while (iter.isValid()) {
|
|
|
|
partialCount++;
|
|
|
|
iter.next();
|
|
|
|
}
|
|
|
|
return partialCount;
|
|
|
|
}
|
|
|
|
} finally {
|
|
|
|
if (sliceBegin != null) {
|
|
|
|
sliceBegin.close();
|
|
|
|
}
|
|
|
|
if (sliceEnd != null) {
|
|
|
|
sliceEnd.close();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.map(commonPool::submit)
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
long count = 0;
|
|
|
|
for (ForkJoinTask<Long> future : futures) {
|
|
|
|
count += future.join();
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
return count;
|
2021-03-18 19:53:32 +01:00
|
|
|
} else {
|
|
|
|
long count = 0;
|
|
|
|
try (RocksIterator iter = db.newIterator(cfh, readOpts)) {
|
|
|
|
iter.seekToFirst();
|
|
|
|
while (iter.isValid()) {
|
|
|
|
count++;
|
|
|
|
iter.next();
|
|
|
|
}
|
|
|
|
return count;
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
@Override
|
2021-01-30 00:24:55 +01:00
|
|
|
public Mono<Entry<byte[], byte[]>> removeOne(LLRange range) {
|
|
|
|
return Mono
|
|
|
|
.fromCallable(() -> {
|
2021-03-13 19:01:36 +01:00
|
|
|
var readOpts = getReadOptions(null);
|
|
|
|
if (range.hasMin()) {
|
|
|
|
readOpts.setIterateLowerBound(new Slice(range.getMin()));
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
readOpts.setIterateUpperBound(new Slice(range.getMax()));
|
|
|
|
}
|
|
|
|
try (RocksIterator iter = db.newIterator(cfh, readOpts)) {
|
2021-03-14 19:38:20 +01:00
|
|
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
2021-03-14 03:13:19 +01:00
|
|
|
iter.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
iter.seekToFirst();
|
|
|
|
}
|
2021-01-30 00:24:55 +01:00
|
|
|
if (!iter.isValid()) {
|
|
|
|
return null;
|
|
|
|
}
|
|
|
|
byte[] key = iter.key();
|
|
|
|
byte[] value = iter.value();
|
|
|
|
db.delete(cfh, key);
|
|
|
|
return Map.entry(key, value);
|
|
|
|
}
|
|
|
|
})
|
2021-03-04 22:01:50 +01:00
|
|
|
.onErrorMap(cause -> new IOException("Failed to delete " + range.toString(), cause))
|
2021-02-01 02:21:53 +01:00
|
|
|
.subscribeOn(dbScheduler);
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|
2021-04-03 19:09:06 +02:00
|
|
|
|
|
|
|
@NotNull
|
|
|
|
public static Tuple3<RocksIterator, Optional<Slice>, Optional<Slice>> getRocksIterator(ReadOptions readOptions,
|
|
|
|
LLRange range,
|
|
|
|
RocksDB db,
|
|
|
|
ColumnFamilyHandle cfh) {
|
|
|
|
Slice sliceMin;
|
|
|
|
Slice sliceMax;
|
|
|
|
if (range.hasMin()) {
|
|
|
|
sliceMin = new Slice(range.getMin());
|
|
|
|
readOptions.setIterateLowerBound(sliceMin);
|
|
|
|
} else {
|
|
|
|
sliceMin = null;
|
|
|
|
}
|
|
|
|
if (range.hasMax()) {
|
|
|
|
sliceMax = new Slice(range.getMax());
|
|
|
|
readOptions.setIterateUpperBound(sliceMax);
|
|
|
|
} else {
|
|
|
|
sliceMax = null;
|
|
|
|
}
|
|
|
|
var rocksIterator = db.newIterator(cfh, readOptions);
|
|
|
|
if (!PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
|
|
|
rocksIterator.seek(range.getMin());
|
|
|
|
} else {
|
|
|
|
rocksIterator.seekToFirst();
|
|
|
|
}
|
|
|
|
return Tuples.of(rocksIterator, Optional.ofNullable(sliceMin), Optional.ofNullable(sliceMax));
|
|
|
|
}
|
2020-12-07 22:15:18 +01:00
|
|
|
}
|