Database options record
This commit is contained in:
parent
507101e453
commit
c6d5beb33b
@ -3,6 +3,7 @@ package it.cavallium.dbengine.database;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import it.cavallium.dbengine.client.IndicizerAnalyzers;
|
||||
import it.cavallium.dbengine.client.IndicizerSimilarities;
|
||||
import it.cavallium.dbengine.database.disk.DatabaseOptions;
|
||||
import it.cavallium.dbengine.lucene.analyzer.TextFieldsAnalyzer;
|
||||
import it.cavallium.dbengine.lucene.analyzer.TextFieldsSimilarity;
|
||||
import java.time.Duration;
|
||||
@ -19,9 +20,7 @@ public interface LLDatabaseConnection {
|
||||
|
||||
Mono<? extends LLKeyValueDatabase> getDatabase(String name,
|
||||
List<Column> columns,
|
||||
Map<String, String> extraFlags,
|
||||
boolean lowMemory,
|
||||
boolean inMemory);
|
||||
DatabaseOptions databaseOptions);
|
||||
|
||||
Mono<? extends LLLuceneIndex> getLuceneIndex(String name,
|
||||
int instancesCount,
|
||||
|
@ -0,0 +1,14 @@
|
||||
package it.cavallium.dbengine.database.disk;
|
||||
|
||||
import io.soabase.recordbuilder.core.RecordBuilder;
|
||||
import it.cavallium.dbengine.database.Column;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@RecordBuilder
|
||||
public record DatabaseOptions(Map<String, String> extraFlags,
|
||||
boolean absoluteConsistency,
|
||||
boolean lowMemory,
|
||||
boolean inMemory,
|
||||
boolean useDirectIO,
|
||||
boolean allowMemoryMapping) {}
|
@ -26,12 +26,10 @@ public class LLLocalDatabaseConnection implements LLDatabaseConnection {
|
||||
|
||||
private final ByteBufAllocator allocator;
|
||||
private final Path basePath;
|
||||
private final boolean crashIfWalError;
|
||||
|
||||
public LLLocalDatabaseConnection(ByteBufAllocator allocator, Path basePath, boolean crashIfWalError) {
|
||||
public LLLocalDatabaseConnection(ByteBufAllocator allocator, Path basePath) {
|
||||
this.allocator = allocator;
|
||||
this.basePath = basePath;
|
||||
this.crashIfWalError = crashIfWalError;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -54,9 +52,7 @@ public class LLLocalDatabaseConnection implements LLDatabaseConnection {
|
||||
@Override
|
||||
public Mono<LLLocalKeyValueDatabase> getDatabase(String name,
|
||||
List<Column> columns,
|
||||
Map<String, String> extraFlags,
|
||||
boolean lowMemory,
|
||||
boolean inMemory) {
|
||||
DatabaseOptions databaseOptions) {
|
||||
return Mono
|
||||
.fromCallable(() -> new LLLocalKeyValueDatabase(
|
||||
allocator,
|
||||
@ -64,10 +60,7 @@ public class LLLocalDatabaseConnection implements LLDatabaseConnection {
|
||||
basePath.resolve("database_" + name),
|
||||
columns,
|
||||
new LinkedList<>(),
|
||||
extraFlags,
|
||||
crashIfWalError,
|
||||
lowMemory,
|
||||
inMemory
|
||||
databaseOptions
|
||||
))
|
||||
.subscribeOn(Schedulers.boundedElastic());
|
||||
}
|
||||
|
@ -20,7 +20,6 @@ import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
@ -43,12 +42,10 @@ import org.rocksdb.RateLimiter;
|
||||
import org.rocksdb.RocksDB;
|
||||
import org.rocksdb.RocksDBException;
|
||||
import org.rocksdb.Snapshot;
|
||||
import org.rocksdb.TableFormatConfig;
|
||||
import org.rocksdb.WALRecoveryMode;
|
||||
import org.rocksdb.WriteBufferManager;
|
||||
import org.warp.commonutils.log.Logger;
|
||||
import org.warp.commonutils.log.LoggerFactory;
|
||||
import reactor.core.publisher.Flux;
|
||||
import reactor.core.publisher.Mono;
|
||||
import reactor.core.scheduler.Scheduler;
|
||||
import reactor.core.scheduler.Schedulers;
|
||||
@ -59,33 +56,35 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
|
||||
private static final boolean USE_DIRECT_IO = true;
|
||||
protected static final Logger logger = LoggerFactory.getLogger(LLLocalKeyValueDatabase.class);
|
||||
private static final ColumnFamilyDescriptor DEFAULT_COLUMN_FAMILY = new ColumnFamilyDescriptor(
|
||||
RocksDB.DEFAULT_COLUMN_FAMILY);
|
||||
|
||||
private final ByteBufAllocator allocator;
|
||||
private final Scheduler dbScheduler;
|
||||
|
||||
// Configurations
|
||||
|
||||
private final Path dbPath;
|
||||
private final boolean inMemory;
|
||||
private final String name;
|
||||
private final DatabaseOptions databaseOptions;
|
||||
|
||||
private final boolean enableColumnsBug;
|
||||
private RocksDB db;
|
||||
private final Map<Column, ColumnFamilyHandle> handles;
|
||||
private final ConcurrentHashMap<Long, Snapshot> snapshotsHandles = new ConcurrentHashMap<>();
|
||||
private final AtomicLong nextSnapshotNumbers = new AtomicLong(1);
|
||||
|
||||
@SuppressWarnings("SwitchStatementWithTooFewBranches")
|
||||
public LLLocalKeyValueDatabase(ByteBufAllocator allocator,
|
||||
String name,
|
||||
Path path,
|
||||
List<Column> columns,
|
||||
List<ColumnFamilyHandle> handles,
|
||||
Map<String, String> extraFlags,
|
||||
boolean crashIfWalError,
|
||||
boolean lowMemory,
|
||||
boolean inMemory) throws IOException {
|
||||
DatabaseOptions databaseOptions) throws IOException {
|
||||
this.name = name;
|
||||
this.allocator = allocator;
|
||||
Options options = openRocksDb(path, crashIfWalError, lowMemory);
|
||||
Options rocksdbOptions = openRocksDb(path, databaseOptions);
|
||||
try {
|
||||
List<ColumnFamilyDescriptor> descriptors = new LinkedList<>();
|
||||
descriptors
|
||||
@ -97,32 +96,38 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
|
||||
// Get databases directory path
|
||||
Path databasesDirPath = path.toAbsolutePath().getParent();
|
||||
|
||||
String dbPathString = databasesDirPath.toString() + File.separatorChar + path.getFileName();
|
||||
Path dbPath = Paths.get(dbPathString);
|
||||
this.dbPath = dbPath;
|
||||
this.inMemory = inMemory;
|
||||
this.name = name;
|
||||
this.dbScheduler = Schedulers.newBoundedElastic(lowMemory ? Runtime.getRuntime().availableProcessors()
|
||||
: Math.max(8, Runtime.getRuntime().availableProcessors()),
|
||||
|
||||
// Set options
|
||||
this.databaseOptions = databaseOptions;
|
||||
|
||||
int threadCap;
|
||||
if (databaseOptions.lowMemory()) {
|
||||
threadCap = Math.max(8, Runtime.getRuntime().availableProcessors());
|
||||
} else {
|
||||
threadCap = Runtime.getRuntime().availableProcessors();
|
||||
}
|
||||
this.dbScheduler = Schedulers.newBoundedElastic(threadCap,
|
||||
Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
|
||||
"db-" + name,
|
||||
60,
|
||||
true
|
||||
);
|
||||
this.enableColumnsBug = "true".equals(extraFlags.getOrDefault("enableColumnBug", "false"));
|
||||
this.enableColumnsBug = "true".equals(databaseOptions.extraFlags().getOrDefault("enableColumnBug", "false"));
|
||||
|
||||
createIfNotExists(descriptors, options, inMemory, this.dbPath, dbPathString);
|
||||
createIfNotExists(descriptors, rocksdbOptions, databaseOptions, dbPath, dbPathString);
|
||||
|
||||
// Create all column families that don't exist
|
||||
createAllColumns(descriptors, options, inMemory, dbPathString);
|
||||
createAllColumns(descriptors, rocksdbOptions, databaseOptions, dbPathString);
|
||||
|
||||
while (true) {
|
||||
try {
|
||||
// a factory method that returns a RocksDB instance
|
||||
this.db = RocksDB.open(new DBOptions(options),
|
||||
this.db = RocksDB.open(new DBOptions(rocksdbOptions),
|
||||
dbPathString,
|
||||
inMemory ? List.of(DEFAULT_COLUMN_FAMILY) : descriptors,
|
||||
databaseOptions.inMemory() ? List.of(DEFAULT_COLUMN_FAMILY) : descriptors,
|
||||
handles
|
||||
);
|
||||
break;
|
||||
@ -130,19 +135,19 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
switch (ex.getMessage()) {
|
||||
case "Direct I/O is not supported by the specified DB." -> {
|
||||
logger.warn(ex.getLocalizedMessage());
|
||||
options
|
||||
rocksdbOptions
|
||||
.setUseDirectReads(false)
|
||||
.setUseDirectIoForFlushAndCompaction(false)
|
||||
.setAllowMmapReads(true)
|
||||
.setAllowMmapWrites(true);
|
||||
.setAllowMmapReads(databaseOptions.allowMemoryMapping())
|
||||
.setAllowMmapWrites(databaseOptions.allowMemoryMapping());
|
||||
}
|
||||
default -> throw ex;
|
||||
}
|
||||
}
|
||||
}
|
||||
createInMemoryColumns(descriptors, inMemory, handles);
|
||||
createInMemoryColumns(descriptors, databaseOptions, handles);
|
||||
this.handles = new HashMap<>();
|
||||
if (enableColumnsBug) {
|
||||
if (enableColumnsBug && !databaseOptions.inMemory()) {
|
||||
for (int i = 0; i < columns.size(); i++) {
|
||||
this.handles.put(columns.get(i), handles.get(i));
|
||||
}
|
||||
@ -220,8 +225,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
}
|
||||
|
||||
@SuppressWarnings({"CommentedOutCode", "PointlessArithmeticExpression"})
|
||||
private static Options openRocksDb(Path path, boolean crashIfWalError, boolean lowMemory)
|
||||
throws IOException {
|
||||
private static Options openRocksDb(Path path, DatabaseOptions databaseOptions) throws IOException {
|
||||
// Get databases directory path
|
||||
Path databasesDirPath = path.toAbsolutePath().getParent();
|
||||
// Create base directories
|
||||
@ -246,7 +250,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
options.setMaxWriteBufferNumber(4);
|
||||
options.setAvoidFlushDuringShutdown(false); // Flush all WALs during shutdown
|
||||
options.setAvoidFlushDuringRecovery(false); // Flush all WALs during startup
|
||||
options.setWalRecoveryMode(crashIfWalError
|
||||
options.setWalRecoveryMode(databaseOptions.absoluteConsistency()
|
||||
? WALRecoveryMode.AbsoluteConsistency
|
||||
: WALRecoveryMode.PointInTimeRecovery); // Crash if the WALs are corrupted.Default: TolerateCorruptedTailRecords
|
||||
options.setDeleteObsoleteFilesPeriodMicros(20 * 1000000); // 20 seconds
|
||||
@ -268,7 +272,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
//options.setWritableFileMaxBufferSize(1024 * 1024); // 1MB by default
|
||||
//options.setCompactionReadaheadSize(2 * 1024 * 1024); // recommend at least 2MB
|
||||
final BlockBasedTableConfig tableOptions = new BlockBasedTableConfig();
|
||||
if (lowMemory) {
|
||||
if (databaseOptions.lowMemory()) {
|
||||
// LOW MEMORY
|
||||
options
|
||||
.setLevelCompactionDynamicLevelBytes(false)
|
||||
@ -315,10 +319,10 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
tableOptions.setFilterPolicy(bloomFilter);
|
||||
options.setWriteBufferManager(new WriteBufferManager(256L * 1024L * 1024L, new LRUCache(128L * 1024L * 1024L))); // 128MiB
|
||||
|
||||
if (USE_DIRECT_IO) {
|
||||
if (databaseOptions.useDirectIO()) {
|
||||
options
|
||||
.setAllowMmapReads(false)
|
||||
.setAllowMmapWrites(false)
|
||||
.setAllowMmapReads(databaseOptions.allowMemoryMapping())
|
||||
.setAllowMmapWrites(databaseOptions.allowMemoryMapping())
|
||||
.setUseDirectIoForFlushAndCompaction(true)
|
||||
.setUseDirectReads(true)
|
||||
// Option to enable readahead in compaction
|
||||
@ -329,8 +333,8 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
;
|
||||
} else {
|
||||
options
|
||||
.setAllowMmapReads(true)
|
||||
.setAllowMmapWrites(true);
|
||||
.setAllowMmapReads(databaseOptions.allowMemoryMapping())
|
||||
.setAllowMmapWrites(databaseOptions.allowMemoryMapping());
|
||||
}
|
||||
}
|
||||
|
||||
@ -341,8 +345,11 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
return options;
|
||||
}
|
||||
|
||||
private void createAllColumns(List<ColumnFamilyDescriptor> totalDescriptors, Options options, boolean inMemory, String dbPathString) throws RocksDBException {
|
||||
if (inMemory) {
|
||||
private void createAllColumns(List<ColumnFamilyDescriptor> totalDescriptors,
|
||||
Options options,
|
||||
DatabaseOptions databaseOptions,
|
||||
String dbPathString) throws RocksDBException {
|
||||
if (databaseOptions.inMemory()) {
|
||||
return;
|
||||
}
|
||||
List<byte[]> columnFamiliesToCreate = new LinkedList<>();
|
||||
@ -385,10 +392,10 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
}
|
||||
|
||||
private void createInMemoryColumns(List<ColumnFamilyDescriptor> totalDescriptors,
|
||||
boolean inMemory,
|
||||
DatabaseOptions databaseOptions,
|
||||
List<ColumnFamilyHandle> handles)
|
||||
throws RocksDBException {
|
||||
if (!inMemory) {
|
||||
if (!databaseOptions.inMemory()) {
|
||||
return;
|
||||
}
|
||||
List<byte[]> columnFamiliesToCreate = new LinkedList<>();
|
||||
@ -407,10 +414,10 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
|
||||
private void createIfNotExists(List<ColumnFamilyDescriptor> descriptors,
|
||||
Options options,
|
||||
boolean inMemory,
|
||||
DatabaseOptions databaseOptions,
|
||||
Path dbPath,
|
||||
String dbPathString) throws RocksDBException {
|
||||
if (inMemory) {
|
||||
if (databaseOptions.inMemory()) {
|
||||
return;
|
||||
}
|
||||
if (Files.notExists(dbPath)) {
|
||||
@ -436,9 +443,7 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
handles.add(db.createColumnFamily(columnFamilyDescriptor));
|
||||
}
|
||||
|
||||
if (!inMemory) {
|
||||
flushAndCloseDb(db, handles);
|
||||
}
|
||||
flushAndCloseDb(db, handles);
|
||||
}
|
||||
}
|
||||
|
||||
@ -481,6 +486,10 @@ public class LLLocalKeyValueDatabase implements LLKeyValueDatabase {
|
||||
return cfh;
|
||||
}
|
||||
|
||||
public DatabaseOptions getDatabaseOptions() {
|
||||
return databaseOptions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mono<Long> getProperty(String propertyName) {
|
||||
return Mono.fromCallable(() -> db.getAggregatedLongProperty(propertyName))
|
||||
|
@ -14,6 +14,7 @@ import it.cavallium.dbengine.database.collections.DatabaseStageEntry;
|
||||
import it.cavallium.dbengine.database.collections.DatabaseStageMap;
|
||||
import it.cavallium.dbengine.database.collections.SubStageGetterHashMap;
|
||||
import it.cavallium.dbengine.database.collections.SubStageGetterMap;
|
||||
import it.cavallium.dbengine.database.disk.DatabaseOptions;
|
||||
import it.cavallium.dbengine.database.disk.LLLocalDatabaseConnection;
|
||||
import it.cavallium.dbengine.database.serialization.Serializer;
|
||||
import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength;
|
||||
@ -54,10 +55,10 @@ public class DbTestUtils {
|
||||
return null;
|
||||
})
|
||||
.subscribeOn(Schedulers.boundedElastic())
|
||||
.then(new LLLocalDatabaseConnection(DbTestUtils.ALLOCATOR, wrkspcPath, true).connect())
|
||||
.then(new LLLocalDatabaseConnection(DbTestUtils.ALLOCATOR, wrkspcPath).connect())
|
||||
.flatMap(conn -> conn.getDatabase("testdb",
|
||||
List.of(Column.dictionary("testmap"), Column.special("ints"), Column.special("longs")),
|
||||
Map.of(), false, true
|
||||
new DatabaseOptions(Map.of(), true, false, true, false, true)
|
||||
)),
|
||||
action,
|
||||
db -> db.close().then(Mono.fromCallable(() -> {
|
||||
|
@ -13,6 +13,7 @@ import it.cavallium.dbengine.database.UpdateMode;
|
||||
import it.cavallium.dbengine.database.collections.DatabaseMapDictionary;
|
||||
import it.cavallium.dbengine.database.collections.DatabaseMapDictionaryDeep;
|
||||
import it.cavallium.dbengine.database.collections.SubStageGetterMap;
|
||||
import it.cavallium.dbengine.database.disk.DatabaseOptions;
|
||||
import it.cavallium.dbengine.database.disk.LLLocalDatabaseConnection;
|
||||
import it.cavallium.dbengine.database.serialization.Serializer;
|
||||
import it.cavallium.dbengine.database.serialization.SerializerFixedBinaryLength;
|
||||
@ -131,8 +132,11 @@ public class OldDatabaseTests {
|
||||
return null;
|
||||
})
|
||||
.subscribeOn(Schedulers.boundedElastic())
|
||||
.then(new LLLocalDatabaseConnection(PooledByteBufAllocator.DEFAULT, wrkspcPath, true).connect())
|
||||
.flatMap(conn -> conn.getDatabase("testdb", List.of(Column.dictionary("testmap")), Map.of(), false, true));
|
||||
.then(new LLLocalDatabaseConnection(PooledByteBufAllocator.DEFAULT, wrkspcPath).connect())
|
||||
.flatMap(conn -> conn.getDatabase("testdb",
|
||||
List.of(Column.dictionary("testmap")),
|
||||
new DatabaseOptions(Map.of(), true, false, true, false, true)
|
||||
));
|
||||
}
|
||||
|
||||
private static final ByteBuf DUMMY_VALUE;
|
||||
|
Loading…
x
Reference in New Issue
Block a user