Various local dict optimizations, customize fillCache in containsRange
This commit is contained in:
parent
cf53eb4f5a
commit
cdb65b31f3
@ -126,7 +126,7 @@ public interface LLDictionary extends LLKeyValueDatabaseStructure {
|
|||||||
return replaceRange(range, canKeysChange, entriesReplacer, false);
|
return replaceRange(range, canKeysChange, entriesReplacer, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range);
|
Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean fillCache);
|
||||||
|
|
||||||
Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean fast);
|
Mono<Long> sizeRange(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> range, boolean fast);
|
||||||
|
|
||||||
|
@ -181,7 +181,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
||||||
return dictionary.isRangeEmpty(resolveSnapshot(snapshot), rangeMono);
|
return dictionary.isRangeEmpty(resolveSnapshot(snapshot), rangeMono, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -194,7 +194,8 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
|
|||||||
public Mono<Boolean> containsKey(@Nullable CompositeSnapshot snapshot, T keySuffix) {
|
public Mono<Boolean> containsKey(@Nullable CompositeSnapshot snapshot, T keySuffix) {
|
||||||
return dictionary
|
return dictionary
|
||||||
.isRangeEmpty(resolveSnapshot(snapshot),
|
.isRangeEmpty(resolveSnapshot(snapshot),
|
||||||
Mono.fromCallable(() -> LLRange.singleUnsafe(serializeKeySuffixToKey(keySuffix)).send())
|
Mono.fromCallable(() -> LLRange.singleUnsafe(serializeKeySuffixToKey(keySuffix)).send()),
|
||||||
|
true
|
||||||
)
|
)
|
||||||
.map(empty -> !empty);
|
.map(empty -> !empty);
|
||||||
}
|
}
|
||||||
|
@ -287,7 +287,7 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
||||||
return dictionary.isRangeEmpty(resolveSnapshot(snapshot), rangeMono);
|
return dictionary.isRangeEmpty(resolveSnapshot(snapshot), rangeMono, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -181,14 +181,14 @@ public class DatabaseSingle<U> extends ResourceSupport<DatabaseStage<U>, Databas
|
|||||||
@Override
|
@Override
|
||||||
public Mono<Long> leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) {
|
public Mono<Long> leavesCount(@Nullable CompositeSnapshot snapshot, boolean fast) {
|
||||||
return dictionary
|
return dictionary
|
||||||
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send))
|
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send), false)
|
||||||
.map(empty -> empty ? 0L : 1L);
|
.map(empty -> empty ? 0L : 1L);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
public Mono<Boolean> isEmpty(@Nullable CompositeSnapshot snapshot) {
|
||||||
return dictionary
|
return dictionary
|
||||||
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send));
|
.isRangeEmpty(resolveSnapshot(snapshot), keyMono.map(LLRange::single).map(ResourceSupport::send), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -12,7 +12,6 @@ import io.micrometer.core.instrument.Counter;
|
|||||||
import io.micrometer.core.instrument.Timer;
|
import io.micrometer.core.instrument.Timer;
|
||||||
import io.net5.buffer.api.Buffer;
|
import io.net5.buffer.api.Buffer;
|
||||||
import io.net5.buffer.api.BufferAllocator;
|
import io.net5.buffer.api.BufferAllocator;
|
||||||
import io.net5.buffer.api.Resource;
|
|
||||||
import io.net5.buffer.api.Send;
|
import io.net5.buffer.api.Send;
|
||||||
import io.net5.buffer.api.internal.ResourceSupport;
|
import io.net5.buffer.api.internal.ResourceSupport;
|
||||||
import io.net5.util.internal.PlatformDependent;
|
import io.net5.util.internal.PlatformDependent;
|
||||||
@ -36,7 +35,6 @@ import java.nio.ByteBuffer;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
@ -65,7 +63,6 @@ import org.rocksdb.Snapshot;
|
|||||||
import org.rocksdb.WriteBatch;
|
import org.rocksdb.WriteBatch;
|
||||||
import org.rocksdb.WriteOptions;
|
import org.rocksdb.WriteOptions;
|
||||||
import reactor.core.publisher.Flux;
|
import reactor.core.publisher.Flux;
|
||||||
import reactor.core.publisher.Hooks;
|
|
||||||
import reactor.core.publisher.Mono;
|
import reactor.core.publisher.Mono;
|
||||||
import reactor.core.scheduler.Scheduler;
|
import reactor.core.scheduler.Scheduler;
|
||||||
import reactor.core.scheduler.Schedulers;
|
import reactor.core.scheduler.Schedulers;
|
||||||
@ -263,7 +260,7 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
boolean existsAlmostCertainly) {
|
boolean existsAlmostCertainly) {
|
||||||
return keyMono
|
return keyMono
|
||||||
.publishOn(dbScheduler)
|
.publishOn(dbScheduler)
|
||||||
.<Send<Buffer>>handle((keySend, sink) -> {
|
.handle((keySend, sink) -> {
|
||||||
try (var key = keySend.receive()) {
|
try (var key = keySend.receive()) {
|
||||||
try {
|
try {
|
||||||
var readOptions = requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
|
var readOptions = requireNonNullElse(resolveSnapshot(snapshot), EMPTY_READ_OPTIONS);
|
||||||
@ -280,90 +277,86 @@ public class LLLocalDictionary implements LLDictionary {
|
|||||||
} else {
|
} else {
|
||||||
sink.complete();
|
sink.complete();
|
||||||
}
|
}
|
||||||
|
} catch (RocksDBException ex) {
|
||||||
|
sink.error(new IOException("Failed to read " + toStringSafe(key) + ": " + ex.getMessage()));
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
sink.error(new IOException("Failed to read " + toStringSafe(key), ex));
|
sink.error(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
|
||||||
.onErrorMap(cause -> new IOException("Failed to read", cause));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
|
||||||
return rangeMono
|
|
||||||
.publishOn(dbScheduler)
|
|
||||||
.handle((rangeSend, sink) -> {
|
|
||||||
try (var range = rangeSend.receive()) {
|
|
||||||
boolean rangeEmpty = !containsRange(snapshot, range);
|
|
||||||
sink.next(rangeEmpty);
|
|
||||||
} catch (Throwable ex) {
|
|
||||||
sink.error(ex);
|
|
||||||
}
|
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean containsRange(@Nullable LLSnapshot snapshot, LLRange range) throws RocksDBException {
|
@Override
|
||||||
assert !Schedulers.isInNonBlockingThread() : "Called containsRange in a nonblocking thread";
|
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fillCache) {
|
||||||
startedContains.increment();
|
return rangeMono.publishOn(dbScheduler).handle((rangeSend, sink) -> {
|
||||||
try {
|
try (var range = rangeSend.receive()) {
|
||||||
var result = containsTime.recordCallable(() -> {
|
assert !Schedulers.isInNonBlockingThread() : "Called isRangeEmpty in a nonblocking thread";
|
||||||
if (range.isSingle()) {
|
startedContains.increment();
|
||||||
var unmodifiableReadOpts = resolveSnapshot(snapshot);
|
try {
|
||||||
return db.exists(unmodifiableReadOpts, range.getSingleUnsafe());
|
var result = containsTime.recordCallable(() -> {
|
||||||
} else {
|
if (range.isSingle()) {
|
||||||
// Temporary resources to release after finished
|
return !containsKey(snapshot, range.getSingleUnsafe());
|
||||||
AbstractSlice<?> slice1 = null;
|
} else {
|
||||||
AbstractSlice<?> slice2 = null;
|
// Temporary resources to release after finished
|
||||||
try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) {
|
AbstractSlice<?> slice1 = null;
|
||||||
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
AbstractSlice<?> slice2 = null;
|
||||||
readOpts.setFillCache(false);
|
|
||||||
if (range.hasMin()) {
|
try (var readOpts = new ReadOptions(resolveSnapshot(snapshot))) {
|
||||||
var rangeMinInternalByteBuffer = asReadOnlyDirect(range.getMinUnsafe());
|
readOpts.setVerifyChecksums(VERIFY_CHECKSUMS_WHEN_NOT_NEEDED);
|
||||||
if (nettyDirect && rangeMinInternalByteBuffer != null) {
|
readOpts.setFillCache(fillCache);
|
||||||
readOpts.setIterateLowerBound(slice1 = new DirectSlice(rangeMinInternalByteBuffer,
|
if (range.hasMin()) {
|
||||||
range.getMinUnsafe().readableBytes()));
|
var rangeMinInternalByteBuffer = asReadOnlyDirect(range.getMinUnsafe());
|
||||||
} else {
|
if (nettyDirect && rangeMinInternalByteBuffer != null) {
|
||||||
readOpts.setIterateLowerBound(slice1 = new Slice(LLUtils.toArray(range.getMinUnsafe())));
|
readOpts.setIterateLowerBound(slice1 = new DirectSlice(rangeMinInternalByteBuffer,
|
||||||
}
|
range.getMinUnsafe().readableBytes()
|
||||||
}
|
));
|
||||||
if (range.hasMax()) {
|
} else {
|
||||||
var rangeMaxInternalByteBuffer = asReadOnlyDirect(range.getMaxUnsafe());
|
readOpts.setIterateLowerBound(slice1 = new Slice(LLUtils.toArray(range.getMinUnsafe())));
|
||||||
if (nettyDirect && rangeMaxInternalByteBuffer != null) {
|
}
|
||||||
readOpts.setIterateUpperBound(slice2 = new DirectSlice(rangeMaxInternalByteBuffer,
|
}
|
||||||
range.getMaxUnsafe().readableBytes()));
|
if (range.hasMax()) {
|
||||||
} else {
|
var rangeMaxInternalByteBuffer = asReadOnlyDirect(range.getMaxUnsafe());
|
||||||
readOpts.setIterateUpperBound(slice2 = new Slice(LLUtils.toArray(range.getMaxUnsafe())));
|
if (nettyDirect && rangeMaxInternalByteBuffer != null) {
|
||||||
}
|
readOpts.setIterateUpperBound(slice2 = new DirectSlice(rangeMaxInternalByteBuffer,
|
||||||
}
|
range.getMaxUnsafe().readableBytes()
|
||||||
try (RocksIterator rocksIterator = db.newIterator(readOpts)) {
|
));
|
||||||
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
} else {
|
||||||
var rangeMinInternalByteBuffer = asReadOnlyDirect(range.getMinUnsafe());
|
readOpts.setIterateUpperBound(slice2 = new Slice(LLUtils.toArray(range.getMaxUnsafe())));
|
||||||
if (nettyDirect && rangeMinInternalByteBuffer != null) {
|
}
|
||||||
rocksIterator.seek(rangeMinInternalByteBuffer);
|
}
|
||||||
} else {
|
try (RocksIterator rocksIterator = db.newIterator(readOpts)) {
|
||||||
rocksIterator.seek(LLUtils.toArray(range.getMinUnsafe()));
|
if (!LLLocalDictionary.PREFER_SEEK_TO_FIRST && range.hasMin()) {
|
||||||
|
var rangeMinInternalByteBuffer = asReadOnlyDirect(range.getMinUnsafe());
|
||||||
|
if (nettyDirect && rangeMinInternalByteBuffer != null) {
|
||||||
|
rocksIterator.seek(rangeMinInternalByteBuffer);
|
||||||
|
} 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();
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
rocksIterator.seekToFirst();
|
|
||||||
}
|
}
|
||||||
rocksIterator.status();
|
|
||||||
return rocksIterator.isValid();
|
|
||||||
}
|
}
|
||||||
} finally {
|
});
|
||||||
if (slice1 != null) slice1.close();
|
assert result != null;
|
||||||
if (slice2 != null) slice2.close();
|
sink.next(!result);
|
||||||
}
|
} finally {
|
||||||
|
endedContains.increment();
|
||||||
}
|
}
|
||||||
});
|
} catch (Throwable ex) {
|
||||||
assert result != null;
|
sink.error(ex);
|
||||||
return result;
|
}
|
||||||
} catch (RocksDBException | RuntimeException e) {
|
});
|
||||||
throw e;
|
|
||||||
} catch (Exception ex) {
|
|
||||||
throw new RuntimeException(ex);
|
|
||||||
} finally {
|
|
||||||
endedContains.increment();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Mono<Boolean> containsKey(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
|
private Mono<Boolean> containsKey(@Nullable LLSnapshot snapshot, Mono<Send<Buffer>> keyMono) {
|
||||||
|
@ -516,7 +516,7 @@ public class LLMemoryDictionary implements LLDictionary {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono) {
|
public Mono<Boolean> isRangeEmpty(@Nullable LLSnapshot snapshot, Mono<Send<LLRange>> rangeMono, boolean fillCache) {
|
||||||
return getRangeKeys(snapshot, rangeMono)
|
return getRangeKeys(snapshot, rangeMono)
|
||||||
.map(buf -> {
|
.map(buf -> {
|
||||||
buf.receive().close();
|
buf.receive().close();
|
||||||
|
Loading…
Reference in New Issue
Block a user