package it.cavallium.dbengine.database.disk; import static io.netty5.buffer.api.StandardAllocationTypes.OFF_HEAP; import static it.cavallium.dbengine.database.LLUtils.MARKER_ROCKSDB; import static it.cavallium.dbengine.database.LLUtils.fromByteArray; import static it.cavallium.dbengine.database.LLUtils.isReadOnlyDirect; import static it.cavallium.dbengine.database.LLUtils.toStringSafe; import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNullElse; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.Timer; import io.netty5.buffer.api.Buffer; import io.netty5.buffer.api.BufferAllocator; import io.netty5.buffer.api.ReadableComponent; import io.netty5.buffer.api.Send; import it.cavallium.dbengine.client.BadBlock; import it.cavallium.dbengine.database.ColumnUtils; import it.cavallium.dbengine.database.LLDelta; import it.cavallium.dbengine.database.LLDictionary; import it.cavallium.dbengine.database.LLDictionaryResultType; import it.cavallium.dbengine.database.LLEntry; import it.cavallium.dbengine.database.LLRange; import it.cavallium.dbengine.database.LLSnapshot; import it.cavallium.dbengine.database.LLUtils; import it.cavallium.dbengine.database.SafeCloseable; import it.cavallium.dbengine.database.UpdateMode; import it.cavallium.dbengine.database.UpdateReturnMode; import it.cavallium.dbengine.database.serialization.KVSerializationFunction; import it.cavallium.dbengine.database.serialization.SerializationFunction; import it.cavallium.dbengine.rpc.current.data.DatabaseOptions; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinTask; import java.util.function.Function; import java.util.stream.IntStream; import org.apache.commons.lang3.tuple.Pair; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.util.Supplier; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.rocksdb.AbstractSlice; import org.rocksdb.CappedWriteBatch; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.CompactRangeOptions; import org.rocksdb.DirectSlice; import org.rocksdb.FlushOptions; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; import org.rocksdb.Slice; import org.rocksdb.Snapshot; import org.rocksdb.WriteBatch; import org.rocksdb.WriteOptions; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; import reactor.util.function.Tuple2; import reactor.util.function.Tuple3; import reactor.util.function.Tuples; public class LLLocalDictionary implements LLDictionary { protected static final Logger logger = LogManager.getLogger(LLLocalDictionary.class); private static final boolean USE_CURRENT_FASTSIZE_FOR_OLD_SNAPSHOTS = false; 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; static final ReadOptions EMPTY_READ_OPTIONS = new UnreleasableReadOptions(new UnmodifiableReadOptions()); static final WriteOptions EMPTY_WRITE_OPTIONS = new UnreleasableWriteOptions(new UnmodifiableWriteOptions()); static final WriteOptions BATCH_WRITE_OPTIONS = new UnreleasableWriteOptions(new UnmodifiableWriteOptions()); static final boolean PREFER_SEEK_TO_FIRST = true; /** * It used to be false, * now it's true to avoid crashes during iterations on completely corrupted files */ static final boolean VERIFY_CHECKSUMS_WHEN_NOT_NEEDED = true; /** * Default: true. Use false to debug problems with windowing. */ static final boolean USE_WINDOW_IN_SET_RANGE = true; /** * Default: true. Use false to debug problems with write batches. */ 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; /** * Default: true. Use false to debug problems with capped write batches. */ static final boolean USE_CAPPED_WRITE_BATCH_IN_SET_RANGE = true; /** * Default: true. Use false to debug problems with write batches deletes. */ static final boolean USE_WRITE_BATCH_IN_SET_RANGE_DELETE = false; static final boolean PARALLEL_EXACT_SIZE = true; private static final byte[] FIRST_KEY = new byte[]{}; /** * Default: true */ private static final boolean USE_DIRECT_BUFFER_BOUNDS = true; /** * 1KiB dummy buffer, write only, used for debugging purposes */ private static final ByteBuffer DUMMY_WRITE_ONLY_BYTE_BUFFER = ByteBuffer.allocateDirect(1024); private final RocksDBColumn db; private final ColumnFamilyHandle cfh; private final String databaseName; private final String columnName; private final Scheduler dbScheduler; private final Function snapshotResolver; private final UpdateMode updateMode; private final boolean nettyDirect; private final BufferAllocator alloc; 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; public LLLocalDictionary( BufferAllocator allocator, @NotNull RocksDBColumn db, String databaseName, String columnName, Scheduler dbScheduler, Function snapshotResolver, UpdateMode updateMode, DatabaseOptions databaseOptions) { requireNonNull(db); this.db = db; this.cfh = db.getColumnFamilyHandle(); this.databaseName = databaseName; this.columnName = columnName; this.dbScheduler = dbScheduler; this.snapshotResolver = snapshotResolver; this.updateMode = updateMode; alloc = allocator; this.nettyDirect = databaseOptions.allowNettyDirect() && alloc.getAllocationType() == OFF_HEAP; 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 .builder("db.read.map.get.timer") .publishPercentiles(0.2, 0.5, 0.95) .publishPercentileHistogram() .tags("db.name", databaseName, "db.column", columnName) .register(meterRegistry); 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); this.containsTime = Timer .builder("db.read.map.contains.timer") .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); } @Override public String getDatabaseName() { return databaseName; } public String getColumnName() { return columnName; } /** * Please don't modify the returned ReadOptions! * If you want to modify it, wrap it into a new ReadOptions! */ private ReadOptions resolveSnapshot(LLSnapshot snapshot) { if (snapshot != null) { return getReadOptions(snapshotResolver.apply(snapshot)); } else { return EMPTY_READ_OPTIONS; } } /** * Please don't modify the returned ReadOptions! * If you want to modify it, wrap it into a new ReadOptions! */ private ReadOptions getReadOptions(Snapshot snapshot) { if (snapshot != null) { return new ReadOptions().setSnapshot(snapshot); } else { return EMPTY_READ_OPTIONS; } } @Override public BufferAllocator getAllocator() { return alloc; } private @NotNull Mono runOnDb(Callable<@Nullable T> callable) { return Mono.fromCallable(callable).subscribeOn(dbScheduler); } @Override public Mono> get(@Nullable LLSnapshot snapshot, Mono> keyMono, boolean existsAlmostCertainly) { return keyMono .publishOn(dbScheduler) .handle((keySend, sink) -> { try (var key = keySend.receive()) { try { var readOptions = requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS); Buffer result; startedGet.increment(); try { result = getTime.recordCallable(() -> db.get(readOptions, key)); } finally { endedGet.increment(); } logger.trace(MARKER_ROCKSDB, "Read {}: {}", () -> toStringSafe(key), () -> toStringSafe(result)); if (result != null) { sink.next(result.send()); } else { sink.complete(); } } catch (RocksDBException ex) { sink.error(new IOException("Failed to read " + toStringSafe(key) + ": " + ex.getMessage())); } catch (Exception ex) { sink.error(ex); } } }); } @Override public Mono isRangeEmpty(@Nullable LLSnapshot snapshot, Mono> rangeMono, boolean fillCache) { return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called isRangeEmpty in a nonblocking thread"; startedContains.increment(); try { Boolean isRangeEmpty = containsTime.recordCallable(() -> { if (range.isSingle()) { return !containsKey(snapshot, range.getSingleUnsafe()); } else { // Temporary resources to release after finished AbstractSlice slice1 = null; AbstractSlice slice2 = null; try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) { readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED); readOpts.setFillCache(fillCache); if (range.hasMin()) { if (nettyDirect && isReadOnlyDirect(range.getMinUnsafe())) { readOpts.setIterateLowerBound(slice1 = new DirectSlice( ((ReadableComponent) range.getMinUnsafe()).readableBuffer(), range.getMinUnsafe().readableBytes() )); } else { readOpts.setIterateLowerBound(slice1 = new Slice(LLUtils.toArray(range.getMinUnsafe()))); } } if (range.hasMax()) { if (nettyDirect && isReadOnlyDirect(range.getMaxUnsafe())) { readOpts.setIterateUpperBound(slice2 = new DirectSlice( ((ReadableComponent) range.getMaxUnsafe()).readableBuffer(), range.getMaxUnsafe().readableBytes() )); } else { readOpts.setIterateUpperBound(slice2 = new Slice(LLUtils.toArray(range.getMaxUnsafe()))); } } try (RocksIterator rocksIterator = db.newIterator(readOpts)) { if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { if (nettyDirect && isReadOnlyDirect(range.getMinUnsafe())) { rocksIterator.seek(((ReadableComponent) range.getMinUnsafe()).readableBuffer()); } else { rocksIterator.seek(LLUtils.toArray(range.getMinUnsafe())); } } else { rocksIterator.seekToFirst(); } rocksIterator.status(); return !rocksIterator.isValid(); } } finally { if (slice1 != null) { slice1.close(); } if (slice2 != null) { slice2.close(); } } } }); assert isRangeEmpty != null; sink.next(isRangeEmpty); } 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); } }); } private boolean containsKey(@Nullable LLSnapshot snapshot, Buffer key) throws RocksDBException { startedContains.increment(); try { var result = containsTime.recordCallable(() -> { var unmodifiableReadOpts = resolveSnapshot(snapshot); return db.exists(unmodifiableReadOpts, key); }); assert result != null; return result; } catch (RocksDBException | RuntimeException e) { throw e; } catch (Exception ex) { throw new RuntimeException(ex); } finally { endedContains.increment(); } } @Override public Mono> put(Mono> keyMono, Mono> valueMono, 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 var previousDataMono = this.getPreviousData(keyMono, resultType, false); // Write the new entry to the database Mono> putMono = entryMono .publishOn(dbScheduler) .handle((entry, sink) -> { try (var key = entry.getKey().receive()) { try (var value = entry.getValue().receive()) { put(key, value); sink.complete(); } catch (RocksDBException ex) { sink.error(ex); } } }); // Read the previous data, then write the new data, then return the previous data return Flux.concat(previousDataMono, putMono).singleOrEmpty(); } 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(); try { putTime.recordCallable(() -> { db.put(EMPTY_WRITE_OPTIONS, key, value); 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(); } } @Override public Mono getUpdateMode() { return Mono.just(updateMode); } @SuppressWarnings("DuplicatedCode") @Override public Mono> update(Mono> keyMono, SerializationFunction<@Nullable Send, @Nullable Buffer> updater, UpdateReturnMode updateReturnMode) { return keyMono .publishOn(dbScheduler) .handle((keySend, sink) -> { try (keySend) { 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; startedUpdates.increment(); try { result = updateTime.recordCallable(() -> db.updateAtomic(EMPTY_READ_OPTIONS, EMPTY_WRITE_OPTIONS, keySend, updater, returnMode)); } finally { endedUpdates.increment(); } 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); } }); } @SuppressWarnings("DuplicatedCode") @Override public Mono> updateAndGetDelta(Mono> keyMono, SerializationFunction<@Nullable Send, @Nullable Buffer> updater) { return keyMono .publishOn(dbScheduler) .handle((keySend, sink) -> { try (keySend) { 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; startedUpdates.increment(); try { result = updateTime.recordCallable(() -> db.updateAtomic(EMPTY_READ_OPTIONS, EMPTY_WRITE_OPTIONS, keySend, updater, UpdateAtomicResultMode.DELTA)); } finally { endedUpdates.increment(); } assert result != null; sink.next(((UpdateAtomicResultDelta) result).delta()); } catch (Exception ex) { sink.error(ex); } }); } @Override public Mono> remove(Mono> keyMono, LLDictionaryResultType resultType) { // Obtain the previous value from the database Mono> previousDataMono = this.getPreviousData(keyMono, resultType, true); // Delete the value from the database Mono> removeMono = keyMono .publishOn(dbScheduler) .handle((keySend, sink) -> { try (var key = keySend.receive()) { logger.trace(MARKER_ROCKSDB, "Deleting {}", () -> toStringSafe(key)); startedRemove.increment(); try { removeTime.recordCallable(() -> { db.delete(EMPTY_WRITE_OPTIONS, key); 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(); } private Mono> getPreviousData(Mono> keyMono, LLDictionaryResultType resultType, boolean existsAlmostCertainly) { return switch (resultType) { case PREVIOUS_VALUE_EXISTENCE -> keyMono .publishOn(dbScheduler) .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); } }); case PREVIOUS_VALUE -> keyMono .publishOn(dbScheduler) .handle((keySend, sink) -> { try (var key = keySend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called getPreviousData in a nonblocking thread"; var result = db.get(EMPTY_READ_OPTIONS, key); logger.trace(MARKER_ROCKSDB, "Reading {}: {}", () -> toStringSafe(key), () -> toStringSafe(result)); if (result == null) { sink.complete(); } else { sink.next(result.send()); } } catch (Exception ex) { sink.error(ex); } }); case VOID -> Mono.empty(); }; } @Override public Flux> getMulti(@Nullable LLSnapshot snapshot, Flux> keys, boolean existsAlmostCertainly) { return keys .buffer(MULTI_GET_WINDOW) .publishOn(dbScheduler) .>>handle((keysWindow, sink) -> { List keyBufsWindow = new ArrayList<>(keysWindow.size()); for (Send bufferSend : keysWindow) { keyBufsWindow.add(bufferSend.receive()); } try { assert !Schedulers.isInNonBlockingThread() : "Called getMulti in a nonblocking thread"; var readOptions = Objects.requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS); List results = db.multiGetAsList(readOptions, LLUtils.toArray(keyBufsWindow)); var mappedResults = new ArrayList>(results.size()); for (int i = 0; i < results.size(); i++) { byte[] val = results.get(i); Optional valueOpt; if (val != null) { // free memory results.set(i, null); valueOpt = Optional.of(LLUtils.fromByteArray(alloc, val)); } else { valueOpt = Optional.empty(); } mappedResults.add(valueOpt); } sink.next(mappedResults); } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to read keys: " + ex.getMessage())); } finally { for (Buffer buffer : keyBufsWindow) { buffer.close(); } } }) .flatMapIterable(list -> list); } @Override public Mono putMulti(Flux> entries) { return entries .buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP)) .publishOn(dbScheduler) .handle((entriesWindowList, sink) -> { var entriesWindow = new ArrayList(entriesWindowList.size()); for (Send entrySend : entriesWindowList) { entriesWindow.add(entrySend.receive()); } try { assert !Schedulers.isInNonBlockingThread() : "Called putMulti in a nonblocking thread"; if (USE_WRITE_BATCHES_IN_PUT_MULTI) { var batch = new CappedWriteBatch(db, alloc, CAPPED_WRITE_BATCH_CAP, RESERVED_WRITE_BATCH_SIZE, MAX_WRITE_BATCH_SIZE, BATCH_WRITE_OPTIONS ); for (LLEntry entry : entriesWindow) { var k = entry.getKey(); var v = entry.getValue(); if (nettyDirect) { batch.put(cfh, k, v); } else { try (var key = k.receive()) { try (var value = v.receive()) { batch.put(cfh, LLUtils.toArray(key), LLUtils.toArray(value)); } } } } batch.writeToDbAndClose(); batch.close(); } else { for (LLEntry entry : entriesWindow) { db.put(EMPTY_WRITE_OPTIONS, entry.getKeyUnsafe(), entry.getValueUnsafe()); } } sink.complete(); } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to write: " + ex.getMessage())); } finally { for (LLEntry llEntry : entriesWindow) { llEntry.close(); } } }) .then(); } @Override public Flux updateMulti(Flux keys, Flux> serializedKeys, KVSerializationFunction, @Nullable Buffer> updateFunction) { return Flux.zip(keys, serializedKeys) .buffer(Math.min(MULTI_GET_WINDOW, CAPPED_WRITE_BATCH_CAP)) .flatMapSequential(ew -> this.>runOnDb(() -> { List> entriesWindow = new ArrayList<>(ew.size()); for (Tuple2> tuple : ew) { entriesWindow.add(tuple.mapT2(Send::receive)); } try { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called updateMulti in a nonblocking thread"); } List keyBufsWindow = new ArrayList<>(entriesWindow.size()); for (Tuple2 objects : entriesWindow) { keyBufsWindow.add(objects.getT2()); } ArrayList, Optional>>> mappedInputs; { var readOptions = Objects.requireNonNullElse(resolveSnapshot(null), EMPTY_READ_OPTIONS); 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() )); } } } var updatedValuesToWrite = new ArrayList(mappedInputs.size()); var valueChangedResult = new ArrayList(mappedInputs.size()); try { for (var mappedInput : mappedInputs) { 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)); } updatedValuesToWrite.add(updatedValue); } else { try (var t3 = mappedInput.getT3().map(Send::receive).orElse(null)) { valueChangedResult.add(!LLUtils.equals(t3, null)); } updatedValuesToWrite.add(null); } } catch (Throwable t) { if (updatedValue != null) { updatedValue.close(); } throw t; } } } finally { for (var mappedInput : mappedInputs) { mappedInput.getT3().ifPresent(Send::close); } } if (USE_WRITE_BATCHES_IN_PUT_MULTI) { var batch = new CappedWriteBatch(db, alloc, CAPPED_WRITE_BATCH_CAP, RESERVED_WRITE_BATCH_SIZE, MAX_WRITE_BATCH_SIZE, BATCH_WRITE_OPTIONS ); int i = 0; for (Tuple2 entry : entriesWindow) { try (var valueToWrite = updatedValuesToWrite.get(i)) { if (valueToWrite == null) { batch.delete(cfh, entry.getT2().send()); } else { batch.put(cfh, entry.getT2().send(), valueToWrite.send()); } } i++; } batch.writeToDbAndClose(); batch.close(); } else { int i = 0; for (Tuple2 entry : entriesWindow) { db.put(EMPTY_WRITE_OPTIONS, entry.getT2(), updatedValuesToWrite.get(i)); i++; } } return valueChangedResult; } finally { for (Tuple2 tuple : entriesWindow) { tuple.getT2().close(); } } }).flatMapIterable(list -> list), /* Max concurrency is 2 to update data while preparing the next segment */ 2); } @Override public Flux> getRange(@Nullable LLSnapshot snapshot, Mono> rangeMono, boolean existsAlmostCertainly) { return rangeMono.flatMapMany(rangeSend -> { try (var range = rangeSend.receive()) { if (range.isSingle()) { var rangeSingleMono = rangeMono.map(r -> r.receive().getSingle()); return getRangeSingle(snapshot, rangeSingleMono, existsAlmostCertainly); } else { return getRangeMulti(snapshot, rangeMono); } } }); } @Override public Flux>> getRangeGrouped(@Nullable LLSnapshot snapshot, Mono> rangeMono, int prefixLength, boolean existsAlmostCertainly) { return rangeMono.flatMapMany(rangeSend -> { try (var range = rangeSend.receive()) { if (range.isSingle()) { var rangeSingleMono = rangeMono.map(r -> r.receive().getSingle()); return getRangeSingle(snapshot, rangeSingleMono, existsAlmostCertainly).map(List::of); } else { return getRangeMultiGrouped(snapshot, rangeMono, prefixLength); } } }); } private Flux> getRangeSingle(LLSnapshot snapshot, Mono> keyMono, boolean existsAlmostCertainly) { return Mono .zip(keyMono, this.get(snapshot, keyMono, existsAlmostCertainly)) .map(result -> LLEntry.of(result.getT1(), result.getT2()).send()) .flux(); } private Flux> getRangeMulti(LLSnapshot snapshot, Mono> rangeMono) { Mono iteratorMono = rangeMono.map(rangeSend -> { ReadOptions resolvedSnapshot = resolveSnapshot(snapshot); return new LLLocalEntryReactiveRocksIterator(db, rangeSend, nettyDirect, resolvedSnapshot); }); return Flux.usingWhen(iteratorMono, iterator -> iterator.flux().subscribeOn(dbScheduler, false), iterator -> Mono.fromRunnable(iterator::close) ); } private Flux>> getRangeMultiGrouped(LLSnapshot snapshot, Mono> rangeMono, int prefixLength) { Mono iteratorMono = rangeMono.map(rangeSend -> { ReadOptions resolvedSnapshot = resolveSnapshot(snapshot); return new LLLocalGroupedEntryReactiveRocksIterator(db, prefixLength, rangeSend, nettyDirect, resolvedSnapshot); }); return Flux.usingWhen( iteratorMono, iterator -> iterator.flux().subscribeOn(dbScheduler, false), iterator -> Mono.fromRunnable(iterator::close) ); } @Override public Flux> getRangeKeys(@Nullable LLSnapshot snapshot, Mono> rangeMono) { return rangeMono.flatMapMany(rangeSend -> { try (var range = rangeSend.receive()) { if (range.isSingle()) { return this.getRangeKeysSingle(snapshot, rangeMono.map(r -> r.receive().getSingle())); } else { return this.getRangeKeysMulti(snapshot, rangeMono); } } }); } @Override public Flux>> getRangeKeysGrouped(@Nullable LLSnapshot snapshot, Mono> rangeMono, int prefixLength) { Mono iteratorMono = rangeMono.map(rangeSend -> { ReadOptions resolvedSnapshot = resolveSnapshot(snapshot); return new LLLocalGroupedKeyReactiveRocksIterator(db, prefixLength, rangeSend, nettyDirect, resolvedSnapshot); }); return Flux.usingWhen(iteratorMono, iterator -> iterator.flux().subscribeOn(dbScheduler, false), iterator -> Mono.fromRunnable(iterator::close) ); } @Override public Flux badBlocks(Mono> rangeMono) { return Flux.usingWhen(rangeMono, rangeSend -> Flux .create(sink -> { var range = rangeSend.receive(); sink.onDispose(range::close); try (var ro = new ReadOptions(getReadOptions(null))) { ro.setFillCache(false); if (!range.isSingle()) { ro.setReadaheadSize(32 * 1024); } ro.setVerifyChecksums(true); try (var rocksIteratorTuple = getRocksIterator(nettyDirect, ro, range, db)) { var rocksIterator = rocksIteratorTuple.iterator(); rocksIterator.seekToFirst(); rocksIterator.status(); while (rocksIterator.isValid() && !sink.isCancelled()) { try { rocksIterator.status(); rocksIterator.key(DUMMY_WRITE_ONLY_BYTE_BUFFER); rocksIterator.status(); rocksIterator.value(DUMMY_WRITE_ONLY_BYTE_BUFFER); rocksIterator.status(); } catch (RocksDBException ex) { sink.next(new BadBlock(databaseName, ColumnUtils.special(columnName), null, ex)); } rocksIterator.next(); } } sink.complete(); } catch (Throwable ex) { sink.error(ex); } }) .subscribeOn(dbScheduler), rangeSend -> Mono.fromRunnable(rangeSend::close) ); } @Override public Flux> getRangeKeyPrefixes(@Nullable LLSnapshot snapshot, Mono> rangeMono, int prefixLength) { Mono iteratorMono = rangeMono.map(range -> { ReadOptions resolvedSnapshot = resolveSnapshot(snapshot); return new LLLocalKeyPrefixReactiveRocksIterator(db, prefixLength, range, nettyDirect, resolvedSnapshot, true); }); return Flux.usingWhen(iteratorMono, iterator -> iterator.flux().subscribeOn(dbScheduler), iterator -> Mono.fromRunnable(iterator::close) ); } private Flux> getRangeKeysSingle(LLSnapshot snapshot, Mono> keyMono) { return keyMono .publishOn(dbScheduler) .>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(); } private Flux> getRangeKeysMulti(LLSnapshot snapshot, Mono> rangeMono) { Mono iteratorMono = rangeMono.map(range -> { ReadOptions resolvedSnapshot = resolveSnapshot(snapshot); return new LLLocalKeyReactiveRocksIterator(db, range, nettyDirect, resolvedSnapshot); }); return Flux.usingWhen(iteratorMono, iterator -> iterator.flux().subscribeOn(dbScheduler, false), iterator -> Mono.fromRunnable(iterator::close) ); } @Override public Mono setRange(Mono> rangeMono, Flux> entries) { if (USE_WINDOW_IN_SET_RANGE) { return rangeMono .publishOn(dbScheduler) .handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called setRange in a nonblocking thread"; if (!USE_WRITE_BATCH_IN_SET_RANGE_DELETE || !USE_WRITE_BATCHES_IN_SET_RANGE) { assert EMPTY_READ_OPTIONS.isOwningHandle(); try (var opts = new ReadOptions(EMPTY_READ_OPTIONS)) { ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, opts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, opts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } assert cfh.isOwningHandle(); assert opts.isOwningHandle(); SafeCloseable seekTo; try (RocksIterator it = db.newIterator(opts)) { if (!PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, it, range.getMinUnsafe()); } else { seekTo = null; it.seekToFirst(); } try { it.status(); while (it.isValid()) { db.delete(EMPTY_WRITE_OPTIONS, it.key()); it.next(); it.status(); } } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } } 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, BATCH_WRITE_OPTIONS )) { if (range.isSingle()) { batch.delete(cfh, range.getSingle()); } else { deleteSmallRangeWriteBatch(batch, range.copy().send()); } batch.writeToDbAndClose(); } } else { try (var batch = new WriteBatch(RESERVED_WRITE_BATCH_SIZE)) { if (range.isSingle()) { batch.delete(cfh, LLUtils.toArray(range.getSingleUnsafe())); } else { deleteSmallRangeWriteBatch(batch, range.copy().send()); } db.write(EMPTY_WRITE_OPTIONS, batch); batch.clear(); } } sink.complete(); } 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 .runOnDb(() -> { List entriesList = new ArrayList<>(entriesListSend.size()); for (Send entrySend : entriesListSend) { entriesList.add(entrySend.receive()); } try { if (!USE_WRITE_BATCHES_IN_SET_RANGE) { for (LLEntry entry : entriesList) { assert entry.isAccessible(); db.put(EMPTY_WRITE_OPTIONS, entry.getKeyUnsafe(), entry.getValueUnsafe()); } } 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, BATCH_WRITE_OPTIONS )) { for (LLEntry entry : entriesList) { assert entry.isAccessible(); if (nettyDirect) { batch.put(cfh, entry.getKey(), entry.getValue()); } else { batch.put(cfh, LLUtils.toArray(entry.getKeyUnsafe()), LLUtils.toArray(entry.getValueUnsafe()) ); } } batch.writeToDbAndClose(); } } else { try (var batch = new WriteBatch(RESERVED_WRITE_BATCH_SIZE)) { for (LLEntry entry : entriesList) { assert entry.isAccessible(); batch.put(cfh, LLUtils.toArray(entry.getKeyUnsafe()), LLUtils.toArray(entry.getValueUnsafe())); } db.write(EMPTY_WRITE_OPTIONS, batch); 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 .getRange(null, rangeMono, false) .publishOn(dbScheduler) .handle((oldValueSend, sink) -> { try (var oldValue = oldValueSend.receive()) { db.delete(EMPTY_WRITE_OPTIONS, oldValue.getKeyUnsafe()); sink.complete(); } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to write range: " + ex.getMessage())); } }) .then(Mono.empty()); var putMono = entries .publishOn(dbScheduler) .handle((entrySend, sink) -> { try (var entry = entrySend.receive()) { if (entry.getKeyUnsafe() != null && entry.getValueUnsafe() != null) { this.put(entry.getKeyUnsafe(), entry.getValueUnsafe()); } sink.complete(); } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to write range: " + ex.getMessage())); } }) .then(Mono.empty()); return deleteMono.then(putMono); } } //todo: this is broken, check why. (is this still true?) private void deleteSmallRangeWriteBatch(CappedWriteBatch writeBatch, Send rangeToReceive) throws RocksDBException { var range = rangeToReceive.receive(); try (var readOpts = new ReadOptions(getReadOptions(null))) { readOpts.setFillCache(false); ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try (var rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { rocksIterator.status(); while (rocksIterator.isValid()) { writeBatch.delete(cfh, LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send()); rocksIterator.next(); rocksIterator.status(); } } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } catch (Throwable e) { range.close(); throw e; } } private void deleteSmallRangeWriteBatch(WriteBatch writeBatch, Send rangeToReceive) throws RocksDBException { try (var range = rangeToReceive.receive()) { try (var readOpts = new ReadOptions(getReadOptions(null))) { readOpts.setFillCache(false); ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try (var rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { rocksIterator.status(); while (rocksIterator.isValid()) { writeBatch.delete(cfh, rocksIterator.key()); rocksIterator.next(); rocksIterator.status(); } } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } } } @Nullable private static SafeCloseable rocksIterSeekTo(boolean allowNettyDirect, RocksIterator rocksIterator, Buffer key) { if (allowNettyDirect && isReadOnlyDirect(key)) { ByteBuffer keyInternalByteBuffer = ((ReadableComponent) key).readableBuffer(); assert keyInternalByteBuffer.position() == 0; rocksIterator.seek(keyInternalByteBuffer); // This is useful to retain the key buffer in memory and avoid deallocations return key::isAccessible; } else { rocksIterator.seek(LLUtils.toArray(key)); return null; } } /** * This method should not modify or move the writerIndex/readerIndex of the key */ private static ReleasableSlice setIterateBound(boolean allowNettyDirect, ReadOptions readOpts, IterateBound boundType, Buffer key) { requireNonNull(key); AbstractSlice slice; if (allowNettyDirect && LLLocalDictionary.USE_DIRECT_BUFFER_BOUNDS && (isReadOnlyDirect(key))) { ByteBuffer keyInternalByteBuffer = ((ReadableComponent) key).readableBuffer(); assert keyInternalByteBuffer.position() == 0; slice = new DirectSlice(keyInternalByteBuffer, key.readableBytes()); assert slice.size() == key.readableBytes(); } else { slice = new Slice(requireNonNull(LLUtils.toArray(key))); } if (boundType == IterateBound.LOWER) { readOpts.setIterateLowerBound(slice); } else { readOpts.setIterateUpperBound(slice); } return new ReleasableSliceImplWithRelease(slice); } private static ReleasableSlice emptyReleasableSlice() { var arr = new byte[0]; return new ReleasableSliceImplWithoutRelease(new Slice(arr)); } /** * This method should not modify or move the writerIndex/readerIndex of the key */ public record ReleasableSliceImplWithoutRelease(AbstractSlice slice) implements ReleasableSlice {} /** * This class should not modify or move the writerIndex/readerIndex of the key */ public record ReleasableSliceImplWithRelease(AbstractSlice slice) implements ReleasableSlice { @Override public void close() { slice.close(); } } public Mono clear() { return Mono .fromCallable(() -> { assert !Schedulers.isInNonBlockingThread() : "Called clear in a nonblocking thread"; boolean shouldCompactLater = false; try (var readOpts = new ReadOptions(getReadOptions(null))) { readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED); // readOpts.setIgnoreRangeDeletions(true); readOpts.setFillCache(false); readOpts.setReadaheadSize(32 * 1024); // 32KiB try (CappedWriteBatch writeBatch = new CappedWriteBatch(db, alloc, CAPPED_WRITE_BATCH_CAP, RESERVED_WRITE_BATCH_SIZE, MAX_WRITE_BATCH_SIZE, BATCH_WRITE_OPTIONS )) { byte[] firstDeletedKey = null; byte[] lastDeletedKey = null; try (RocksIterator rocksIterator = db.newIterator(readOpts)) { // If the database supports transactions, delete each key one by one if (db.supportsTransactions()) { rocksIterator.seekToFirst(); rocksIterator.status(); while (rocksIterator.isValid()) { writeBatch.delete(cfh, rocksIterator.key()); rocksIterator.next(); rocksIterator.status(); } } else { rocksIterator.seekToLast(); rocksIterator.status(); if (rocksIterator.isValid()) { firstDeletedKey = FIRST_KEY; lastDeletedKey = rocksIterator.key(); writeBatch.deleteRange(cfh, FIRST_KEY, rocksIterator.key()); writeBatch.delete(cfh, rocksIterator.key()); shouldCompactLater = true; } } } writeBatch.writeToDbAndClose(); if (shouldCompactLater) { // Compact range db.suggestCompactRange(); if (lastDeletedKey != null) { db.compactRange(firstDeletedKey, lastDeletedKey, new CompactRangeOptions() .setAllowWriteStall(false) .setExclusiveManualCompaction(false) .setChangeLevel(false) ); } } db.flush(new FlushOptions().setWaitForFlush(true).setAllowWriteStall(true)); db.flushWal(true); } return null; } }) .onErrorMap(cause -> new IOException("Failed to clear", cause)) .subscribeOn(dbScheduler); } @Override public Mono sizeRange(@Nullable LLSnapshot snapshot, Mono> rangeMono, boolean fast) { return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called sizeRange in a nonblocking thread"; if (range.isAll()) { sink.next(fast ? fastSizeAll(snapshot) : exactSizeAll(snapshot)); } else { try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) { readOpts.setFillCache(false); readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED); ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try { if (fast) { readOpts.setIgnoreRangeDeletions(true); } try (var rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { long i = 0; rocksIterator.status(); while (rocksIterator.isValid()) { rocksIterator.next(); rocksIterator.status(); i++; } sink.next(i); } finally { if (seekTo != null) { seekTo.close(); } } } } finally { maxBound.close(); } } finally { minBound.close(); } } } } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to get size of range: " + ex.getMessage())); } }); } @Override public Mono> getOne(@Nullable LLSnapshot snapshot, Mono> rangeMono) { return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called getOne in a nonblocking thread"; try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) { ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try (var rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { rocksIterator.status(); 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()); } } } else { sink.complete(); } } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to get one entry: " + ex.getMessage())); } }); } @Override public Mono> getOneKey(@Nullable LLSnapshot snapshot, Mono> rangeMono) { return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called getOneKey in a nonblocking thread"; try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) { ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try (var rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { rocksIterator.status(); if (rocksIterator.isValid()) { sink.next(LLUtils.readDirectNioBuffer(alloc, rocksIterator::key).send()); } else { sink.complete(); } } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } } catch (RocksDBException ex) { sink.error(new RocksDBException("Failed to get one key: " + ex.getMessage())); } }); } private long fastSizeAll(@Nullable LLSnapshot snapshot) throws RocksDBException { try (var rocksdbSnapshot = new ReadOptions(resolveSnapshot(snapshot))) { if (USE_CURRENT_FASTSIZE_FOR_OLD_SNAPSHOTS || rocksdbSnapshot.snapshot() == null) { try { return db.getLongProperty("rocksdb.estimate-num-keys"); } catch (RocksDBException e) { logger.error(MARKER_ROCKSDB, "Failed to get RocksDB estimated keys count property", e); return 0; } } else if (PARALLEL_EXACT_SIZE) { return exactSizeAll(snapshot); } else { rocksdbSnapshot.setFillCache(false); rocksdbSnapshot.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED); rocksdbSnapshot.setIgnoreRangeDeletions(true); long count = 0; try (RocksIterator rocksIterator = db.newIterator(rocksdbSnapshot)) { rocksIterator.seekToFirst(); rocksIterator.status(); // If it's a fast size of a snapshot, count only up to 100'000 elements while (rocksIterator.isValid() && count < 100_000) { count++; rocksIterator.next(); rocksIterator.status(); } return count; } } } } private long exactSizeAll(@Nullable LLSnapshot snapshot) { if (Schedulers.isInNonBlockingThread()) { throw new UnsupportedOperationException("Called exactSizeAll in a nonblocking thread"); } try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) { readOpts.setFillCache(false); readOpts.setReadaheadSize(128 * 1024); // 128KiB readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED); 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 partialCount = 0; try (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 rocksIterator = db.newIterator(rangeReadOpts)) { rocksIterator.seekToFirst(); rocksIterator.status(); while (rocksIterator.isValid()) { partialCount++; rocksIterator.next(); rocksIterator.status(); } return partialCount; } } finally { if (sliceBegin != null) { sliceBegin.close(); } if (sliceEnd != null) { sliceEnd.close(); } } } }) .map(commonPool::submit) .toList(); long count = 0; for (ForkJoinTask future : futures) { count += future.join(); } return count; } else { long count = 0; try (RocksIterator iter = db.newIterator(readOpts)) { iter.seekToFirst(); while (iter.isValid()) { count++; iter.next(); } return count; } } } } @Override public Mono> removeOne(Mono> rangeMono) { return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> { try (var range = rangeSend.receive()) { assert !Schedulers.isInNonBlockingThread() : "Called removeOne in a nonblocking thread"; try (var readOpts = new ReadOptions(getReadOptions(null))) { ReleasableSlice minBound; if (range.hasMin()) { minBound = setIterateBound(nettyDirect, readOpts, IterateBound.LOWER, range.getMinUnsafe()); } else { minBound = emptyReleasableSlice(); } try { ReleasableSlice maxBound; if (range.hasMax()) { maxBound = setIterateBound(nettyDirect, readOpts, IterateBound.UPPER, range.getMaxUnsafe()); } else { maxBound = emptyReleasableSlice(); } try (RocksIterator rocksIterator = db.newIterator(readOpts)) { SafeCloseable seekTo; if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = rocksIterSeekTo(nettyDirect, rocksIterator, range.getMinUnsafe()); } else { seekTo = null; rocksIterator.seekToFirst(); } try { rocksIterator.status(); if (!rocksIterator.isValid()) { sink.complete(); return; } Buffer key = LLUtils.readDirectNioBuffer(alloc, rocksIterator::key); Buffer value = LLUtils.readDirectNioBuffer(alloc, rocksIterator::value); db.delete(EMPTY_WRITE_OPTIONS, key); sink.next(LLEntry.of(key, value).send()); } finally { if (seekTo != null) { seekTo.close(); } } } finally { maxBound.close(); } } finally { minBound.close(); } } } catch (RocksDBException ex) { sink.error(ex); } }); } /** * This method should not modify or move the writerIndex/readerIndex of the buffers inside the range */ @NotNull public static RocksIteratorTuple getRocksIterator( boolean allowNettyDirect, ReadOptions readOptions, LLRange range, RocksDBColumn db) { assert !Schedulers.isInNonBlockingThread() : "Called getRocksIterator in a nonblocking thread"; ReleasableSlice sliceMin; ReleasableSlice sliceMax; if (range.hasMin()) { sliceMin = setIterateBound(allowNettyDirect, readOptions, IterateBound.LOWER, range.getMinUnsafe()); } else { sliceMin = emptyReleasableSlice(); } if (range.hasMax()) { sliceMax = setIterateBound(allowNettyDirect, readOptions, IterateBound.UPPER, range.getMaxUnsafe()); } else { sliceMax = emptyReleasableSlice(); } var rocksIterator = db.newIterator(readOptions); SafeCloseable seekTo; if (!PREFER_SEEK_TO_FIRST && range.hasMin()) { seekTo = Objects.requireNonNullElseGet(rocksIterSeekTo(allowNettyDirect, rocksIterator, range.getMinUnsafe()), () -> ((SafeCloseable) () -> {})); } else { seekTo = () -> {}; rocksIterator.seekToFirst(); } return new RocksIteratorTuple(List.of(readOptions), rocksIterator, sliceMin, sliceMax, seekTo); } }