diff --git a/.github/workflows/ci-bdnbenchmark.yml b/.github/workflows/ci-bdnbenchmark.yml index a0b6921562..0b9dc3c68a 100644 --- a/.github/workflows/ci-bdnbenchmark.yml +++ b/.github/workflows/ci-bdnbenchmark.yml @@ -42,7 +42,7 @@ jobs: os: [ ubuntu-latest, windows-latest ] framework: [ 'net8.0' ] configuration: [ 'Release' ] - test: [ 'Operations.BasicOperations', 'Operations.ObjectOperations', 'Operations.HashObjectOperations', 'Operations.SortedSetOperations', 'Cluster.ClusterMigrate', 'Cluster.ClusterOperations', 'Lua.LuaScripts', 'Lua.LuaScriptCacheOperations','Lua.LuaRunnerOperations','Operations.CustomOperations', 'Operations.RawStringOperations', 'Operations.ScriptOperations', 'Operations.ModuleOperations', 'Operations.PubSubOperations', 'Network.BasicOperations', 'Network.RawStringOperations' ] + test: [ 'Operations.BasicOperations', 'Operations.ObjectOperations', 'Operations.HashObjectOperations', 'Operations.SortedSetOperations', 'Cluster.ClusterMigrate', 'Cluster.ClusterOperations', 'Lua.LuaScripts', 'Lua.LuaScriptCacheOperations','Lua.LuaRunnerOperations','Operations.CustomOperations', 'Operations.RawStringOperations', 'Operations.ScriptOperations', 'Operations.ModuleOperations', 'Operations.PubSubOperations', 'Operations.ServerOperations', 'Network.BasicOperations', 'Network.RawStringOperations' ] steps: - name: Check out code uses: actions/checkout@v4 diff --git a/benchmark/BDN.benchmark/Operations/ServerOperations.cs b/benchmark/BDN.benchmark/Operations/ServerOperations.cs new file mode 100644 index 0000000000..bd78fd0575 --- /dev/null +++ b/benchmark/BDN.benchmark/Operations/ServerOperations.cs @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using BenchmarkDotNet.Attributes; +using Embedded.server; + +namespace BDN.benchmark.Operations +{ + /// + /// Benchmark for ServerOperations + /// + [MemoryDiagnoser] + public unsafe class ServerOperations : OperationsBase + { + static ReadOnlySpan SELECTUNSELECT => "*2\r\n$6\r\nSELECT\r\n$1\r\n1\r\n*2\r\n$6\r\nSELECT\r\n$1\r\n0\r\n"u8; + Request selectUnselect; + + static ReadOnlySpan SWAPDB => "*3\r\n$6\r\nSWAPDB\r\n$1\r\n1\r\n$1\r\n0\r\n"u8; + Request swapDb; + + public override void GlobalSetup() + { + base.GlobalSetup(); + + SetupOperation(ref selectUnselect, SELECTUNSELECT); + SetupOperation(ref swapDb, SWAPDB); + + // Pre-populate data in DB0 + SlowConsumeMessage("*3\r\n$3\r\nSET\r\n$1\r\na\r\n$1\r\na\r\n"u8); + SlowConsumeMessage("*3\r\n$5\r\nLPUSH\r\n$1\r\nd\r\n$1\r\nf\r\n"u8); + SlowConsumeMessage("*3\r\n$4\r\nSADD\r\n$1\r\ne\r\n$1\r\nb\r\n"u8); + + // Pre-populate data in DB1 + SlowConsumeMessage("*2\r\n$6\r\nSELECT\r\n$1\r\n1\r\n"u8); + SlowConsumeMessage("*3\r\n$3\r\nSET\r\n$1\r\nb\r\n$1\r\nb\r\n"u8); + SlowConsumeMessage("*3\r\n$5\r\nLPUSH\r\n$1\r\nf\r\n$1\r\nh\r\n"u8); + SlowConsumeMessage("*3\r\n$4\r\nSADD\r\n$1\r\ng\r\n$1\r\ni\r\n"u8); + } + + [Benchmark] + public void SelectUnselect() + { + Send(selectUnselect); + } + + [Benchmark] + public void SwapDb() + { + Send(swapDb); + } + } +} \ No newline at end of file diff --git a/libs/cluster/Server/Migration/MigrateSessionKeys.cs b/libs/cluster/Server/Migration/MigrateSessionKeys.cs index abc5028eba..f19f731fb9 100644 --- a/libs/cluster/Server/Migration/MigrateSessionKeys.cs +++ b/libs/cluster/Server/Migration/MigrateSessionKeys.cs @@ -182,14 +182,14 @@ public bool MigrateKeys() return false; // Migrate main store keys - _gcs.InitializeIterationBuffer(clusterProvider.storeWrapper.loggingFrequncy); + _gcs.InitializeIterationBuffer(clusterProvider.storeWrapper.loggingFrequency); if (!MigrateKeysFromMainStore()) return false; // Migrate object store keys if (!clusterProvider.serverOptions.DisableObjects) { - _gcs.InitializeIterationBuffer(clusterProvider.storeWrapper.loggingFrequncy); + _gcs.InitializeIterationBuffer(clusterProvider.storeWrapper.loggingFrequency); if (!MigrateKeysFromObjectStore()) return false; } diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 26f6664b4c..3e789d0fca 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -26,7 +26,7 @@ private void TrackImportProgress(int keyCount, bool isMainStore, bool completed { totalKeyCount += keyCount; var duration = TimeSpan.FromTicks(Stopwatch.GetTimestamp() - lastLog); - if (completed || lastLog == 0 || duration >= clusterProvider.storeWrapper.loggingFrequncy) + if (completed || lastLog == 0 || duration >= clusterProvider.storeWrapper.loggingFrequency) { logger?.LogTrace("[{op}]: isMainStore:({storeType}) totalKeyCount:({totalKeyCount})", completed ? "COMPLETED" : "IMPORTING", isMainStore, totalKeyCount.ToString("N0")); lastLog = Stopwatch.GetTimestamp(); diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs index ba85044b82..77bccfc945 100644 --- a/libs/host/Configuration/Options.cs +++ b/libs/host/Configuration/Options.cs @@ -569,6 +569,10 @@ internal sealed class Options [Option("unixsocketperm", Required = false, HelpText = "Unix socket permissions in octal (Unix platforms only)")] public int UnixSocketPermission { get; set; } + [IntRangeValidation(1, 256, isRequired: false)] + [Option("max-databases", Required = false, HelpText = "Max number of logical databases allowed in a single Garnet server instance")] + public int MaxDatabases { get; set; } + /// /// This property contains all arguments that were not parsed by the command line argument parser /// @@ -813,7 +817,8 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) SkipRDBRestoreChecksumValidation = SkipRDBRestoreChecksumValidation.GetValueOrDefault(), LuaOptions = EnableLua.GetValueOrDefault() ? new LuaOptions(LuaMemoryManagementMode, LuaScriptMemoryLimit, LuaScriptTimeoutMs == 0 ? Timeout.InfiniteTimeSpan : TimeSpan.FromMilliseconds(LuaScriptTimeoutMs), LuaLoggingMode, logger) : null, UnixSocketPath = UnixSocketPath, - UnixSocketPermission = unixSocketPermissions + UnixSocketPermission = unixSocketPermissions, + MaxDatabases = MaxDatabases, }; } diff --git a/libs/host/Configuration/Redis/RedisOptions.cs b/libs/host/Configuration/Redis/RedisOptions.cs index d3faa21e84..1658fb5339 100644 --- a/libs/host/Configuration/Redis/RedisOptions.cs +++ b/libs/host/Configuration/Redis/RedisOptions.cs @@ -97,6 +97,9 @@ internal class RedisOptions [RedisOption("slowlog-max-len", nameof(Options.SlowLogMaxEntries))] public Option SlowLogMaxLen { get; set; } + + [RedisOption("databases", nameof(Options.MaxDatabases))] + public Option Databases { get; set; } } /// diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index ef5fd1a5df..296ef10b2c 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -46,10 +46,6 @@ static string GetVersion() private readonly GarnetServerOptions opts; private IGarnetServer server; - private TsavoriteKV store; - private TsavoriteKV objectStore; - private IDevice aofDevice; - private TsavoriteLog appendOnlyFile; private SubscribeBroker subscribeBroker; private KVSettings kvSettings; private KVSettings objKvSettings; @@ -227,24 +223,14 @@ private void InitializeServer() if (!setMax && !ThreadPool.SetMaxThreads(maxThreads, maxCPThreads)) throw new Exception($"Unable to call ThreadPool.SetMaxThreads with {maxThreads}, {maxCPThreads}"); - CreateMainStore(clusterFactory, out var checkpointDir); - CreateObjectStore(clusterFactory, customCommandManager, checkpointDir, out var objectStoreSizeTracker); + StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate = (int dbId) => + CreateDatabase(dbId, opts, clusterFactory, customCommandManager); if (!opts.DisablePubSub) subscribeBroker = new SubscribeBroker(null, opts.PubSubPageSizeBytes(), opts.SubscriberRefreshFrequencyMs, true, logger); - CreateAOF(); - logger?.LogTrace("TLS is {tlsEnabled}", opts.TlsOptions == null ? "disabled" : "enabled"); - if (logger != null) - { - var configMemoryLimit = (store.IndexSize * 64) + store.Log.MaxMemorySizeBytes + (store.ReadCache?.MaxMemorySizeBytes ?? 0) + (appendOnlyFile?.MaxMemorySizeBytes ?? 0); - if (objectStore != null) - configMemoryLimit += objectStore.IndexSize * 64 + objectStore.Log.MaxMemorySizeBytes + (objectStore.ReadCache?.MaxMemorySizeBytes ?? 0) + (objectStoreSizeTracker?.TargetSize ?? 0) + (objectStoreSizeTracker?.ReadCacheTargetSize ?? 0); - logger.LogInformation("Total configured memory limit: {configMemoryLimit}", configMemoryLimit); - } - if (opts.EndPoint is UnixDomainSocketEndPoint) { // Delete existing unix socket file, if it exists. @@ -252,10 +238,29 @@ private void InitializeServer() } // Create Garnet TCP server if none was provided. - this.server ??= new GarnetServerTcp(opts.EndPoint, 0, opts.TlsOptions, opts.NetworkSendThrottleMax, opts.NetworkConnectionLimit, opts.UnixSocketPath, opts.UnixSocketPermission, logger); + this.server ??= new GarnetServerTcp(opts.EndPoint, 0, opts.TlsOptions, opts.NetworkSendThrottleMax, + opts.NetworkConnectionLimit, opts.UnixSocketPath, opts.UnixSocketPermission, logger); - storeWrapper = new StoreWrapper(version, RedisProtocolVersion, server, store, objectStore, objectStoreSizeTracker, - customCommandManager, appendOnlyFile, opts, subscribeBroker, clusterFactory: clusterFactory, loggerFactory: loggerFactory); + storeWrapper = new StoreWrapper(version, RedisProtocolVersion, server, + customCommandManager, opts, subscribeBroker, + createDatabaseDelegate: createDatabaseDelegate, + clusterFactory: clusterFactory, + loggerFactory: loggerFactory); + + if (logger != null) + { + var configMemoryLimit = (storeWrapper.store.IndexSize * 64) + + storeWrapper.store.Log.MaxMemorySizeBytes + + (storeWrapper.store.ReadCache?.MaxMemorySizeBytes ?? 0) + + (storeWrapper.appendOnlyFile?.MaxMemorySizeBytes ?? 0); + if (storeWrapper.objectStore != null) + configMemoryLimit += storeWrapper.objectStore.IndexSize * 64 + + storeWrapper.objectStore.Log.MaxMemorySizeBytes + + (storeWrapper.objectStore.ReadCache?.MaxMemorySizeBytes ?? 0) + + (storeWrapper.objectStoreSizeTracker?.TargetSize ?? 0) + + (storeWrapper.objectStoreSizeTracker?.ReadCacheTargetSize ?? 0); + logger.LogInformation("Total configured memory limit: {configMemoryLimit}", configMemoryLimit); + } // Create session provider for Garnet Provider = new GarnetProvider(storeWrapper, subscribeBroker); @@ -270,6 +275,17 @@ private void InitializeServer() LoadModules(customCommandManager); } + private GarnetDatabase CreateDatabase(int dbId, GarnetServerOptions serverOptions, ClusterFactory clusterFactory, + CustomCommandManager customCommandManager) + { + var store = CreateMainStore(dbId, clusterFactory); + var objectStore = CreateObjectStore(dbId, clusterFactory, customCommandManager, out var objectStoreSizeTracker); + var (aofDevice, aof) = CreateAOF(dbId); + return new GarnetDatabase(dbId, store, objectStore, objectStoreSizeTracker, aofDevice, aof, + serverOptions.AdjustedIndexMaxCacheLines == 0, + serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0); + } + private void LoadModules(CustomCommandManager customCommandManager) { if (opts.LoadModuleCS == null) @@ -294,82 +310,77 @@ private void LoadModules(CustomCommandManager customCommandManager) } } - private void CreateMainStore(IClusterFactory clusterFactory, out string checkpointDir) + private TsavoriteKV CreateMainStore(int dbId, IClusterFactory clusterFactory) { kvSettings = opts.GetSettings(loggerFactory, out logFactory); - checkpointDir = opts.CheckpointDir ?? opts.LogDir; - // Run checkpoint on its own thread to control p99 kvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; - if (opts.EnableCluster) - { - kvSettings.CheckpointManager = clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, - new DefaultCheckpointNamingScheme(checkpointDir + "/Store/checkpoints"), isMainStore: true, logger); - } - else - { - kvSettings.CheckpointManager = new DeviceLogCommitCheckpointManager(opts.DeviceFactoryCreator, - new DefaultCheckpointNamingScheme(checkpointDir + "/Store/checkpoints"), removeOutdated: true); - } + var baseName = opts.GetMainStoreCheckpointDirectory(dbId); + var defaultNamingScheme = new DefaultCheckpointNamingScheme(baseName); - store = new(kvSettings + kvSettings.CheckpointManager = opts.EnableCluster ? + clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: true, logger) : + new DeviceLogCommitCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true); + + return new TsavoriteKV(kvSettings , StoreFunctions.Create() , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); } - private void CreateObjectStore(IClusterFactory clusterFactory, CustomCommandManager customCommandManager, string CheckpointDir, out CacheSizeTracker objectStoreSizeTracker) + private TsavoriteKV CreateObjectStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager, out CacheSizeTracker objectStoreSizeTracker) { objectStoreSizeTracker = null; - if (!opts.DisableObjects) - { - objKvSettings = opts.GetObjectStoreSettings(this.loggerFactory?.CreateLogger("TsavoriteKV [obj]"), - out var objHeapMemorySize, out var objReadCacheHeapMemorySize); + if (opts.DisableObjects) + return null; - // Run checkpoint on its own thread to control p99 - objKvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; + objKvSettings = opts.GetObjectStoreSettings(this.loggerFactory?.CreateLogger("TsavoriteKV [obj]"), + out var objHeapMemorySize, out var objReadCacheHeapMemorySize); + + // Run checkpoint on its own thread to control p99 + objKvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; + + var baseName = opts.GetObjectStoreCheckpointDirectory(dbId); + var defaultNamingScheme = new DefaultCheckpointNamingScheme(baseName); + + objKvSettings.CheckpointManager = opts.EnableCluster ? + clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: false, logger) : + new DeviceLogCommitCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true); + + var objStore = new TsavoriteKV( + objKvSettings, + StoreFunctions.Create(new ByteArrayKeyComparer(), + () => new ByteArrayBinaryObjectSerializer(), + () => new GarnetObjectSerializer(customCommandManager)), + (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); + + if (objHeapMemorySize > 0 || objReadCacheHeapMemorySize > 0) + objectStoreSizeTracker = new CacheSizeTracker(objStore, objKvSettings, objHeapMemorySize, objReadCacheHeapMemorySize, + this.loggerFactory); + + return objStore; - if (opts.EnableCluster) - objKvSettings.CheckpointManager = clusterFactory.CreateCheckpointManager( - opts.DeviceFactoryCreator, - new DefaultCheckpointNamingScheme(CheckpointDir + "/ObjectStore/checkpoints"), - isMainStore: false, logger); - else - objKvSettings.CheckpointManager = new DeviceLogCommitCheckpointManager(opts.DeviceFactoryCreator, - new DefaultCheckpointNamingScheme(CheckpointDir + "/ObjectStore/checkpoints"), - removeOutdated: true); - - objectStore = new(objKvSettings - , StoreFunctions.Create(new ByteArrayKeyComparer(), - () => new ByteArrayBinaryObjectSerializer(), - () => new GarnetObjectSerializer(customCommandManager)) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); - - if (objHeapMemorySize > 0 || objReadCacheHeapMemorySize > 0) - objectStoreSizeTracker = new CacheSizeTracker(objectStore, objKvSettings, objHeapMemorySize, objReadCacheHeapMemorySize, - this.loggerFactory); - } } - private void CreateAOF() + private (IDevice, TsavoriteLog) CreateAOF(int dbId) { if (opts.EnableAOF) { if (opts.FastAofTruncate && opts.CommitFrequencyMs != -1) throw new Exception("Need to set CommitFrequencyMs to -1 (manual commits) with MainMemoryReplication"); - opts.GetAofSettings(out var aofSettings); - aofDevice = aofSettings.LogDevice; - appendOnlyFile = new TsavoriteLog(aofSettings, logger: this.loggerFactory?.CreateLogger("TsavoriteLog [aof]")); - + opts.GetAofSettings(dbId, out var aofSettings); + var aofDevice = aofSettings.LogDevice; + var appendOnlyFile = new TsavoriteLog(aofSettings, logger: this.loggerFactory?.CreateLogger("TsavoriteLog [aof]")); if (opts.CommitFrequencyMs < 0 && opts.WaitForCommit) throw new Exception("Cannot use CommitWait with manual commits"); - return; + return (aofDevice, appendOnlyFile); } if (opts.CommitFrequencyMs != 0 || opts.WaitForCommit) throw new Exception("Cannot use CommitFrequencyMs or CommitWait without EnableAOF"); + return (null, null); } /// @@ -415,13 +426,9 @@ private void InternalDispose() Provider?.Dispose(); server.Dispose(); subscribeBroker?.Dispose(); - store.Dispose(); - appendOnlyFile?.Dispose(); - aofDevice?.Dispose(); kvSettings.LogDevice?.Dispose(); if (!opts.DisableObjects) { - objectStore.Dispose(); objKvSettings.LogDevice?.Dispose(); objKvSettings.ObjectLogDevice?.Dispose(); } diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf index 4741f65e63..2f1ea1cc38 100644 --- a/libs/host/defaults.conf +++ b/libs/host/defaults.conf @@ -384,5 +384,8 @@ "UnixSocketPath": null, /* Unix socket permissions in octal (Unix platforms only) */ - "UnixSocketPermission": 0 + "UnixSocketPermission": 0, + + /* Max number of logical databases allowed in a single Garnet server instance */ + "MaxDatabases": 16 } \ No newline at end of file diff --git a/libs/resources/RespCommandsDocs.json b/libs/resources/RespCommandsDocs.json index f42b5bcccc..0c0b281d38 100644 --- a/libs/resources/RespCommandsDocs.json +++ b/libs/resources/RespCommandsDocs.json @@ -214,6 +214,14 @@ "Type": "PureToken", "Token": "SCHEDULE", "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" } ] }, @@ -1707,7 +1715,17 @@ "Command": "COMMITAOF", "Name": "COMMITAOF", "Summary": "Commit to append-only file.", - "Group": "Server" + "Group": "Server", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] }, { "Command": "CONFIG", @@ -4175,7 +4193,17 @@ "Name": "LASTSAVE", "Summary": "Returns the Unix timestamp of the last successful save to disk.", "Group": "Server", - "Complexity": "O(1)" + "Complexity": "O(1)", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] }, { "Command": "LATENCY", @@ -5852,7 +5880,17 @@ "Name": "SAVE", "Summary": "Synchronously saves the database(s) to disk.", "Group": "Server", - "Complexity": "O(N) where N is the total number of keys in all databases" + "Complexity": "O(N) where N is the total number of keys in all databases", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] }, { "Command": "SCAN", @@ -6932,6 +6970,25 @@ } ] }, + { + "Command": "SWAPDB", + "Name": "SWAPDB", + "Summary": "Swaps two Memurai databases", + "Group": "Server", + "Complexity": "O(N) where N is the count of clients watching or blocking on keys from both databases.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "INDEX1", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "INDEX2", + "Type": "Integer" + } + ] + }, { "Command": "TIME", "Name": "TIME", diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index dd1499f05d..4d5497726f 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -2563,7 +2563,7 @@ { "Command": "LASTSAVE", "Name": "LASTSAVE", - "Arity": 1, + "Arity": -1, "Flags": "Fast, Loading, Stale", "AclCategories": "Admin, Dangerous, Fast", "Tips": [ @@ -3765,7 +3765,7 @@ { "Command": "SAVE", "Name": "SAVE", - "Arity": 1, + "Arity": -1, "Flags": "Admin, NoAsyncLoading, NoMulti, NoScript", "AclCategories": "Admin, Dangerous, Slow" }, @@ -4643,6 +4643,13 @@ } ] }, + { + "Command": "SWAPDB", + "Name": "SWAPDB", + "Arity": 3, + "Flags": "Fast, Write", + "AclCategories": "Dangerous, Fast, KeySpace, Write" + }, { "Command": "TIME", "Name": "TIME", diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index e3f8ee95c4..ed86663297 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -33,20 +33,17 @@ public sealed unsafe partial class AofProcessor private readonly CustomProcedureInput customProcInput; private readonly SessionParseState parseState; - /// - /// Replication offset - /// - internal long ReplicationOffset { get; private set; } + int activeDbId; /// /// Session for main store /// - readonly BasicContext basicContext; + BasicContext basicContext; /// /// Session for object store /// - readonly BasicContext objectStoreBasicContext; + BasicContext objectStoreBasicContext; readonly Dictionary> inflightTxns; readonly byte[] buffer; @@ -54,7 +51,6 @@ public sealed unsafe partial class AofProcessor readonly byte* bufferPtr; readonly ILogger logger; - readonly bool recordToAof; /// /// Create new AOF processor @@ -65,31 +61,12 @@ public AofProcessor( ILogger logger = null) { this.storeWrapper = storeWrapper; - this.recordToAof = recordToAof; - - ReplicationOffset = 0; - - var replayAofStoreWrapper = new StoreWrapper( - storeWrapper.version, - storeWrapper.redisProtocolVersion, - null, - storeWrapper.store, - storeWrapper.objectStore, - storeWrapper.objectStoreSizeTracker, - storeWrapper.customCommandManager, - recordToAof ? storeWrapper.appendOnlyFile : null, - storeWrapper.serverOptions, - storeWrapper.subscribeBroker, - accessControlList: storeWrapper.accessControlList, - loggerFactory: storeWrapper.loggerFactory); - this.respServerSession = new RespServerSession(0, networkSender: null, storeWrapper: replayAofStoreWrapper, subscribeBroker: null, authenticator: null, enableScripts: false); + var replayAofStoreWrapper = new StoreWrapper(storeWrapper, recordToAof); - var session = respServerSession.storageSession.basicContext.Session; - basicContext = session.BasicContext; - var objectStoreSession = respServerSession.storageSession.objectStoreBasicContext.Session; - if (objectStoreSession is not null) - objectStoreBasicContext = objectStoreSession.BasicContext; + this.activeDbId = 0; + this.respServerSession = new RespServerSession(0, networkSender: null, storeWrapper: replayAofStoreWrapper, subscribeBroker: null, authenticator: null, enableScripts: false); + SwitchActiveDatabaseContext(ref storeWrapper.DefaultDatabase, true); parseState.Initialize(); storeInput.parseState = parseState; @@ -108,29 +85,41 @@ public AofProcessor( /// public void Dispose() { - basicContext.Session?.Dispose(); - objectStoreBasicContext.Session?.Dispose(); + var databaseSessionsSnapshot = respServerSession.databaseSessions.Map; + foreach (var dbSession in databaseSessionsSnapshot) + { + dbSession.StorageSession.basicContext.Session?.Dispose(); + dbSession.StorageSession.objectStoreBasicContext.Session?.Dispose(); + } + handle.Free(); } /// /// Recover store using AOF /// - public unsafe void Recover(long untilAddress = -1) + /// Database to recover + /// + /// Replication offset + public unsafe long Recover(ref GarnetDatabase db, long untilAddress = -1) { logger?.LogInformation("Begin AOF recovery"); - RecoverReplay(untilAddress); + return RecoverReplay(ref db, untilAddress); } MemoryResult output = default; - private unsafe void RecoverReplay(long untilAddress) + private unsafe long RecoverReplay(ref GarnetDatabase db, long untilAddress) { logger?.LogInformation("Begin AOF replay"); try { int count = 0; - if (untilAddress == -1) untilAddress = storeWrapper.appendOnlyFile.TailAddress; - using var scan = storeWrapper.appendOnlyFile.Scan(storeWrapper.appendOnlyFile.BeginAddress, untilAddress); + var appendOnlyFile = db.AppendOnlyFile; + + SwitchActiveDatabaseContext(ref db); + + if (untilAddress == -1) untilAddress = appendOnlyFile.TailAddress; + using var scan = appendOnlyFile.Scan(appendOnlyFile.BeginAddress, untilAddress); while (scan.GetNext(MemoryPool.Shared, out var entry, out var length, out _, out long nextAofAddress)) { @@ -140,10 +129,8 @@ private unsafe void RecoverReplay(long untilAddress) logger?.LogInformation("Completed AOF replay of {count} records, until AOF address {nextAofAddress}", count, nextAofAddress); } - // Update ReplicationOffset - ReplicationOffset = untilAddress; - logger?.LogInformation("Completed full AOF log replay of {count} records", count); + return untilAddress; } catch (Exception ex) { @@ -157,6 +144,8 @@ private unsafe void RecoverReplay(long untilAddress) output.MemoryOwner?.Dispose(); respServerSession.Dispose(); } + + return -1; } internal unsafe void ProcessAofRecord(IMemoryOwner entry, int length, bool asReplica = false) @@ -291,6 +280,25 @@ private unsafe bool ReplayOp(byte* entryPtr) return true; } + private void SwitchActiveDatabaseContext(ref GarnetDatabase db, bool initialSetup = false) + { + if (respServerSession.activeDbId != db.Id) + { + var switchDbSuccessful = respServerSession.TrySwitchActiveDatabaseSession(db.Id); + Debug.Assert(switchDbSuccessful); + } + + if (this.activeDbId != db.Id || initialSetup) + { + var session = respServerSession.storageSession.basicContext.Session; + basicContext = session.BasicContext; + var objectStoreSession = respServerSession.storageSession.objectStoreBasicContext.Session; + if (objectStoreSession is not null) + objectStoreBasicContext = objectStoreSession.BasicContext; + this.activeDbId = db.Id; + } + } + unsafe void RunStoredProc(byte id, CustomProcedureInput customProcInput, byte* ptr) { var curr = ptr + sizeof(AofHeader); diff --git a/libs/server/Custom/ExpandableMap.cs b/libs/server/Custom/ExpandableMap.cs index 3960f4fb01..f77cf75e5f 100644 --- a/libs/server/Custom/ExpandableMap.cs +++ b/libs/server/Custom/ExpandableMap.cs @@ -195,7 +195,7 @@ private bool TryUpdateActualSize(int id) /// Item value /// True if should not attempt to expand the underlying array /// True if assignment succeeded - private bool TrySetValueUnsafe(int id, ref T value, bool noExpansion) + internal bool TrySetValueUnsafe(int id, ref T value, bool noExpansion) { var idx = id - minId; if (idx < 0 || idx >= maxSize) return false; diff --git a/libs/server/Custom/IDefaultChecker.cs b/libs/server/Custom/IDefaultChecker.cs new file mode 100644 index 0000000000..cd049ef9ac --- /dev/null +++ b/libs/server/Custom/IDefaultChecker.cs @@ -0,0 +1,10 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Garnet.server +{ + public interface IDefaultChecker + { + public bool IsDefault(); + } +} \ No newline at end of file diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs new file mode 100644 index 0000000000..faea42cc9b --- /dev/null +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -0,0 +1,585 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.server +{ + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; + + using ObjectStoreAllocator = GenericAllocator>>; + using ObjectStoreFunctions = StoreFunctions>; + + internal abstract class DatabaseManagerBase : IDatabaseManager + { + /// + public abstract ref GarnetDatabase DefaultDatabase { get; } + + /// + public TsavoriteKV MainStore => DefaultDatabase.MainStore; + + /// + public TsavoriteKV ObjectStore => DefaultDatabase.ObjectStore; + + /// + public TsavoriteLog AppendOnlyFile => DefaultDatabase.AppendOnlyFile; + + /// + public DateTimeOffset LastSaveTime => DefaultDatabase.LastSaveTime; + + /// + public CacheSizeTracker ObjectStoreSizeTracker => DefaultDatabase.ObjectStoreSizeTracker; + + /// + public WatchVersionMap VersionMap => DefaultDatabase.VersionMap; + + /// + public abstract int DatabaseCount { get; } + + /// + /// Store Wrapper + /// + public readonly StoreWrapper StoreWrapper; + + /// + public abstract ref GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, out bool added); + + /// + public abstract bool TryPauseCheckpoints(int dbId); + + /// + /// Continuously try to take a lock for checkpointing until acquired or token was cancelled + /// + /// ID of database to lock + /// Cancellation token + /// True if lock acquired + public abstract Task TryPauseCheckpointsContinuousAsync(int dbId, CancellationToken token = default); + + /// + public abstract void ResumeCheckpoints(int dbId); + + /// + public abstract void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, + bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null); + + /// + public abstract bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default); + + /// + public abstract bool TakeCheckpoint(bool background, int dbId, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default); + + /// + public abstract Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, int dbId = 0); + + /// + public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, + CancellationToken token = default, ILogger logger = null); + + /// + public abstract Task CommitToAofAsync(CancellationToken token = default, ILogger logger = null); + + /// + public abstract Task CommitToAofAsync(int dbId, CancellationToken token = default, ILogger logger = null); + + /// + public abstract Task WaitForCommitToAofAsync(CancellationToken token = default, ILogger logger = null); + + /// + public abstract void RecoverAOF(); + + /// + public abstract long ReplayAOF(long untilAddress = -1); + + /// + public abstract void DoCompaction(CancellationToken token = default, ILogger logger = null); + + /// + public abstract bool GrowIndexesIfNeeded(CancellationToken token = default); + + /// + public abstract void StartObjectSizeTrackers(CancellationToken token = default); + + /// + public abstract void Reset(int dbId = 0); + + /// + public abstract void ResetRevivificationStats(); + + /// + public abstract void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0); + + /// + public abstract GarnetDatabase[] GetDatabasesSnapshot(); + + /// + public abstract ref GarnetDatabase TryGetDatabase(int dbId, out bool found); + + /// + public abstract void FlushDatabase(bool unsafeTruncateLog, int dbId = 0); + + /// + public abstract void FlushAllDatabases(bool unsafeTruncateLog); + + /// + public abstract bool TrySwapDatabases(int dbId1, int dbId2); + + public abstract FunctionsState CreateFunctionsState(int dbId = 0); + + /// + public abstract void Dispose(); + + /// + public abstract IDatabaseManager Clone(bool enableAof); + + /// + /// Delegate for creating a new logical database + /// + public readonly StoreWrapper.DatabaseCreatorDelegate CreateDatabaseDelegate; + + /// + /// The main logger instance associated with the database manager. + /// + protected ILogger Logger; + + /// + /// The logger factory used to create logger instances + /// + protected ILoggerFactory LoggerFactory; + + protected bool Disposed; + + protected DatabaseManagerBase(StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate, StoreWrapper storeWrapper) + { + this.CreateDatabaseDelegate = createDatabaseDelegate; + this.StoreWrapper = storeWrapper; + this.LoggerFactory = storeWrapper.loggerFactory; + } + + protected abstract ref GarnetDatabase GetDatabaseByRef(int dbId = 0); + + protected void RecoverDatabaseCheckpoint(ref GarnetDatabase db, out long storeVersion, out long objectStoreVersion) + { + storeVersion = db.MainStore.Recover(); + objectStoreVersion = -1; + + if (db.ObjectStore != null) + objectStoreVersion = db.ObjectStore.Recover(); + + if (storeVersion > 0 || objectStoreVersion > 0) + { + db.LastSaveTime = DateTimeOffset.UtcNow; + } + } + + protected async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, CheckpointType checkpointType, + bool tryIncremental, + StoreType storeType, ILogger logger = null) + { + logger?.LogInformation("Initiating checkpoint; full = {full}, type = {checkpointType}, tryIncremental = {tryIncremental}, storeType = {storeType}, dbId = {dbId}", full, checkpointType, tryIncremental, storeType, db.Id); + + long checkpointCoveredAofAddress = 0; + if (db.AppendOnlyFile != null) + { + if (StoreWrapper.serverOptions.EnableCluster) + StoreWrapper.clusterProvider.OnCheckpointInitiated(out checkpointCoveredAofAddress); + else + checkpointCoveredAofAddress = db.AppendOnlyFile.TailAddress; + + if (checkpointCoveredAofAddress > 0) + logger?.LogInformation("Will truncate AOF to {tailAddress} after checkpoint (files deleted after next commit)", checkpointCoveredAofAddress); + } + + (bool success, Guid token) storeCheckpointResult = default; + (bool success, Guid token) objectStoreCheckpointResult = default; + if (full) + { + if (storeType is StoreType.Main or StoreType.All) + { + storeCheckpointResult = await db.MainStore.TakeFullCheckpointAsync(checkpointType); + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.clusterProvider.IsPrimary()) + EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, db.MainStore.CurrentVersion); + } + + if (db.ObjectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) + { + objectStoreCheckpointResult = await db.ObjectStore.TakeFullCheckpointAsync(checkpointType); + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.clusterProvider.IsPrimary()) + EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, db.ObjectStore.CurrentVersion); + } + } + else + { + if (storeType is StoreType.Main or StoreType.All) + { + storeCheckpointResult = await db.MainStore.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.clusterProvider.IsPrimary()) + EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, db.MainStore.CurrentVersion); + } + + if (db.ObjectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) + { + objectStoreCheckpointResult = await db.ObjectStore.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.clusterProvider.IsPrimary()) + EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, db.ObjectStore.CurrentVersion); + } + } + + // If cluster is enabled the replication manager is responsible for truncating AOF + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.serverOptions.EnableAOF) + { + StoreWrapper.clusterProvider.SafeTruncateAOF(storeType, full, checkpointCoveredAofAddress, + storeCheckpointResult.token, objectStoreCheckpointResult.token); + } + else + { + db.AppendOnlyFile?.TruncateUntil(checkpointCoveredAofAddress); + db.AppendOnlyFile?.Commit(); + } + + if (db.ObjectStore != null) + { + // During the checkpoint, we may have serialized Garnet objects in (v) versions of objects. + // We can now safely remove these serialized versions as they are no longer needed. + using var iter1 = db.ObjectStore.Log.Scan(db.ObjectStore.Log.ReadOnlyAddress, + db.ObjectStore.Log.TailAddress, ScanBufferingMode.SinglePageBuffering, includeSealedRecords: true); + while (iter1.GetNext(out _, out _, out var value)) + { + if (value != null) + ((GarnetObjectBase)value).serialized = null; + } + } + + logger?.LogInformation("Completed checkpoint"); + } + + /// + /// Asynchronously checkpoint a single database + /// + /// Store type to checkpoint + /// Database to checkpoint + /// Logger + /// Cancellation token + /// Task + protected async Task TakeCheckpointAsync(GarnetDatabase db, StoreType storeType, ILogger logger = null, CancellationToken token = default) + { + try + { + DoCompaction(ref db); + var lastSaveStoreTailAddress = db.MainStore.Log.TailAddress; + var lastSaveObjectStoreTailAddress = (db.ObjectStore?.Log.TailAddress).GetValueOrDefault(); + + var full = db.LastSaveStoreTailAddress == 0 || + lastSaveStoreTailAddress - db.LastSaveStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval || + (db.ObjectStore != null && (db.LastSaveObjectStoreTailAddress == 0 || + lastSaveObjectStoreTailAddress - db.LastSaveObjectStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval)); + + var tryIncremental = StoreWrapper.serverOptions.EnableIncrementalSnapshots; + if (db.MainStore.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit) + tryIncremental = false; + if (db.ObjectStore?.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit) + tryIncremental = false; + + var checkpointType = StoreWrapper.serverOptions.UseFoldOverCheckpoints ? CheckpointType.FoldOver : CheckpointType.Snapshot; + await InitiateCheckpointAsync(db, full, checkpointType, tryIncremental, storeType, logger); + + if (full) + { + if (storeType is StoreType.Main or StoreType.All) + db.LastSaveStoreTailAddress = lastSaveStoreTailAddress; + if (storeType is StoreType.Object or StoreType.All) + db.LastSaveObjectStoreTailAddress = lastSaveObjectStoreTailAddress; + } + } + catch (Exception ex) + { + logger?.LogError(ex, "Checkpointing threw exception"); + } + } + + protected bool TryPauseCheckpoints(ref GarnetDatabase db) + => db.CheckpointingLock.TryWriteLock(); + + protected void ResumeCheckpoints(ref GarnetDatabase db) + => db.CheckpointingLock.WriteUnlock(); + + protected void RecoverDatabaseAOF(ref GarnetDatabase db) + { + if (db.AppendOnlyFile == null) return; + + db.AppendOnlyFile.Recover(); + Logger?.LogInformation("Recovered AOF: begin address = {beginAddress}, tail address = {tailAddress}", + db.AppendOnlyFile.BeginAddress, db.AppendOnlyFile.TailAddress); + } + + protected long ReplayDatabaseAOF(AofProcessor aofProcessor, ref GarnetDatabase db, long untilAddress = -1) + { + long replicationOffset = 0; + try + { + replicationOffset = aofProcessor.Recover(ref db, untilAddress); + db.LastSaveTime = DateTimeOffset.UtcNow; + } + catch (Exception ex) + { + Logger?.LogError(ex, "Error during recovery of AofProcessor"); + if (StoreWrapper.serverOptions.FailOnRecoveryError) + throw; + } + + return replicationOffset; + } + + protected void ResetDatabase(ref GarnetDatabase db) + { + try + { + if (db.MainStore.Log.TailAddress > 64) + db.MainStore.Reset(); + if (db.ObjectStore?.Log.TailAddress > 64) + db.ObjectStore?.Reset(); + db.AppendOnlyFile?.Reset(); + + var lastSave = DateTimeOffset.FromUnixTimeSeconds(0); + db.LastSaveTime = lastSave; + } + catch (Exception ex) + { + Logger?.LogError(ex, "Error during reset of store"); + } + } + + protected void EnqueueDatabaseCommit(ref GarnetDatabase db, AofEntryType entryType, long version) + { + if (db.AppendOnlyFile == null) return; + + AofHeader header = new() + { + opType = entryType, + storeVersion = version, + sessionID = -1 + }; + + db.AppendOnlyFile.Enqueue(header, out _); + } + + protected void FlushDatabase(ref GarnetDatabase db, bool unsafeTruncateLog) + { + db.MainStore.Log.ShiftBeginAddress(db.MainStore.Log.TailAddress, truncateLog: unsafeTruncateLog); + db.ObjectStore?.Log.ShiftBeginAddress(db.ObjectStore.Log.TailAddress, truncateLog: unsafeTruncateLog); + } + + protected void DoCompaction(ref GarnetDatabase db, ILogger logger = null) + { + try + { + // Periodic compaction -> no need to compact before checkpointing + if (StoreWrapper.serverOptions.CompactionFrequencySecs > 0) return; + + DoCompaction(ref db, StoreWrapper.serverOptions.CompactionMaxSegments, + StoreWrapper.serverOptions.ObjectStoreCompactionMaxSegments, 1, + StoreWrapper.serverOptions.CompactionType, StoreWrapper.serverOptions.CompactionForceDelete); + } + catch (Exception ex) + { + logger?.LogError(ex, + "Exception raised during compaction. AOF tail address = {tailAddress}; AOF committed until address = {commitAddress}; ", + db.AppendOnlyFile.TailAddress, db.AppendOnlyFile.CommittedUntilAddress); + throw; + } + } + + private void DoCompaction(ref GarnetDatabase db, int mainStoreMaxSegments, int objectStoreMaxSegments, int numSegmentsToCompact, LogCompactionType compactionType, bool compactionForceDelete) + { + if (compactionType == LogCompactionType.None) return; + + var mainStoreLog = db.MainStore.Log; + + var mainStoreMaxLogSize = (1L << StoreWrapper.serverOptions.SegmentSizeBits()) * mainStoreMaxSegments; + + if (mainStoreLog.ReadOnlyAddress - mainStoreLog.BeginAddress > mainStoreMaxLogSize) + { + var readOnlyAddress = mainStoreLog.ReadOnlyAddress; + var compactLength = (1L << StoreWrapper.serverOptions.SegmentSizeBits()) * (mainStoreMaxSegments - numSegmentsToCompact); + var untilAddress = readOnlyAddress - compactLength; + Logger?.LogInformation( + "Begin main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", + untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress); + + switch (compactionType) + { + case LogCompactionType.Shift: + mainStoreLog.ShiftBeginAddress(untilAddress, true, compactionForceDelete); + break; + + case LogCompactionType.Scan: + mainStoreLog.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Scan); + if (compactionForceDelete) + { + CompactionCommitAof(ref db); + mainStoreLog.Truncate(); + } + break; + + case LogCompactionType.Lookup: + mainStoreLog.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Lookup); + if (compactionForceDelete) + { + CompactionCommitAof(ref db); + mainStoreLog.Truncate(); + } + break; + } + + Logger?.LogInformation( + "End main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", + untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress); + } + + if (db.ObjectStore == null) return; + + var objectStoreLog = db.ObjectStore.Log; + + var objectStoreMaxLogSize = (1L << StoreWrapper.serverOptions.ObjectStoreSegmentSizeBits()) * objectStoreMaxSegments; + + if (objectStoreLog.ReadOnlyAddress - objectStoreLog.BeginAddress > objectStoreMaxLogSize) + { + var readOnlyAddress = objectStoreLog.ReadOnlyAddress; + var compactLength = (1L << StoreWrapper.serverOptions.ObjectStoreSegmentSizeBits()) * (objectStoreMaxSegments - numSegmentsToCompact); + var untilAddress = readOnlyAddress - compactLength; + Logger?.LogInformation( + "Begin object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", + untilAddress, objectStoreLog.BeginAddress, readOnlyAddress, objectStoreLog.TailAddress); + + switch (compactionType) + { + case LogCompactionType.Shift: + objectStoreLog.ShiftBeginAddress(untilAddress, compactionForceDelete); + break; + + case LogCompactionType.Scan: + objectStoreLog.Compact>( + new SimpleSessionFunctions(), untilAddress, CompactionType.Scan); + if (compactionForceDelete) + { + CompactionCommitAof(ref db); + objectStoreLog.Truncate(); + } + break; + + case LogCompactionType.Lookup: + objectStoreLog.Compact>( + new SimpleSessionFunctions(), untilAddress, CompactionType.Lookup); + if (compactionForceDelete) + { + CompactionCommitAof(ref db); + objectStoreLog.Truncate(); + } + break; + } + + Logger?.LogInformation( + "End object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", + untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress); + } + } + + private void CompactionCommitAof(ref GarnetDatabase db) + { + // If we are the primary, we commit the AOF. + // If we are the replica, we commit the AOF only if fast commit is disabled + // because we do not want to clobber AOF addresses. + // TODO: replica should instead wait until the next AOF commit is done via primary + if (StoreWrapper.serverOptions.EnableAOF) + { + if (StoreWrapper.serverOptions.EnableCluster && StoreWrapper.clusterProvider.IsReplica()) + { + if (!StoreWrapper.serverOptions.EnableFastCommit) + db.AppendOnlyFile?.CommitAsync().ConfigureAwait(false).GetAwaiter().GetResult(); + } + else + { + db.AppendOnlyFile?.CommitAsync().ConfigureAwait(false).GetAwaiter().GetResult(); + } + } + } + + protected bool GrowIndexesIfNeeded(ref GarnetDatabase db) + { + var indexesMaxedOut = true; + + if (!DefaultDatabase.MainStoreIndexMaxedOut) + { + var dbMainStore = DefaultDatabase.MainStore; + if (GrowIndexIfNeeded(StoreType.Main, + StoreWrapper.serverOptions.AdjustedIndexMaxCacheLines, dbMainStore.OverflowBucketAllocations, + () => dbMainStore.IndexSize, async () => await dbMainStore.GrowIndexAsync())) + { + db.MainStoreIndexMaxedOut = true; + } + else + { + indexesMaxedOut = false; + } + } + + if (!db.ObjectStoreIndexMaxedOut) + { + var dbObjectStore = db.ObjectStore; + if (GrowIndexIfNeeded(StoreType.Object, + StoreWrapper.serverOptions.AdjustedObjectStoreIndexMaxCacheLines, + dbObjectStore.OverflowBucketAllocations, + () => dbObjectStore.IndexSize, async () => await dbObjectStore.GrowIndexAsync())) + { + db.ObjectStoreIndexMaxedOut = true; + } + else + { + indexesMaxedOut = false; + } + } + + return indexesMaxedOut; + } + + /// + /// Grows index if current size is smaller than max size. + /// Decision is based on whether overflow bucket allocation is more than a threshold which indicates a contention + /// in the index leading many allocations to the same bucket. + /// + /// + /// + /// + /// + /// + /// True if index has reached its max size + protected bool GrowIndexIfNeeded(StoreType storeType, long indexMaxSize, long overflowCount, Func indexSizeRetriever, Action growAction) + { + Logger?.LogDebug( + $"IndexAutoGrowTask[{{storeType}}]: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}", + storeType, indexSizeRetriever(), indexMaxSize, overflowCount); + + if (indexSizeRetriever() < indexMaxSize && + overflowCount > (indexSizeRetriever() * StoreWrapper.serverOptions.IndexResizeThreshold / 100)) + { + Logger?.LogInformation( + $"IndexAutoGrowTask[{{storeType}}]: overflowCount {{overflowCount}} ratio more than threshold {{indexResizeThreshold}}%. Doubling index size...", + storeType, overflowCount, StoreWrapper.serverOptions.IndexResizeThreshold); + growAction(); + } + + if (indexSizeRetriever() < indexMaxSize) return false; + + Logger?.LogDebug( + $"IndexAutoGrowTask[{{storeType}}]: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}", + storeType, indexSizeRetriever(), indexMaxSize, overflowCount); + return true; + } + } +} \ No newline at end of file diff --git a/libs/server/Databases/DatabaseManagerFactory.cs b/libs/server/Databases/DatabaseManagerFactory.cs new file mode 100644 index 0000000000..a1a9bae89a --- /dev/null +++ b/libs/server/Databases/DatabaseManagerFactory.cs @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Linq; + +namespace Garnet.server +{ + /// + /// Factory class for creating new instances of IDatabaseManager + /// + public class DatabaseManagerFactory + { + /// + /// Create a new instance of IDatabaseManager + /// + /// Garnet server options + /// Delegate for creating a new logical database + /// Store wrapper instance + /// True if database manager should create a default database instance (default: true) + /// + public static IDatabaseManager CreateDatabaseManager(GarnetServerOptions serverOptions, + StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate, StoreWrapper storeWrapper, bool createDefaultDatabase = true) + { + return ShouldCreateMultipleDatabaseManager(serverOptions, createDatabaseDelegate) ? + new MultiDatabaseManager(createDatabaseDelegate, storeWrapper, createDefaultDatabase) : + new SingleDatabaseManager(createDatabaseDelegate, storeWrapper, createDefaultDatabase); + } + + private static bool ShouldCreateMultipleDatabaseManager(GarnetServerOptions serverOptions, + StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate) + { + // If multiple databases are not allowed or recovery is disabled, create a single database manager + if (!serverOptions.AllowMultiDb || !serverOptions.Recover) + return false; + + // If there are multiple databases to recover, create a multi database manager, otherwise create a single database manager. + using (createDatabaseDelegate(0)) + { + // Check if there are multiple databases to recover from checkpoint + var checkpointParentDir = serverOptions.MainStoreCheckpointBaseDirectory; + var checkpointDirBaseName = serverOptions.GetCheckpointDirectoryName(0); + + if (MultiDatabaseManager.TryGetSavedDatabaseIds(checkpointParentDir, checkpointDirBaseName, + out var dbIds) && dbIds.Any(id => id != 0)) + return true; + + // Check if there are multiple databases to recover from AOF + if (serverOptions.EnableAOF) + { + var aofParentDir = serverOptions.AppendOnlyFileBaseDirectory; + var aofDirBaseName = serverOptions.GetAppendOnlyFileDirectoryName(0); + + if (MultiDatabaseManager.TryGetSavedDatabaseIds(aofParentDir, aofDirBaseName, + out dbIds) && dbIds.Any(id => id != 0)) + return true; + } + + return false; + } + } + } +} \ No newline at end of file diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs new file mode 100644 index 0000000000..ec30bf48f8 --- /dev/null +++ b/libs/server/Databases/IDatabaseManager.cs @@ -0,0 +1,238 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.server +{ + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; + + using ObjectStoreAllocator = GenericAllocator>>; + using ObjectStoreFunctions = StoreFunctions>; + + public interface IDatabaseManager : IDisposable + { + /// + /// Reference to default database (DB 0) + /// + public ref GarnetDatabase DefaultDatabase { get; } + + /// + /// Store (of DB 0) + /// + public TsavoriteKV MainStore { get; } + + /// + /// Object store (of DB 0) + /// + public TsavoriteKV ObjectStore { get; } + + /// + /// AOF (of DB 0) + /// + public TsavoriteLog AppendOnlyFile { get; } + + /// + /// Last save time (of DB 0) + /// + public DateTimeOffset LastSaveTime { get; } + + /// + /// Object store size tracker (of DB 0) + /// + public CacheSizeTracker ObjectStoreSizeTracker { get; } + + /// + /// Version map (of DB 0) + /// + internal WatchVersionMap VersionMap { get; } + + /// + /// Number of current logical databases + /// + public int DatabaseCount { get; } + + /// + /// Try to get or add a new database + /// + /// Database ID + /// Database was found or added successfully + /// True if database was added + /// Reference to retrieved or added database + public ref GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, out bool added); + + /// + /// Mark the beginning of a checkpoint by taking and a lock to avoid concurrent checkpointing + /// + /// ID of database to lock + /// True if lock acquired + public bool TryPauseCheckpoints(int dbId); + + /// + /// Release checkpoint task lock + /// + /// ID of database to unlock + public void ResumeCheckpoints(int dbId); + + public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null); + + /// + /// Take checkpoint of all active databases + /// + /// True if method can return before checkpoint is taken + /// Store type to checkpoint + /// Logger + /// Cancellation token + /// False if another checkpointing process is already in progress + public bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null, CancellationToken token = default); + + /// + /// Take checkpoint of specified database ID + /// + /// True if method can return before checkpoint is taken + /// ID of database to checkpoint + /// Store type to checkpoint + /// Logger + /// Cancellation token + /// False if another checkpointing process is already in progress + public bool TakeCheckpoint(bool background, int dbId, StoreType storeType = StoreType.All, + ILogger logger = null, CancellationToken token = default); + + /// + /// Take a checkpoint if no checkpoint was taken after the provided time offset + /// + /// Time offset + /// ID of database to checkpoint (default: DB 0) + /// Task + public Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, int dbId = 0); + + /// + /// Take a checkpoint of all active databases whose AOF size has reached a specified limit + /// + /// AOF size limit + /// Cancellation token + /// Logger + /// Task + public Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, CancellationToken token = default, + ILogger logger = null); + + /// + /// Commit to AOF for all active databases + /// + /// Cancellation token + /// Logger + /// Task + public Task CommitToAofAsync(CancellationToken token = default, ILogger logger = null); + + /// + /// Commit to AOF for specified database + /// + /// ID of database to commit + /// Cancellation token + /// Logger + /// Task + public Task CommitToAofAsync(int dbId, CancellationToken token = default, ILogger logger = null); + + /// + /// Wait for commit to AOF for all active databases + /// + /// Cancellation token + /// Logger + /// Task + public Task WaitForCommitToAofAsync(CancellationToken token = default, ILogger logger = null); + + /// + /// Recover AOF + /// + public void RecoverAOF(); + + /// + /// When replaying AOF we do not want to write AOF records again. + /// + public long ReplayAOF(long untilAddress = -1); + + /// + /// Do compaction + /// + public void DoCompaction(CancellationToken token = default, ILogger logger = null); + + /// + /// Grows indexes of both main store and object store for all active databases if current size is too small + /// + /// True if indexes are maxed out + public bool GrowIndexesIfNeeded(CancellationToken token = default); + + /// + /// Start object size trackers for all active databases + /// + public void StartObjectSizeTrackers(CancellationToken token = default); + + /// + /// Reset + /// + /// Database ID + public void Reset(int dbId = 0); + + /// + /// Resets the revivification stats. + /// + public void ResetRevivificationStats(); + + /// + /// Append a checkpoint commit to the AOF + /// + /// + /// + /// + public void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0); + + /// + /// Get a snapshot of all active databases + /// + /// Array of active databases + public GarnetDatabase[] GetDatabasesSnapshot(); + + /// + /// Get database DB ID + /// + /// DB Id + /// True if database was found + /// Reference to database + public ref GarnetDatabase TryGetDatabase(int dbId, out bool found); + + /// + /// Flush database with specified ID + /// + /// Truncate log + /// Database ID + public void FlushDatabase(bool unsafeTruncateLog, int dbId = 0); + + /// + /// Flush all active databases + /// + /// Truncate log + public void FlushAllDatabases(bool unsafeTruncateLog); + + /// + /// Try to swap between two database instances + /// + /// First database ID + /// Second database ID + /// True if swap successful + public bool TrySwapDatabases(int dbId1, int dbId2); + + /// + /// Create a shallow copy of the IDatabaseManager instance and copy databases to the new instance + /// + /// True if AOF should be enabled in the clone + /// + public IDatabaseManager Clone(bool enableAof); + + internal FunctionsState CreateFunctionsState(int dbId = 0); + } +} \ No newline at end of file diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs new file mode 100644 index 0000000000..67ef79a15c --- /dev/null +++ b/libs/server/Databases/MultiDatabaseManager.cs @@ -0,0 +1,1019 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.IO; +using System.Linq; +using System.Threading; +using System.Threading.Tasks; +using Garnet.common; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.server +{ + internal class MultiDatabaseManager : DatabaseManagerBase + { + /// + public override ref GarnetDatabase DefaultDatabase => ref databases.Map[0]; + + /// + public override int DatabaseCount => activeDbIdsLength; + + readonly CancellationTokenSource cts = new(); + + readonly int maxDatabases; + + // Map of databases by database ID (by default: of size 1, contains only DB 0) + ExpandableMap databases; + + SingleWriterMultiReaderLock databasesLock; + + // Array containing active database IDs + int[] activeDbIds; + + // Total number of current active database IDs + int activeDbIdsLength; + + // Last DB ID activated + int lastActivatedDbId = -1; + + // Reusable task array for tracking checkpointing of multiple DBs + // Used by recurring checkpointing task if multiple DBs exist + Task[] checkpointTasks; + + // Reusable array for storing database IDs for checkpointing + int[] dbIdsToCheckpoint; + + public MultiDatabaseManager(StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate, + StoreWrapper storeWrapper, bool createDefaultDatabase = true) : base(createDatabaseDelegate, storeWrapper) + { + maxDatabases = storeWrapper.serverOptions.MaxDatabases; + Logger = storeWrapper.loggerFactory?.CreateLogger(nameof(MultiDatabaseManager)); + + // Create default databases map of size 1 + databases = new ExpandableMap(1, 0, maxDatabases - 1); + + // Create default database of index 0 (unless specified otherwise) + if (createDefaultDatabase) + { + var db = createDatabaseDelegate(0); + + // Set new database in map + if (!TryAddDatabase(0, ref db)) + throw new GarnetException("Failed to set initial database in databases map"); + } + } + + public MultiDatabaseManager(MultiDatabaseManager src, bool enableAof) : this(src.CreateDatabaseDelegate, + src.StoreWrapper, createDefaultDatabase: false) + { + CopyDatabases(src, enableAof); + } + + public MultiDatabaseManager(SingleDatabaseManager src) : + this(src.CreateDatabaseDelegate, src.StoreWrapper, false) + { + CopyDatabases(src, src.StoreWrapper.serverOptions.EnableAOF); + } + + /// + public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) + { + if (replicaRecover) + throw new GarnetException( + $"Unexpected call to {nameof(MultiDatabaseManager)}.{nameof(RecoverCheckpoint)} with {nameof(replicaRecover)} == true."); + + var checkpointParentDir = StoreWrapper.serverOptions.MainStoreCheckpointBaseDirectory; + var checkpointDirBaseName = StoreWrapper.serverOptions.GetCheckpointDirectoryName(0); + + int[] dbIdsToRecover; + try + { + if (!TryGetSavedDatabaseIds(checkpointParentDir, checkpointDirBaseName, out dbIdsToRecover)) + return; + } + catch (Exception ex) + { + Logger?.LogInformation(ex, + "Error during recovery of database ids; checkpointParentDir = {checkpointParentDir}; checkpointDirBaseName = {checkpointDirBaseName}", + checkpointParentDir, checkpointDirBaseName); + if (StoreWrapper.serverOptions.FailOnRecoveryError) + throw; + return; + } + + long storeVersion = -1, objectStoreVersion = -1; + + foreach (var dbId in dbIdsToRecover) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Failed to retrieve or create database for checkpoint recovery (DB ID = {dbId})."); + + try + { + RecoverDatabaseCheckpoint(ref db, out storeVersion, out objectStoreVersion); + } + catch (TsavoriteNoHybridLogException ex) + { + // No hybrid log being found is not the same as an error in recovery. e.g. fresh start + Logger?.LogInformation(ex, + "No Hybrid Log found for recovery; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", + storeVersion, objectStoreVersion); + } + catch (Exception ex) + { + Logger?.LogInformation(ex, + "Error during recovery of store; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", + storeVersion, objectStoreVersion); + if (StoreWrapper.serverOptions.FailOnRecoveryError) + throw; + } + } + } + + /// + public override bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default) + { + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return false; + + try + { + var activeDbIdsSize = activeDbIdsLength; + Array.Copy(activeDbIds, dbIdsToCheckpoint, activeDbIdsSize); + + TakeDatabasesCheckpointAsync(storeType, activeDbIdsSize, logger: logger, token: token).GetAwaiter().GetResult(); + } + finally + { + databasesLock.ReadUnlock(); + } + + return true; + } + + /// + public override bool TakeCheckpoint(bool background, int dbId, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + if (!TryPauseCheckpointsContinuousAsync(dbId, token).GetAwaiter().GetResult()) + return false; + + var checkpointTask = TakeCheckpointAsync(databasesMapSnapshot[dbId], storeType, logger: logger, token: token).ContinueWith( + _ => + { + TryUpdateLastSaveTimeAsync(dbId, token).GetAwaiter().GetResult(); + ResumeCheckpoints(dbId); + }, TaskContinuationOptions.ExecuteSynchronously).GetAwaiter(); + + if (background) + return true; + + checkpointTask.GetResult(); + return true; + } + + /// + public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, int dbId = 0) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + // Take lock to ensure no other task will be taking a checkpoint + var checkpointsPaused = TryPauseCheckpoints(dbId); + + try + { + // If an external task has taken a checkpoint beyond the provided entryTime return + if (!checkpointsPaused || databasesMapSnapshot[dbId].LastSaveTime > entryTime) + return; + + // Necessary to take a checkpoint because the latest checkpoint is before entryTime + await TakeCheckpointAsync(databasesMapSnapshot[dbId], StoreType.All, logger: Logger); + + TryUpdateLastSaveTimeAsync(dbId).GetAwaiter().GetResult(); + } + finally + { + ResumeCheckpoints(dbId); + } + } + + /// + public override async Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, CancellationToken token = default, + ILogger logger = null) + { + var lockAcquired = await TryGetDatabasesReadLockAsync(token); + if (!lockAcquired) return; + + try + { + var databasesMapSnapshot = databases.Map; + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + var dbIdsIdx = 0; + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var db = databasesMapSnapshot[dbId]; + Debug.Assert(!db.IsDefault()); + + var dbAofSize = db.AppendOnlyFile.TailAddress - db.AppendOnlyFile.BeginAddress; + if (dbAofSize > aofSizeLimit) + { + logger?.LogInformation("Enforcing AOF size limit currentAofSize: {dbAofSize} > AofSizeLimit: {aofSizeLimit} (Database ID: {dbId})", + dbAofSize, aofSizeLimit, dbId); + dbIdsToCheckpoint[dbIdsIdx++] = dbId; + break; + } + } + + if (dbIdsIdx == 0) return; + + await TakeDatabasesCheckpointAsync(StoreType.All, dbIdsIdx, logger: logger, token: token); + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override async Task CommitToAofAsync(CancellationToken token = default, ILogger logger = null) + { + var databasesMapSnapshot = databases.Map; + + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + // Take a read lock to make sure that swap-db operation is not in progress + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return; + + var aofTasks = new Task<(long, long)>[activeDbIdsSize]; + + try + { + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var db = databasesMapSnapshot[dbId]; + Debug.Assert(!db.IsDefault()); + + aofTasks[i] = db.AppendOnlyFile.CommitAsync(token: token).AsTask().ContinueWith(_ => (db.AppendOnlyFile.TailAddress, db.AppendOnlyFile.CommittedUntilAddress), token); + } + + var exThrown = false; + try + { + await Task.WhenAll(aofTasks); + } + catch (Exception) + { + // Only first exception is caught here, if any. + // Proper handling of this and consequent exceptions in the next loop. + exThrown = true; + } + + foreach (var t in aofTasks) + { + if (!t.IsFaulted || t.Exception == null) continue; + + logger?.LogError(t.Exception, + "Exception raised while committing to AOF. AOF tail address = {tailAddress}; AOF committed until address = {commitAddress}; ", + t.Result.Item1, t.Result.Item2); + } + + if (exThrown) + throw new GarnetException($"Error occurred while committing to AOF in {nameof(MultiDatabaseManager)}. Refer to previous log messages for more details."); + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override async Task CommitToAofAsync(int dbId, CancellationToken token = default, ILogger logger = null) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + await databasesMapSnapshot[dbId].AppendOnlyFile.CommitAsync(token: token); + } + + /// + public override async Task WaitForCommitToAofAsync(CancellationToken token = default, ILogger logger = null) + { + // Take a read lock to make sure that swap-db operation is not in progress + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return; + + try + { + var databasesMapSnapshot = databases.Map; + + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + var aofTasks = new Task[activeDbIdsSize]; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var db = databasesMapSnapshot[dbId]; + Debug.Assert(!db.IsDefault()); + + aofTasks[i] = db.AppendOnlyFile.WaitForCommitAsync(token: token).AsTask(); + } + + await Task.WhenAll(aofTasks); + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override void RecoverAOF() + { + var aofParentDir = StoreWrapper.serverOptions.AppendOnlyFileBaseDirectory; + var aofDirBaseName = StoreWrapper.serverOptions.GetAppendOnlyFileDirectoryName(0); + + int[] dbIdsToRecover; + try + { + if (!TryGetSavedDatabaseIds(aofParentDir, aofDirBaseName, out dbIdsToRecover)) + return; + + } + catch (Exception ex) + { + Logger?.LogInformation(ex, + "Error during recovery of database ids; aofParentDir = {aofParentDir}; aofDirBaseName = {aofDirBaseName}", + aofParentDir, aofDirBaseName); + return; + } + + foreach (var dbId in dbIdsToRecover) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Failed to retrieve or create database for AOF recovery (DB ID = {dbId})."); + + RecoverDatabaseAOF(ref db); + } + } + + /// + public override long ReplayAOF(long untilAddress = -1) + { + if (!StoreWrapper.serverOptions.EnableAOF) + return -1; + + // When replaying AOF we do not want to write record again to AOF. + // So initialize local AofProcessor with recordToAof: false. + var aofProcessor = new AofProcessor(StoreWrapper, recordToAof: false, Logger); + + long replicationOffset = 0; + try + { + var databasesMapSnapshot = databases.Map; + + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var offset = ReplayDatabaseAOF(aofProcessor, ref databasesMapSnapshot[dbId], dbId == 0 ? untilAddress : -1); + if (dbId == 0) replicationOffset = offset; + } + } + finally + { + aofProcessor.Dispose(); + } + + return replicationOffset; + } + + /// + public override void DoCompaction(CancellationToken token = default, ILogger logger = null) + { + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return; + + try + { + var databasesMapSnapshot = databases.Map; + + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + var exThrown = false; + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var db = databasesMapSnapshot[dbId]; + Debug.Assert(!db.IsDefault()); + + try + { + DoCompaction(ref db); + } + catch (Exception) + { + exThrown = true; + } + } + + if (exThrown) + throw new GarnetException($"Error occurred during compaction in {nameof(MultiDatabaseManager)}. Refer to previous log messages for more details."); + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override bool GrowIndexesIfNeeded(CancellationToken token = default) + { + var allIndexesMaxedOut = true; + + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return false; + + try + { + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + var databasesMapSnapshot = databases.Map; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + + var indexesMaxedOut = GrowIndexesIfNeeded(ref databasesMapSnapshot[dbId]); + if (allIndexesMaxedOut && !indexesMaxedOut) + allIndexesMaxedOut = false; + } + + return allIndexesMaxedOut; + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override void StartObjectSizeTrackers(CancellationToken token = default) + { + var lockAcquired = TryGetDatabasesReadLockAsync(token).Result; + if (!lockAcquired) return; + + try + { + var databasesMapSnapshot = databases.Map; + + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var db = databasesMapSnapshot[dbId]; + Debug.Assert(!db.IsDefault()); + + db.ObjectStoreSizeTracker?.Start(token); + } + } + finally + { + databasesLock.ReadUnlock(); + } + } + + /// + public override void Reset(int dbId = 0) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Database with ID {dbId} was not found."); + + ResetDatabase(ref db); + } + + /// + public override void ResetRevivificationStats() + { + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + var databaseMapSnapshot = databases.Map; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + databaseMapSnapshot[dbId].MainStore.ResetRevivificationStats(); + databaseMapSnapshot[dbId].ObjectStore?.ResetRevivificationStats(); + } + } + + public override void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Database with ID {dbId} was not found."); + + EnqueueDatabaseCommit(ref db, entryType, version); + } + + /// + public override GarnetDatabase[] GetDatabasesSnapshot() + { + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + var databaseMapSnapshot = databases.Map; + var databasesSnapshot = new GarnetDatabase[activeDbIdsSize]; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + databasesSnapshot[i] = databaseMapSnapshot[dbId]; + } + + return databasesSnapshot; + } + + /// + public override bool TrySwapDatabases(int dbId1, int dbId2) + { + if (dbId1 == dbId2) return true; + + ref var db1 = ref TryGetOrAddDatabase(dbId1, out var success, out _); + if (!success) + return false; + + ref var db2 = ref TryGetOrAddDatabase(dbId2, out success, out _); + if (!success) + return false; + + databasesLock.WriteLock(); + try + { + var databaseMapSnapshot = databases.Map; + var tmp = db1; + databaseMapSnapshot[dbId1] = db2; + databaseMapSnapshot[dbId2] = tmp; + + databaseMapSnapshot[dbId1].Id = dbId1; + databaseMapSnapshot[dbId2].Id = dbId2; + + var sessions = StoreWrapper.TcpServer?.ActiveConsumers().ToArray(); + if (sessions == null) return true; + if (sessions.Length > 1) return false; + + foreach (var session in sessions) + { + if (session is not RespServerSession respServerSession) continue; + + respServerSession.TrySwapDatabaseSessions(dbId1, dbId2); + } + } + finally + { + databasesLock.WriteUnlock(); + } + + return true; + } + + /// + public override IDatabaseManager Clone(bool enableAof) => new MultiDatabaseManager(this, enableAof); + + protected override ref GarnetDatabase GetDatabaseByRef(int dbId = 0) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + return ref databasesMapSnapshot[dbId]; + } + + public override FunctionsState CreateFunctionsState(int dbId = 0) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Database with ID {dbId} was not found."); + + return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper.customCommandManager, null, db.ObjectStoreSizeTracker, + StoreWrapper.GarnetObjectSerializer); + } + + /// + public override ref GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, out bool added) + { + added = false; + success = false; + + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + + if (dbId >= 0 && dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()) + { + success = true; + return ref databasesMapSnapshot[dbId]; + } + + databases.mapLock.WriteLock(); + + try + { + if (dbId >= 0 && dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()) + { + success = true; + return ref databasesMapSnapshot[dbId]; + } + + var db = CreateDatabaseDelegate(dbId); + if (!databases.TrySetValueUnsafe(dbId, ref db, false)) + return ref GarnetDatabase.Empty; + } + finally + { + databases.mapLock.WriteUnlock(); + } + + added = true; + success = true; + + HandleDatabaseAdded(dbId); + + databasesMapSnapshot = databases.Map; + return ref databasesMapSnapshot[dbId]; + } + + /// + public override bool TryPauseCheckpoints(int dbId) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Database with ID {dbId} was not found."); + + return TryPauseCheckpoints(ref db); + } + + /// + public override async Task TryPauseCheckpointsContinuousAsync(int dbId, CancellationToken token = default) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + var checkpointsPaused = TryPauseCheckpoints(ref databasesMapSnapshot[dbId]); + + while (!checkpointsPaused && !token.IsCancellationRequested && !Disposed) + { + await Task.Yield(); + checkpointsPaused = TryPauseCheckpoints(ref databasesMapSnapshot[dbId]); + } + + return checkpointsPaused; + } + + /// + public override void ResumeCheckpoints(int dbId) + { + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + Debug.Assert(dbId < databasesMapSize && !databasesMapSnapshot[dbId].IsDefault()); + + ResumeCheckpoints(ref databasesMapSnapshot[dbId]); + } + + /// + public override ref GarnetDatabase TryGetDatabase(int dbId, out bool found) + { + found = false; + + var databasesMapSize = databases.ActualSize; + var databasesMapSnapshot = databases.Map; + + if (dbId == 0) + { + Debug.Assert(!databasesMapSnapshot[0].IsDefault()); + found = true; + return ref databasesMapSnapshot[0]; + } + + // Check if database already exists + if (dbId < databasesMapSize) + { + if (!databasesMapSnapshot[dbId].IsDefault()) + { + found = true; + return ref databasesMapSnapshot[dbId]; + } + } + + found = false; + return ref GarnetDatabase.Empty; + } + + /// + public override void FlushDatabase(bool unsafeTruncateLog, int dbId = 0) + { + ref var db = ref TryGetOrAddDatabase(dbId, out var success, out _); + if (!success) + throw new GarnetException($"Database with ID {dbId} was not found."); + + FlushDatabase(ref db, unsafeTruncateLog); + } + + /// + public override void FlushAllDatabases(bool unsafeTruncateLog) + { + var activeDbIdsSize = activeDbIdsLength; + var activeDbIdsSnapshot = activeDbIds; + var databaseMapSnapshot = databases.Map; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + FlushDatabase(ref databaseMapSnapshot[dbId], unsafeTruncateLog); + } + } + + /// + /// Continuously try to take a database read lock + /// + /// Cancellation token + /// True if lock acquired + public async Task TryGetDatabasesReadLockAsync(CancellationToken token = default) + { + var lockAcquired = databasesLock.TryReadLock(); + + while (!lockAcquired && !token.IsCancellationRequested && !Disposed) + { + await Task.Yield(); + lockAcquired = databasesLock.TryReadLock(); + } + + return lockAcquired; + } + + /// + /// Continuously try to take a database write lock + /// + /// Cancellation token + /// True if lock acquired + public async Task TryGetDatabasesWriteLockAsync(CancellationToken token = default) + { + var lockAcquired = databasesLock.TryWriteLock(); + + while (!lockAcquired && !token.IsCancellationRequested && !Disposed) + { + await Task.Yield(); + lockAcquired = databasesLock.TryWriteLock(); + } + + return lockAcquired; + } + + /// + /// Retrieves saved database IDs from parent checkpoint / AOF path + /// e.g. if path contains directories: baseName, baseName_1, baseName_2, baseName_10 + /// DB IDs 0,1,2,10 will be returned + /// + /// Parent path + /// Base name of directories containing database-specific checkpoints / AOFs + /// DB IDs extracted from parent path + /// True if successful + internal static bool TryGetSavedDatabaseIds(string path, string baseName, out int[] dbIds) + { + dbIds = default; + if (!Directory.Exists(path)) return false; + + var dirs = Directory.GetDirectories(path, $"{baseName}*", SearchOption.TopDirectoryOnly); + dbIds = new int[dirs.Length]; + for (var i = 0; i < dirs.Length; i++) + { + var dirName = new DirectoryInfo(dirs[i]).Name; + var sepIdx = dirName.IndexOf('_'); + var dbId = 0; + + if (sepIdx != -1 && !int.TryParse(dirName.AsSpan(sepIdx + 1), out dbId)) + continue; + + dbIds[i] = dbId; + } + + return true; + } + + /// + /// Try to add a new database + /// + /// Database ID + /// Database + /// + private bool TryAddDatabase(int dbId, ref GarnetDatabase db) + { + if (!databases.TrySetValue(dbId, ref db)) + return false; + + HandleDatabaseAdded(dbId); + return true; + } + + /// + /// Handle a new database added + /// + /// ID of database added + private void HandleDatabaseAdded(int dbId) + { + // If size tracker exists and is stopped, start it (only if DB 0 size tracker is started as well) + var db = databases.Map[dbId]; + if (dbId != 0 && ObjectStoreSizeTracker != null && !ObjectStoreSizeTracker.Stopped && + db.ObjectStoreSizeTracker != null && db.ObjectStoreSizeTracker.Stopped) + db.ObjectStoreSizeTracker.Start(cts.Token); + + var dbIdIdx = Interlocked.Increment(ref lastActivatedDbId); + + // If there is no size increase needed for activeDbIds, set the added ID in the array + if (activeDbIds != null && dbIdIdx < activeDbIds.Length) + { + activeDbIds[dbIdIdx] = dbId; + Interlocked.Increment(ref activeDbIdsLength); + return; + } + + if (!TryGetDatabasesWriteLockAsync().GetAwaiter().GetResult()) return; + + try + { + // Select the next size of activeDbIds (as multiple of 2 from the existing size) + var newSize = activeDbIds?.Length ?? 1; + while (dbIdIdx + 1 > newSize) + { + newSize = Math.Min(maxDatabases, newSize * 2); + } + + // Set an updated instance of activeDbIds + var activeDbIdsSnapshot = activeDbIds; + var activeDbIdsUpdated = new int[newSize]; + + if (activeDbIdsSnapshot != null) + { + Array.Copy(activeDbIdsSnapshot, activeDbIdsUpdated, dbIdIdx); + } + + // Set the last added ID + activeDbIdsUpdated[dbIdIdx] = dbId; + + activeDbIds = activeDbIdsUpdated; + activeDbIdsLength = dbIdIdx + 1; + checkpointTasks = new Task[activeDbIdsLength]; + dbIdsToCheckpoint = new int[activeDbIdsLength]; + } + finally + { + databasesLock.WriteUnlock(); + } + } + + /// + /// Copy active databases from specified IDatabaseManager instance + /// + /// Source IDatabaseManager + /// True if should enable AOF in copied databases + private void CopyDatabases(IDatabaseManager src, bool enableAof) + { + switch (src) + { + case SingleDatabaseManager sdbm: + var defaultDbCopy = new GarnetDatabase(ref sdbm.DefaultDatabase, enableAof); + TryAddDatabase(0, ref defaultDbCopy); + return; + case MultiDatabaseManager mdbm: + var activeDbIdsSize = mdbm.activeDbIdsLength; + var activeDbIdsSnapshot = mdbm.activeDbIds; + var databasesMapSnapshot = mdbm.databases.Map; + + for (var i = 0; i < activeDbIdsSize; i++) + { + var dbId = activeDbIdsSnapshot[i]; + var dbCopy = new GarnetDatabase(ref databasesMapSnapshot[dbId], enableAof); + TryAddDatabase(dbId, ref dbCopy); + } + + return; + default: + throw new NotImplementedException(); + } + } + + /// + /// Asynchronously checkpoint multiple databases and wait for all to complete + /// + /// Store type to checkpoint + /// Number of databases to checkpoint (first dbIdsCount indexes from dbIdsToCheckpoint) + /// Logger + /// Cancellation token + /// False if checkpointing already in progress + private async Task TakeDatabasesCheckpointAsync(StoreType storeType, int dbIdsCount, ILogger logger = null, + CancellationToken token = default) + { + Debug.Assert(checkpointTasks != null); + Debug.Assert(dbIdsCount <= dbIdsToCheckpoint.Length); + + for (var i = 0; i < checkpointTasks.Length; i++) + checkpointTasks[i] = Task.CompletedTask; + + var lockAcquired = await TryGetDatabasesReadLockAsync(token); + if (!lockAcquired) return; + + try + { + var databaseMapSnapshot = databases.Map; + + var currIdx = 0; + while (currIdx < dbIdsCount) + { + var dbId = dbIdsToCheckpoint[currIdx]; + + // Prevent parallel checkpoint + if (!await TryPauseCheckpointsContinuousAsync(dbId, token)) + continue; + + checkpointTasks[currIdx] = TakeCheckpointAsync(databaseMapSnapshot[dbId], storeType, logger: logger, token: token).ContinueWith( + _ => + { + TryUpdateLastSaveTimeAsync(dbId, token).GetAwaiter().GetResult(); + ResumeCheckpoints(dbId); + }, TaskContinuationOptions.ExecuteSynchronously); + + currIdx++; + } + + await Task.WhenAll(checkpointTasks); + } + catch (Exception ex) + { + logger?.LogError(ex, "Checkpointing threw exception"); + } + finally + { + databasesLock.ReadUnlock(); + } + } + + private async Task TryUpdateLastSaveTimeAsync(int dbId, CancellationToken token = default) + { + var lockAcquired = await TryGetDatabasesReadLockAsync(token); + if (!lockAcquired) return false; + + var databasesMapSnapshot = databases.Map; + + try + { + databasesMapSnapshot[dbId].LastSaveTime = DateTimeOffset.UtcNow; + return true; + } + finally + { + databasesLock.ReadUnlock(); + } + } + + public override void Dispose() + { + if (Disposed) return; + + cts.Cancel(); + Disposed = true; + + // Disable changes to databases map and dispose all databases + databases.mapLock.WriteLock(); + foreach (var db in databases.Map) + db.Dispose(); + + cts.Dispose(); + } + } +} \ No newline at end of file diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs new file mode 100644 index 0000000000..a94355c2a7 --- /dev/null +++ b/libs/server/Databases/SingleDatabaseManager.cs @@ -0,0 +1,348 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.server +{ + internal class SingleDatabaseManager : DatabaseManagerBase + { + /// + public override ref GarnetDatabase DefaultDatabase => ref defaultDatabase; + + /// + public override int DatabaseCount => 1; + + GarnetDatabase defaultDatabase; + + public SingleDatabaseManager(StoreWrapper.DatabaseCreatorDelegate createDatabaseDelegate, StoreWrapper storeWrapper, bool createDefaultDatabase = true) : + base(createDatabaseDelegate, storeWrapper) + { + Logger = storeWrapper.loggerFactory?.CreateLogger(nameof(SingleDatabaseManager)); + + // Create default database of index 0 (unless specified otherwise) + if (createDefaultDatabase) + { + defaultDatabase = createDatabaseDelegate(0); + } + } + + public SingleDatabaseManager(SingleDatabaseManager src, bool enableAof) : this(src.CreateDatabaseDelegate, src.StoreWrapper, createDefaultDatabase: false) + { + CopyDatabases(src, enableAof); + } + + /// + public override ref GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, out bool added) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + success = true; + added = false; + return ref DefaultDatabase; + } + + public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) + { + long storeVersion = -1, objectStoreVersion = -1; + try + { + if (replicaRecover) + { + // Note: Since replicaRecover only pertains to cluster-mode, we can use the default store pointers (since multi-db mode is disabled in cluster-mode) + if (metadata!.storeIndexToken != default && metadata.storeHlogToken != default) + { + storeVersion = !recoverMainStoreFromToken ? MainStore.Recover() : MainStore.Recover(metadata.storeIndexToken, metadata.storeHlogToken); + } + + if (ObjectStore != null) + { + if (metadata.objectStoreIndexToken != default && metadata.objectStoreHlogToken != default) + { + objectStoreVersion = !recoverObjectStoreFromToken ? ObjectStore.Recover() : ObjectStore.Recover(metadata.objectStoreIndexToken, metadata.objectStoreHlogToken); + } + } + + if (storeVersion > 0 || objectStoreVersion > 0) + DefaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; + } + else + { + RecoverDatabaseCheckpoint(ref DefaultDatabase, out storeVersion, out objectStoreVersion); + } + } + catch (TsavoriteNoHybridLogException ex) + { + // No hybrid log being found is not the same as an error in recovery. e.g. fresh start + Logger?.LogInformation(ex, + "No Hybrid Log found for recovery; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", + storeVersion, objectStoreVersion); + } + catch (Exception ex) + { + Logger?.LogInformation(ex, + "Error during recovery of store; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", + storeVersion, objectStoreVersion); + + if (StoreWrapper.serverOptions.FailOnRecoveryError) + throw; + } + } + + /// + public override bool TryPauseCheckpoints(int dbId) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + return TryPauseCheckpoints(ref DefaultDatabase); + } + + /// + public override async Task TryPauseCheckpointsContinuousAsync(int dbId, + CancellationToken token = default) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + var checkpointsPaused = TryPauseCheckpoints(ref DefaultDatabase); + + while (!checkpointsPaused && !token.IsCancellationRequested && !Disposed) + { + await Task.Yield(); + checkpointsPaused = TryPauseCheckpoints(ref DefaultDatabase); + } + + return checkpointsPaused; + } + + /// + public override void ResumeCheckpoints(int dbId) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + ResumeCheckpoints(ref DefaultDatabase); + } + + /// + public override bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default) + { + if (!TryPauseCheckpointsContinuousAsync(DefaultDatabase.Id, token: token).GetAwaiter().GetResult()) + return false; + + var checkpointTask = TakeCheckpointAsync(DefaultDatabase, storeType, logger: logger, token: token).ContinueWith( + _ => + { + DefaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; + ResumeCheckpoints(DefaultDatabase.Id); + }, TaskContinuationOptions.ExecuteSynchronously).GetAwaiter(); + + if (background) + return true; + + checkpointTask.GetResult(); + return true; + } + + /// + public override bool TakeCheckpoint(bool background, int dbId, StoreType storeType = StoreType.All, + ILogger logger = null, + CancellationToken token = default) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + return TakeCheckpoint(background, storeType, logger, token); + } + + /// + public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, int dbId = 0) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + // Take lock to ensure no other task will be taking a checkpoint + var checkpointsPaused = TryPauseCheckpoints(dbId); + + try + { + // If an external task has taken a checkpoint beyond the provided entryTime return + if (!checkpointsPaused || DefaultDatabase.LastSaveTime > entryTime) + return; + + // Necessary to take a checkpoint because the latest checkpoint is before entryTime + await TakeCheckpointAsync(DefaultDatabase, StoreType.All, logger: Logger); + + DefaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; + } + finally + { + ResumeCheckpoints(dbId); + } + } + + /// + public override async Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, + CancellationToken token = default, ILogger logger = null) + { + var aofSize = AppendOnlyFile.TailAddress - AppendOnlyFile.BeginAddress; + if (aofSize <= aofSizeLimit) return; + + if (!TryPauseCheckpointsContinuousAsync(DefaultDatabase.Id, token: token).GetAwaiter().GetResult()) + return; + + logger?.LogInformation("Enforcing AOF size limit currentAofSize: {aofSize} > AofSizeLimit: {aofSizeLimit}", + aofSize, aofSizeLimit); + + try + { + await TakeCheckpointAsync(DefaultDatabase, StoreType.All, logger: logger, token: token); + + DefaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; + } + finally + { + ResumeCheckpoints(DefaultDatabase.Id); + } + } + + /// + public override async Task CommitToAofAsync(CancellationToken token = default, ILogger logger = null) + { + try + { + await AppendOnlyFile.CommitAsync(token: token); + } + catch (Exception ex) + { + logger?.LogError(ex, + "Exception raised while committing to AOF. AOF tail address = {tailAddress}; AOF committed until address = {commitAddress}; ", + AppendOnlyFile.TailAddress, AppendOnlyFile.CommittedUntilAddress); + throw; + } + } + + /// + public override async Task CommitToAofAsync(int dbId, CancellationToken token = default, ILogger logger = null) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + await CommitToAofAsync(token, logger); + } + + public override async Task WaitForCommitToAofAsync(CancellationToken token = default, ILogger logger = null) + { + await AppendOnlyFile.WaitForCommitAsync(token: token); + } + + /// + public override void RecoverAOF() => RecoverDatabaseAOF(ref DefaultDatabase); + + /// + public override long ReplayAOF(long untilAddress = -1) + { + if (!StoreWrapper.serverOptions.EnableAOF) + return -1; + + // When replaying AOF we do not want to write record again to AOF. + // So initialize local AofProcessor with recordToAof: false. + var aofProcessor = new AofProcessor(StoreWrapper, recordToAof: false, Logger); + + try + { + return ReplayDatabaseAOF(aofProcessor, ref DefaultDatabase, untilAddress); + } + finally + { + aofProcessor.Dispose(); + } + } + + /// + public override void DoCompaction(CancellationToken token = default, ILogger logger = null) => DoCompaction(ref DefaultDatabase); + + public override bool GrowIndexesIfNeeded(CancellationToken token = default) => + GrowIndexesIfNeeded(ref DefaultDatabase); + + /// + public override void StartObjectSizeTrackers(CancellationToken token = default) => + ObjectStoreSizeTracker?.Start(token); + + /// + public override void Reset(int dbId = 0) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + ResetDatabase(ref DefaultDatabase); + } + + /// + public override void ResetRevivificationStats() + { + MainStore.ResetRevivificationStats(); + ObjectStore?.ResetRevivificationStats(); + } + + /// + public override void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + EnqueueDatabaseCommit(ref DefaultDatabase, entryType, version); + } + + public override GarnetDatabase[] GetDatabasesSnapshot() => [DefaultDatabase]; + + /// + public override ref GarnetDatabase TryGetDatabase(int dbId, out bool found) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + found = true; + return ref DefaultDatabase; + } + + /// + public override void FlushDatabase(bool unsafeTruncateLog, int dbId = 0) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + FlushDatabase(ref DefaultDatabase, unsafeTruncateLog); + } + + /// + public override void FlushAllDatabases(bool unsafeTruncateLog) => + FlushDatabase(ref DefaultDatabase, unsafeTruncateLog); + + /// + public override bool TrySwapDatabases(int dbId1, int dbId2) => false; + + /// + public override IDatabaseManager Clone(bool enableAof) => new SingleDatabaseManager(this, enableAof); + + protected override ref GarnetDatabase GetDatabaseByRef(int dbId = 0) => ref DefaultDatabase; + + public override FunctionsState CreateFunctionsState(int dbId = 0) + { + ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); + + return new(AppendOnlyFile, VersionMap, StoreWrapper.customCommandManager, null, ObjectStoreSizeTracker, + StoreWrapper.GarnetObjectSerializer); + } + + private void CopyDatabases(SingleDatabaseManager src, bool enableAof) + { + DefaultDatabase = new GarnetDatabase(ref src.DefaultDatabase, enableAof); + } + + public override void Dispose() + { + if (Disposed) return; + + DefaultDatabase.Dispose(); + + Disposed = true; + } + } +} \ No newline at end of file diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs new file mode 100644 index 0000000000..0373a59c9b --- /dev/null +++ b/libs/server/GarnetDatabase.cs @@ -0,0 +1,159 @@ +using System; +using System.Threading; +using Garnet.common; +using Tsavorite.core; + +namespace Garnet.server +{ + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; + + using ObjectStoreAllocator = GenericAllocator>>; + using ObjectStoreFunctions = StoreFunctions>; + + /// + /// Represents a logical database in Garnet + /// + public struct GarnetDatabase : IDefaultChecker, IDisposable + { + /// + /// Default size for version map + /// + // TODO: Change map size to a reasonable number + const int DefaultVersionMapSize = 1 << 16; + + /// + /// Database ID + /// + public int Id; + + /// + /// Main Store + /// + public TsavoriteKV MainStore; + + /// + /// Object Store + /// + public TsavoriteKV ObjectStore; + + /// + /// Size Tracker for Object Store + /// + public CacheSizeTracker ObjectStoreSizeTracker; + + /// + /// Device used for AOF logging + /// + public IDevice AofDevice; + + /// + /// AOF log + /// + public TsavoriteLog AppendOnlyFile; + + /// + /// Version map + /// + public WatchVersionMap VersionMap; + + /// + /// Tail address of main store log at last save + /// + public long LastSaveStoreTailAddress; + + /// + /// Tail address of object store log at last save + /// + public long LastSaveObjectStoreTailAddress; + + /// + /// Last time checkpoint of database was taken + /// + public DateTimeOffset LastSaveTime; + + /// + /// True if database's main store index has maxed-out + /// + public bool MainStoreIndexMaxedOut; + + /// + /// True if database's object store index has maxed-out + /// + public bool ObjectStoreIndexMaxedOut; + + /// + /// Reader-Writer lock for database checkpointing + /// + public SingleWriterMultiReaderLock CheckpointingLock; + + bool disposed = false; + + public GarnetDatabase(int id, TsavoriteKV mainStore, + TsavoriteKV objectStore, + CacheSizeTracker objectStoreSizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile, + bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut) : this() + { + Id = id; + MainStore = mainStore; + ObjectStore = objectStore; + ObjectStoreSizeTracker = objectStoreSizeTracker; + AofDevice = aofDevice; + AppendOnlyFile = appendOnlyFile; + MainStoreIndexMaxedOut = mainStoreIndexMaxedOut; + ObjectStoreIndexMaxedOut = objectStoreIndexMaxedOut; + } + + public GarnetDatabase(ref GarnetDatabase srcDb, bool enableAof) : this() + { + Id = srcDb.Id; + MainStore = srcDb.MainStore; + ObjectStore = srcDb.ObjectStore; + ObjectStoreSizeTracker = srcDb.ObjectStoreSizeTracker; + AofDevice = enableAof ? srcDb.AofDevice : null; + AppendOnlyFile = enableAof ? srcDb.AppendOnlyFile : null; + MainStoreIndexMaxedOut = srcDb.MainStoreIndexMaxedOut; + ObjectStoreIndexMaxedOut = srcDb.ObjectStoreIndexMaxedOut; + } + + public GarnetDatabase() + { + VersionMap = new WatchVersionMap(DefaultVersionMapSize); + LastSaveStoreTailAddress = 0; + LastSaveObjectStoreTailAddress = 0; + LastSaveTime = DateTimeOffset.FromUnixTimeSeconds(0); + } + + public bool IsDefault() => MainStore == null; + + public void Dispose() + { + if (disposed) return; + + // Wait for checkpoints to complete and disable checkpointing + CheckpointingLock.CloseLock(); + + MainStore?.Dispose(); + ObjectStore?.Dispose(); + AofDevice?.Dispose(); + AppendOnlyFile?.Dispose(); + + if (ObjectStoreSizeTracker != null) + { + // If tracker has previously started, wait for it to stop + if (!ObjectStoreSizeTracker.TryPreventStart()) + { + while (!ObjectStoreSizeTracker.Stopped) + Thread.Yield(); + } + } + + disposed = true; + } + + /// + /// Instance of empty database + /// + internal static GarnetDatabase Empty; + } +} \ No newline at end of file diff --git a/libs/server/Metrics/GarnetServerMonitor.cs b/libs/server/Metrics/GarnetServerMonitor.cs index 3ab5ab2c23..a27b41737b 100644 --- a/libs/server/Metrics/GarnetServerMonitor.cs +++ b/libs/server/Metrics/GarnetServerMonitor.cs @@ -193,8 +193,7 @@ private void ResetStats() storeWrapper.clusterProvider?.ResetGossipStats(); - storeWrapper.store.ResetRevivificationStats(); - storeWrapper.objectStore?.ResetRevivificationStats(); + storeWrapper.ResetRevivificationStats(); resetEventFlags[InfoMetricsType.STATS] = false; } diff --git a/libs/server/Metrics/Info/GarnetInfoMetrics.cs b/libs/server/Metrics/Info/GarnetInfoMetrics.cs index d9137b78c5..3b5cbda61b 100644 --- a/libs/server/Metrics/Info/GarnetInfoMetrics.cs +++ b/libs/server/Metrics/Info/GarnetInfoMetrics.cs @@ -6,6 +6,7 @@ using System.Diagnostics; using System.Globalization; using System.Linq; +using System.Text; using Garnet.common; namespace Garnet.server @@ -55,7 +56,7 @@ private void PopulateServerInfo(StoreWrapper storeWrapper) new("monitor_task", storeWrapper.serverOptions.MetricsSamplingFrequency > 0 ? "enabled" : "disabled"), new("monitor_freq", storeWrapper.serverOptions.MetricsSamplingFrequency.ToString()), new("latency_monitor", storeWrapper.serverOptions.LatencyMonitor ? "enabled" : "disabled"), - new("run_id", storeWrapper.run_id), + new("run_id", storeWrapper.runId), new("redis_version", storeWrapper.redisProtocolVersion), new("redis_mode", storeWrapper.serverOptions.EnableCluster ? "cluster" : "standalone"), ]; @@ -63,10 +64,10 @@ private void PopulateServerInfo(StoreWrapper storeWrapper) private void PopulateMemoryInfo(StoreWrapper storeWrapper) { - var main_store_index_size = storeWrapper.store.IndexSize * 64; - var main_store_log_memory_size = storeWrapper.store.Log.MemorySizeBytes; - var main_store_read_cache_size = (storeWrapper.store.ReadCache != null ? storeWrapper.store.ReadCache.MemorySizeBytes : 0); - var total_main_store_size = main_store_index_size + main_store_log_memory_size + main_store_read_cache_size; + var main_store_index_size = -1L; + var main_store_log_memory_size = -1L; + var main_store_read_cache_size = -1L; + var total_main_store_size = -1L; var object_store_index_size = -1L; var object_store_log_memory_size = -1L; @@ -74,18 +75,32 @@ private void PopulateMemoryInfo(StoreWrapper storeWrapper) var object_store_heap_memory_size = -1L; var object_store_read_cache_heap_memory_size = -1L; var total_object_store_size = -1L; - var disableObj = storeWrapper.serverOptions.DisableObjects; - var aof_log_memory_size = storeWrapper.appendOnlyFile?.MemorySizeBytes ?? -1; + var aof_log_memory_size = -1L; + + var databases = storeWrapper.GetDatabasesSnapshot(); + var disableObj = storeWrapper.serverOptions.DisableObjects; - if (!disableObj) + foreach (var db in databases) { - object_store_index_size = storeWrapper.objectStore.IndexSize * 64; - object_store_log_memory_size = storeWrapper.objectStore.Log.MemorySizeBytes; - object_store_read_cache_log_memory_size = storeWrapper.objectStore.ReadCache?.MemorySizeBytes ?? 0; - object_store_heap_memory_size = storeWrapper.objectStoreSizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0; - object_store_read_cache_heap_memory_size = storeWrapper.objectStoreSizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0; - total_object_store_size = object_store_index_size + object_store_log_memory_size + object_store_read_cache_log_memory_size + object_store_heap_memory_size + object_store_read_cache_heap_memory_size; + main_store_index_size += db.MainStore.IndexSize * 64; + main_store_log_memory_size += db.MainStore.Log.MemorySizeBytes; + main_store_read_cache_size += db.MainStore.ReadCache?.MemorySizeBytes ?? 0; + total_main_store_size += main_store_index_size + main_store_log_memory_size + main_store_read_cache_size; + + aof_log_memory_size = db.AppendOnlyFile?.MemorySizeBytes ?? -1; + + if (!disableObj) + { + object_store_index_size += db.ObjectStore.IndexSize * 64; + object_store_log_memory_size += db.ObjectStore.Log.MemorySizeBytes; + object_store_read_cache_log_memory_size += db.ObjectStore.ReadCache?.MemorySizeBytes ?? 0; + object_store_heap_memory_size += db.ObjectStoreSizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0; + object_store_read_cache_heap_memory_size += db.ObjectStoreSizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0; + total_object_store_size += object_store_index_size + object_store_log_memory_size + + object_store_read_cache_log_memory_size + object_store_heap_memory_size + + object_store_read_cache_heap_memory_size; + } } var gcMemoryInfo = GC.GetGCMemoryInfo(); @@ -203,76 +218,160 @@ private void PopulateStatsInfo(StoreWrapper storeWrapper) private void PopulateStoreStats(StoreWrapper storeWrapper) { - storeInfo = - [ - new("CurrentVersion", storeWrapper.store.CurrentVersion.ToString()), - new("LastCheckpointedVersion", storeWrapper.store.LastCheckpointedVersion.ToString()), - new("SystemState", storeWrapper.store.SystemState.ToString()), - new("IndexSize", storeWrapper.store.IndexSize.ToString()), - new("LogDir", storeWrapper.serverOptions.LogDir), - new("Log.BeginAddress", storeWrapper.store.Log.BeginAddress.ToString()), - new("Log.BufferSize", storeWrapper.store.Log.BufferSize.ToString()), - new("Log.EmptyPageCount", storeWrapper.store.Log.EmptyPageCount.ToString()), - new("Log.FixedRecordSize", storeWrapper.store.Log.FixedRecordSize.ToString()), - new("Log.HeadAddress", storeWrapper.store.Log.HeadAddress.ToString()), - new("Log.MemorySizeBytes", storeWrapper.store.Log.MemorySizeBytes.ToString()), - new("Log.SafeReadOnlyAddress", storeWrapper.store.Log.SafeReadOnlyAddress.ToString()), - new("Log.TailAddress", storeWrapper.store.Log.TailAddress.ToString()), - new("ReadCache.BeginAddress", storeWrapper.store.ReadCache?.BeginAddress.ToString() ?? "N/A"), - new("ReadCache.BufferSize", storeWrapper.store.ReadCache?.BufferSize.ToString() ?? "N/A"), - new("ReadCache.EmptyPageCount", storeWrapper.store.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), - new("ReadCache.HeadAddress", storeWrapper.store.ReadCache?.HeadAddress.ToString() ?? "N/A"), - new("ReadCache.MemorySizeBytes", storeWrapper.store.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), - new("ReadCache.TailAddress", storeWrapper.store.ReadCache?.TailAddress.ToString() ?? "N/A"), - ]; + var databases = storeWrapper.GetDatabasesSnapshot(); + + for (var i = 0; i < databases.Length; i++) + { + var storeStats = GetDatabaseStoreStats(storeWrapper, ref databases[i]); + if (i == 0) + storeInfo = new MetricsItem[databases.Length * storeStats.Length]; + + Array.Copy(storeStats, 0, storeInfo, storeStats.Length * i, storeStats.Length); + } } + private MetricsItem[] GetDatabaseStoreStats(StoreWrapper storeWrapper, ref GarnetDatabase db) => + [ + new($"db{db.Id}.CurrentVersion", db.MainStore.CurrentVersion.ToString()), + new($"db{db.Id}.LastCheckpointedVersion", db.MainStore.LastCheckpointedVersion.ToString()), + new($"db{db.Id}.SystemState", db.MainStore.SystemState.ToString()), + new($"db{db.Id}.IndexSize", db.MainStore.IndexSize.ToString()), + new($"db{db.Id}.LogDir", storeWrapper.serverOptions.LogDir), + new($"db{db.Id}.Log.BeginAddress", db.MainStore.Log.BeginAddress.ToString()), + new($"db{db.Id}.Log.BufferSize", db.MainStore.Log.BufferSize.ToString()), + new($"db{db.Id}.Log.EmptyPageCount", db.MainStore.Log.EmptyPageCount.ToString()), + new($"db{db.Id}.Log.FixedRecordSize", db.MainStore.Log.FixedRecordSize.ToString()), + new($"db{db.Id}.Log.HeadAddress", db.MainStore.Log.HeadAddress.ToString()), + new($"db{db.Id}.Log.MemorySizeBytes", db.MainStore.Log.MemorySizeBytes.ToString()), + new($"db{db.Id}.Log.SafeReadOnlyAddress", db.MainStore.Log.SafeReadOnlyAddress.ToString()), + new($"db{db.Id}.Log.TailAddress", db.MainStore.Log.TailAddress.ToString()), + new($"db{db.Id}.ReadCache.BeginAddress", db.MainStore.ReadCache?.BeginAddress.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.BufferSize", db.MainStore.ReadCache?.BufferSize.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.EmptyPageCount", db.MainStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.HeadAddress", db.MainStore.ReadCache?.HeadAddress.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.MemorySizeBytes", db.MainStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.TailAddress", db.MainStore.ReadCache?.TailAddress.ToString() ?? "N/A"), + ]; + private void PopulateObjectStoreStats(StoreWrapper storeWrapper) { - objectStoreInfo = - [ - new("CurrentVersion", storeWrapper.objectStore.CurrentVersion.ToString()), - new("LastCheckpointedVersion", storeWrapper.objectStore.LastCheckpointedVersion.ToString()), - new("SystemState", storeWrapper.objectStore.SystemState.ToString()), - new("IndexSize", storeWrapper.objectStore.IndexSize.ToString()), - new("LogDir", storeWrapper.serverOptions.LogDir), - new("Log.BeginAddress", storeWrapper.objectStore.Log.BeginAddress.ToString()), - new("Log.BufferSize", storeWrapper.objectStore.Log.BufferSize.ToString()), - new("Log.EmptyPageCount", storeWrapper.objectStore.Log.EmptyPageCount.ToString()), - new("Log.FixedRecordSize", storeWrapper.objectStore.Log.FixedRecordSize.ToString()), - new("Log.HeadAddress", storeWrapper.objectStore.Log.HeadAddress.ToString()), - new("Log.MemorySizeBytes", storeWrapper.objectStore.Log.MemorySizeBytes.ToString()), - new("Log.SafeReadOnlyAddress", storeWrapper.objectStore.Log.SafeReadOnlyAddress.ToString()), - new("Log.TailAddress", storeWrapper.objectStore.Log.TailAddress.ToString()), - new("ReadCache.BeginAddress", storeWrapper.objectStore.ReadCache?.BeginAddress.ToString() ?? "N/A"), - new("ReadCache.BufferSize", storeWrapper.objectStore.ReadCache?.BufferSize.ToString() ?? "N/A"), - new("ReadCache.EmptyPageCount", storeWrapper.objectStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), - new("ReadCache.HeadAddress", storeWrapper.objectStore.ReadCache?.HeadAddress.ToString() ?? "N/A"), - new("ReadCache.MemorySizeBytes", storeWrapper.objectStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), - new("ReadCache.TailAddress", storeWrapper.objectStore.ReadCache?.TailAddress.ToString() ?? "N/A"), - ]; + var databases = storeWrapper.GetDatabasesSnapshot(); + + for (var i = 0; i < databases.Length; i++) + { + var storeStats = GetDatabaseObjectStoreStats(storeWrapper, ref databases[i]); + if (i == 0) + objectStoreInfo = new MetricsItem[databases.Length * storeStats.Length]; + + Array.Copy(storeStats, 0, objectStoreInfo, storeStats.Length * i, storeStats.Length); + } } - private void PopulateStoreHashDistribution(StoreWrapper storeWrapper) => storeHashDistrInfo = [new("", storeWrapper.store.DumpDistribution())]; + private MetricsItem[] GetDatabaseObjectStoreStats(StoreWrapper storeWrapper, ref GarnetDatabase db) => + [ + new($"db{db.Id}.CurrentVersion", db.ObjectStore.CurrentVersion.ToString()), + new($"db{db.Id}.LastCheckpointedVersion", db.ObjectStore.LastCheckpointedVersion.ToString()), + new($"db{db.Id}.SystemState", db.ObjectStore.SystemState.ToString()), + new($"db{db.Id}.IndexSize", db.ObjectStore.IndexSize.ToString()), + new($"db{db.Id}.LogDir", storeWrapper.serverOptions.LogDir), + new($"db{db.Id}.Log.BeginAddress", db.ObjectStore.Log.BeginAddress.ToString()), + new($"db{db.Id}.Log.BufferSize", db.ObjectStore.Log.BufferSize.ToString()), + new($"db{db.Id}.Log.EmptyPageCount", db.ObjectStore.Log.EmptyPageCount.ToString()), + new($"db{db.Id}.Log.FixedRecordSize", db.ObjectStore.Log.FixedRecordSize.ToString()), + new($"db{db.Id}.Log.HeadAddress", db.ObjectStore.Log.HeadAddress.ToString()), + new($"db{db.Id}.Log.MemorySizeBytes", db.ObjectStore.Log.MemorySizeBytes.ToString()), + new($"db{db.Id}.Log.SafeReadOnlyAddress", db.ObjectStore.Log.SafeReadOnlyAddress.ToString()), + new($"db{db.Id}.Log.TailAddress", db.ObjectStore.Log.TailAddress.ToString()), + new($"db{db.Id}.ReadCache.BeginAddress", db.ObjectStore.ReadCache?.BeginAddress.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.BufferSize", db.ObjectStore.ReadCache?.BufferSize.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.EmptyPageCount", db.ObjectStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.HeadAddress", db.ObjectStore.ReadCache?.HeadAddress.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.MemorySizeBytes", db.ObjectStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), + new($"db{db.Id}.ReadCache.TailAddress", db.ObjectStore.ReadCache?.TailAddress.ToString() ?? "N/A"), + ]; + + private void PopulateStoreHashDistribution(StoreWrapper storeWrapper) + { + var databases = storeWrapper.GetDatabasesSnapshot(); - private void PopulateObjectStoreHashDistribution(StoreWrapper storeWrapper) => objectStoreHashDistrInfo = [new("", storeWrapper.objectStore.DumpDistribution())]; + var sb = new StringBuilder(); + foreach (var db in databases) + { + sb.AppendLine($"db{db.Id}:"); + sb.Append(db.MainStore.DumpDistribution()); + } - private void PopulateStoreRevivInfo(StoreWrapper storeWrapper) => storeRevivInfo = [new("", storeWrapper.store.DumpRevivificationStats())]; + storeHashDistrInfo = [new("", sb.ToString())]; + } - private void PopulateObjectStoreRevivInfo(StoreWrapper storeWrapper) => objectStoreRevivInfo = [new("", storeWrapper.objectStore.DumpRevivificationStats())]; + private void PopulateObjectStoreHashDistribution(StoreWrapper storeWrapper) + { + var databases = storeWrapper.GetDatabasesSnapshot(); + + var sb = new StringBuilder(); + foreach (var db in databases) + { + sb.AppendLine($"db{db.Id}:"); + sb.Append(db.ObjectStore.DumpDistribution()); + } + + objectStoreHashDistrInfo = [new("", sb.ToString())]; + } + + private void PopulateStoreRevivInfo(StoreWrapper storeWrapper) + { + var databases = storeWrapper.GetDatabasesSnapshot(); + + var sb = new StringBuilder(); + foreach (var db in databases) + { + sb.AppendLine($"db{db.Id}:"); + sb.Append(db.MainStore.DumpRevivificationStats()); + } + + storeRevivInfo = [new("", sb.ToString())]; + } + + private void PopulateObjectStoreRevivInfo(StoreWrapper storeWrapper) + { + var databases = storeWrapper.GetDatabasesSnapshot(); + var sb = new StringBuilder(); + foreach (var db in databases) + { + sb.AppendLine($"db{db.Id}:"); + sb.Append(db.ObjectStore.DumpRevivificationStats()); + } + + objectStoreRevivInfo = [new("", sb.ToString())]; + } private void PopulatePersistenceInfo(StoreWrapper storeWrapper) { - bool aofEnabled = storeWrapper.serverOptions.EnableAOF; - persistenceInfo = - [ - new("CommittedBeginAddress", !aofEnabled ? "N/A" : storeWrapper.appendOnlyFile.CommittedBeginAddress.ToString()), - new("CommittedUntilAddress", !aofEnabled ? "N/A" : storeWrapper.appendOnlyFile.CommittedUntilAddress.ToString()), - new("FlushedUntilAddress", !aofEnabled ? "N/A" : storeWrapper.appendOnlyFile.FlushedUntilAddress.ToString()), - new("BeginAddress", !aofEnabled ? "N/A" : storeWrapper.appendOnlyFile.BeginAddress.ToString()), - new("TailAddress", !aofEnabled ? "N/A" : storeWrapper.appendOnlyFile.TailAddress.ToString()), - new("SafeAofAddress", !aofEnabled ? "N/A" : storeWrapper.SafeAofAddress.ToString()) - ]; + var databases = storeWrapper.GetDatabasesSnapshot(); + + for (var i = 0; i < databases.Length; i++) + { + var persistenceStats = GetDatabasePersistenceStats(storeWrapper, ref databases[i]); + if (i == 0) + persistenceInfo = new MetricsItem[databases.Length * persistenceStats.Length]; + + Array.Copy(persistenceStats, 0, persistenceInfo, persistenceStats.Length * i, persistenceStats.Length); + } + } + + private MetricsItem[] GetDatabasePersistenceStats(StoreWrapper storeWrapper, ref GarnetDatabase db) + { + var aofEnabled = storeWrapper.serverOptions.EnableAOF; + + return + [ + new($"db{db.Id}.CommittedBeginAddress", !aofEnabled ? "N/A" : db.AppendOnlyFile.CommittedBeginAddress.ToString()), + new($"db{db.Id}.CommittedUntilAddress", !aofEnabled ? "N/A" : db.AppendOnlyFile.CommittedUntilAddress.ToString()), + new($"db{db.Id}.FlushedUntilAddress", !aofEnabled ? "N/A" : db.AppendOnlyFile.FlushedUntilAddress.ToString()), + new($"db{db.Id}.BeginAddress", !aofEnabled ? "N/A" : db.AppendOnlyFile.BeginAddress.ToString()), + new($"db{db.Id}.TailAddress", !aofEnabled ? "N/A" : db.AppendOnlyFile.TailAddress.ToString()), + new($"db{db.Id}.SafeAofAddress", !aofEnabled ? "N/A" : storeWrapper.safeAofAddress.ToString()) + ]; } private void PopulateClientsInfo(StoreWrapper storeWrapper) @@ -361,25 +460,25 @@ public string GetRespInfo(InfoMetricsType section, StoreWrapper storeWrapper) PopulateStoreStats(storeWrapper); return GetSectionRespInfo(InfoMetricsType.STORE, storeInfo); case InfoMetricsType.OBJECTSTORE: - if (storeWrapper.objectStore == null) return ""; + if (storeWrapper.serverOptions.DisableObjects) return ""; PopulateObjectStoreStats(storeWrapper); return GetSectionRespInfo(InfoMetricsType.OBJECTSTORE, objectStoreInfo); case InfoMetricsType.STOREHASHTABLE: PopulateStoreHashDistribution(storeWrapper); return GetSectionRespInfo(InfoMetricsType.STOREHASHTABLE, storeHashDistrInfo); case InfoMetricsType.OBJECTSTOREHASHTABLE: - if (storeWrapper.objectStore == null) return ""; + if (storeWrapper.serverOptions.DisableObjects) return ""; PopulateObjectStoreHashDistribution(storeWrapper); return GetSectionRespInfo(InfoMetricsType.OBJECTSTOREHASHTABLE, objectStoreHashDistrInfo); case InfoMetricsType.STOREREVIV: PopulateStoreRevivInfo(storeWrapper); return GetSectionRespInfo(InfoMetricsType.STOREREVIV, storeRevivInfo); case InfoMetricsType.OBJECTSTOREREVIV: - if (storeWrapper.objectStore == null) return ""; + if (storeWrapper.serverOptions.DisableObjects) return ""; PopulateObjectStoreRevivInfo(storeWrapper); return GetSectionRespInfo(InfoMetricsType.OBJECTSTOREREVIV, objectStoreRevivInfo); case InfoMetricsType.PERSISTENCE: - if (storeWrapper.appendOnlyFile == null) return ""; + if (!storeWrapper.serverOptions.EnableAOF) return ""; PopulatePersistenceInfo(storeWrapper); return GetSectionRespInfo(InfoMetricsType.PERSISTENCE, persistenceInfo); case InfoMetricsType.CLIENTS: @@ -435,25 +534,25 @@ private MetricsItem[] GetMetricInternal(InfoMetricsType section, StoreWrapper st PopulateStoreStats(storeWrapper); return storeInfo; case InfoMetricsType.OBJECTSTORE: - if (storeWrapper.objectStore == null) return null; + if (storeWrapper.serverOptions.DisableObjects) return null; PopulateObjectStoreStats(storeWrapper); return objectStoreInfo; case InfoMetricsType.STOREHASHTABLE: PopulateStoreHashDistribution(storeWrapper); return storeHashDistrInfo; case InfoMetricsType.OBJECTSTOREHASHTABLE: - if (storeWrapper.objectStore == null) return null; + if (storeWrapper.serverOptions.DisableObjects) return null; PopulateObjectStoreHashDistribution(storeWrapper); return objectStoreHashDistrInfo; case InfoMetricsType.STOREREVIV: PopulateStoreRevivInfo(storeWrapper); return storeRevivInfo; case InfoMetricsType.OBJECTSTOREREVIV: - if (storeWrapper.objectStore == null) return null; + if (storeWrapper.serverOptions.DisableObjects) return null; PopulateObjectStoreRevivInfo(storeWrapper); return objectStoreRevivInfo; case InfoMetricsType.PERSISTENCE: - if (storeWrapper.appendOnlyFile == null) return null; + if (!storeWrapper.serverOptions.EnableAOF) return null; PopulatePersistenceInfo(storeWrapper); return persistenceInfo; case InfoMetricsType.CLIENTS: diff --git a/libs/server/Providers/GarnetProvider.cs b/libs/server/Providers/GarnetProvider.cs index d9c28b7c97..1ccbc6ac24 100644 --- a/libs/server/Providers/GarnetProvider.cs +++ b/libs/server/Providers/GarnetProvider.cs @@ -35,7 +35,7 @@ public sealed class GarnetProvider : TsavoriteKVProviderBase /// Create TsavoriteKV backend /// - /// /// /// - /// /// - public TsavoriteKVProviderBase(TsavoriteKV store, TParameterSerializer serializer, - SubscribeBroker broker = null, bool recoverStore = false, MaxSizeSettings maxSizeSettings = default) + public TsavoriteKVProviderBase(TParameterSerializer serializer, + SubscribeBroker broker = null, MaxSizeSettings maxSizeSettings = default) { - this.store = store; - if (recoverStore) - { - try - { - store.Recover(); - } - catch - { } - } this.broker = broker; this.serializer = serializer; this.maxSizeSettings = maxSizeSettings ?? new MaxSizeSettings(); diff --git a/libs/server/Resp/AdminCommands.cs b/libs/server/Resp/AdminCommands.cs index 5905b61eee..0914e5e77b 100644 --- a/libs/server/Resp/AdminCommands.cs +++ b/libs/server/Resp/AdminCommands.cs @@ -162,9 +162,9 @@ static void OnACLOrNoScriptFailure(RespServerSession self, RespCommand cmd) } } - void CommitAof() + void CommitAof(int dbId = -1) { - storeWrapper.appendOnlyFile?.CommitAsync().ConfigureAwait(false).GetAwaiter().GetResult(); + storeWrapper.CommitAOFAsync(dbId).ConfigureAwait(false).GetAwaiter().GetResult(); } private bool NetworkMonitor() @@ -575,12 +575,41 @@ private bool NetworkModuleLoad(CustomCommandManager customCommandManager) private bool NetworkCOMMITAOF() { - if (parseState.Count != 0) + if (parseState.Count > 1) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.COMMITAOF)); } - CommitAof(); + // By default - commit AOF for all active databases, unless database ID specified + var dbId = -1; + + // Check if ID specified + if (parseState.Count == 1) + { + if (!parseState.TryGetInt(0, out dbId)) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId != 0 && storeWrapper.serverOptions.EnableCluster) + { + // Cluster mode does not allow DBID + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_CLUSTER_MODE, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId >= storeWrapper.serverOptions.MaxDatabases) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_INVALID_INDEX, ref dcurr, dend)) + SendAndReset(); + return true; + } + } + + CommitAof(dbId); while (!RespWriteUtils.TryWriteSimpleString("AOF file committed"u8, ref dcurr, dend)) SendAndReset(); @@ -811,14 +840,47 @@ private bool NetworkROLE() return true; } + /// + /// SAVE [DBID] + /// + /// private bool NetworkSAVE() { - if (parseState.Count != 0) + if (parseState.Count > 1) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.SAVE)); } - if (!storeWrapper.TakeCheckpoint(false, StoreType.All, logger)) + // By default - save all active databases, unless database ID specified + var dbId = -1; + + // Check if ID specified + if (parseState.Count == 1) + { + if (!parseState.TryGetInt(0, out dbId)) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId != 0 && storeWrapper.serverOptions.EnableCluster) + { + // Cluster mode does not allow DBID + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_CLUSTER_MODE, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId >= storeWrapper.serverOptions.MaxDatabases) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_INVALID_INDEX, ref dcurr, dend)) + SendAndReset(); + return true; + } + } + + if (!storeWrapper.TakeCheckpoint(false, dbId: dbId, logger: logger)) { while (!RespWriteUtils.TryWriteError("ERR checkpoint already in progress"u8, ref dcurr, dend)) SendAndReset(); @@ -832,28 +894,105 @@ private bool NetworkSAVE() return true; } + /// + /// LASTSAVE [DBID] + /// + /// private bool NetworkLASTSAVE() { - if (parseState.Count != 0) + if (parseState.Count > 1) { - return AbortWithWrongNumberOfArguments(nameof(RespCommand.SAVE)); + return AbortWithWrongNumberOfArguments(nameof(RespCommand.LASTSAVE)); } - var seconds = storeWrapper.lastSaveTime.ToUnixTimeSeconds(); + // By default - get the last saved timestamp for current active database, unless database ID specified + var dbId = activeDbId; + + // Check if ID specified + if (parseState.Count == 1) + { + if (!parseState.TryGetInt(0, out dbId)) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId != 0 && storeWrapper.serverOptions.EnableCluster) + { + // Cluster mode does not allow DBID + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_CLUSTER_MODE, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId >= storeWrapper.serverOptions.MaxDatabases) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_INVALID_INDEX, ref dcurr, dend)) + SendAndReset(); + return true; + } + } + + var dbFound = storeWrapper.TryGetOrAddDatabase(dbId, out var db, out _); + Debug.Assert(dbFound); + + var seconds = db.LastSaveTime.ToUnixTimeSeconds(); while (!RespWriteUtils.TryWriteInt64(seconds, ref dcurr, dend)) SendAndReset(); return true; } + /// + /// BGSAVE [SCHEDULE] [DBID] + /// + /// private bool NetworkBGSAVE() { - if (parseState.Count > 1) + if (parseState.Count > 2) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.BGSAVE)); } - var success = storeWrapper.TakeCheckpoint(true, StoreType.All, logger); + // By default - save all active databases, unless database ID specified + var dbId = -1; + + var tokenIdx = 0; + if (parseState.Count > 0) + { + if (parseState.GetArgSliceByRef(tokenIdx).ReadOnlySpan + .EqualsUpperCaseSpanIgnoringCase(CmdStrings.SCHEDULE)) + tokenIdx++; + + // Check if ID specified + if (parseState.Count - tokenIdx > 0) + { + if (!parseState.TryGetInt(tokenIdx, out dbId)) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId != 0 && storeWrapper.serverOptions.EnableCluster) + { + // Cluster mode does not allow DBID + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_CLUSTER_MODE, ref dcurr, dend)) + SendAndReset(); + return true; + } + + if (dbId >= storeWrapper.serverOptions.MaxDatabases) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_INVALID_INDEX, ref dcurr, dend)) + SendAndReset(); + return true; + } + } + } + + var success = storeWrapper.TakeCheckpoint(true, dbId: dbId, logger: logger); if (success) { while (!RespWriteUtils.TryWriteSimpleString("Background saving started"u8, ref dcurr, dend)) diff --git a/libs/server/Resp/ArrayCommands.cs b/libs/server/Resp/ArrayCommands.cs index 9b506705a6..8d6c61df51 100644 --- a/libs/server/Resp/ArrayCommands.cs +++ b/libs/server/Resp/ArrayCommands.cs @@ -231,10 +231,68 @@ private bool NetworkSELECT() return true; } - if (index < storeWrapper.databaseNum) + if (index < storeWrapper.serverOptions.MaxDatabases) { - while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) + if (index == this.activeDbId || this.TrySwitchActiveDatabaseSession(index)) + { + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) + SendAndReset(); + } + else + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_DB_INDEX_OUT_OF_RANGE, ref dcurr, dend)) + SendAndReset(); + } + } + else + { + if (storeWrapper.serverOptions.EnableCluster) + { + // Cluster mode does not allow DBID + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_CLUSTER_MODE, ref dcurr, dend)) + SendAndReset(); + } + else + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_SELECT_INVALID_INDEX, ref dcurr, dend)) + SendAndReset(); + } + } + + return true; + } + + /// + /// SWAPDB + /// + /// + private bool NetworkSWAPDB() + { + if (parseState.Count != 2) + { + return AbortWithWrongNumberOfArguments(nameof(RespCommand.SWAPDB)); + } + + // Validate index + if (!parseState.TryGetInt(0, out var index1) || !parseState.TryGetInt(1, out var index2)) + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) SendAndReset(); + return true; + } + + if (index1 < storeWrapper.serverOptions.MaxDatabases && index2 < storeWrapper.serverOptions.MaxDatabases) + { + if (storeWrapper.TrySwapDatabases(index1, index2)) + { + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) + SendAndReset(); + } + else + { + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_DBSWAP_UNSUPPORTED, ref dcurr, dend)) + SendAndReset(); + } } else { @@ -250,6 +308,7 @@ private bool NetworkSELECT() SendAndReset(); } } + return true; } diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs index 29d355446b..e34d2998c4 100644 --- a/libs/server/Resp/BasicCommands.cs +++ b/libs/server/Resp/BasicCommands.cs @@ -1663,19 +1663,26 @@ void FlushDb(RespCommand cmd) } if (async) - Task.Run(() => ExecuteFlushDb(unsafeTruncateLog)).ConfigureAwait(false); + Task.Run(() => ExecuteFlushDb(cmd, unsafeTruncateLog)).ConfigureAwait(false); else - ExecuteFlushDb(unsafeTruncateLog); + ExecuteFlushDb(cmd, unsafeTruncateLog); logger?.LogInformation($"Running {nameof(cmd)} {{async}} {{mode}}", async ? "async" : "sync", unsafeTruncateLog ? " with unsafetruncatelog." : string.Empty); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); } - void ExecuteFlushDb(bool unsafeTruncateLog) + void ExecuteFlushDb(RespCommand cmd, bool unsafeTruncateLog) { - storeWrapper.store.Log.ShiftBeginAddress(storeWrapper.store.Log.TailAddress, truncateLog: unsafeTruncateLog); - storeWrapper.objectStore?.Log.ShiftBeginAddress(storeWrapper.objectStore.Log.TailAddress, truncateLog: unsafeTruncateLog); + switch (cmd) + { + case RespCommand.FLUSHDB: + storeWrapper.FlushDatabase(unsafeTruncateLog, activeDbId); + break; + case RespCommand.FLUSHALL: + storeWrapper.FlushAllDatabases(unsafeTruncateLog); + break; + } } /// @@ -1691,6 +1698,7 @@ private static void WriteClientInfo(IClusterProvider provider, StringBuilder int var localEndpoint = targetSession.networkSender.LocalEndpointName; var clientName = targetSession.clientName; var user = targetSession._userHandle.User; + var db = targetSession.activeDbId; var resp = targetSession.respProtocolVersion; var nodeId = targetSession?.clusterSession?.RemoteNodeId; @@ -1734,6 +1742,7 @@ private static void WriteClientInfo(IClusterProvider provider, StringBuilder int } } + into.Append($" db={db}"); into.Append($" resp={resp}"); into.Append($" lib-name={targetSession.clientLibName}"); into.Append($" lib-ver={targetSession.clientLibVersion}"); diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index d5b135bce0..a3c0295aa7 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -150,6 +150,7 @@ static partial class CmdStrings public static ReadOnlySpan ERROR => "ERROR"u8; public static ReadOnlySpan INCRBY => "INCRBY"u8; public static ReadOnlySpan NOGET => "NOGET"u8; + public static ReadOnlySpan SCHEDULE => "SCHEDULE"u8; /// /// Response strings @@ -218,6 +219,8 @@ static partial class CmdStrings public static ReadOnlySpan RESP_ERR_PROTOCOL_VALUE_IS_NOT_INTEGER => "ERR Protocol version is not an integer or out of range."u8; public static ReadOnlySpan RESP_ERR_GENERIC_INDEX_OUT_RANGE => "ERR index out of range"u8; public static ReadOnlySpan RESP_ERR_GENERIC_SELECT_INVALID_INDEX => "ERR invalid database index."u8; + public static ReadOnlySpan RESP_ERR_DB_INDEX_OUT_OF_RANGE => "ERR DB index is out of range."u8; + public static ReadOnlySpan RESP_ERR_DBSWAP_UNSUPPORTED => "ERR DBSWAP is currently unsupported when multiple clients are connected."u8; public static ReadOnlySpan RESP_ERR_GENERIC_SELECT_CLUSTER_MODE => "ERR SELECT is not allowed in cluster mode"u8; public static ReadOnlySpan RESP_ERR_NO_TRANSACTION_PROCEDURE => "ERR Could not get transaction procedure"u8; public static ReadOnlySpan RESP_ERR_WRONG_NUMBER_OF_ARGUMENTS => "ERR wrong number of arguments for command"u8; diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs new file mode 100644 index 0000000000..b2a83453c0 --- /dev/null +++ b/libs/server/Resp/GarnetDatabaseSession.cs @@ -0,0 +1,68 @@ +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext>, + GenericAllocator>>>>; + using LockableGarnetApi = GarnetApi, + SpanByteAllocator>>, + LockableContext>, + GenericAllocator>>>>; + + internal struct GarnetDatabaseSession : IDefaultChecker, IDisposable + { + /// + /// Database ID + /// + public int Id; + + /// + /// Storage session + /// + public StorageSession StorageSession; + + /// + /// Garnet API + /// + public BasicGarnetApi GarnetApi; + + /// + /// Lockable Garnet API + /// + public LockableGarnetApi LockableGarnetApi; + + bool disposed = false; + + public GarnetDatabaseSession(int id, StorageSession storageSession, BasicGarnetApi garnetApi, LockableGarnetApi lockableGarnetApi) + { + this.Id = id; + this.StorageSession = storageSession; + this.GarnetApi = garnetApi; + this.LockableGarnetApi = lockableGarnetApi; + } + + public void Dispose() + { + if (disposed) + return; + + StorageSession?.Dispose(); + + disposed = true; + } + + public bool IsDefault() => StorageSession == null; + + /// + /// Instance of empty database session + /// + internal static GarnetDatabaseSession Empty; + } +} \ No newline at end of file diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 1c74fd8ac1..e090cfbe96 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -183,6 +183,7 @@ public enum RespCommand : ushort SPOP, SREM, SUNIONSTORE, + SWAPDB, UNLINK, ZADD, ZDIFFSTORE, @@ -387,6 +388,7 @@ public static class RespCommandExtensions RespCommand.ASYNC, RespCommand.PING, RespCommand.SELECT, + RespCommand.SWAPDB, RespCommand.ECHO, RespCommand.MONITOR, RespCommand.MODULE_LOADCS, @@ -1271,6 +1273,10 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan } } } + else if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("SWAPDB\r\n"u8)) + { + return RespCommand.SWAPDB; + } break; case 'U': @@ -2575,7 +2581,7 @@ private RespCommand ParseCommand(bool writeErrorOnFailure, out bool success) } endReadHead = (int)(ptr - recvBufferPtr); - if (storeWrapper.appendOnlyFile != null && storeWrapper.serverOptions.WaitForCommit) + if (storeWrapper.serverOptions.EnableAOF && storeWrapper.serverOptions.WaitForCommit) HandleAofCommitMode(cmd); return cmd; diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index b401f75282..19c160e729 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -91,12 +91,19 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase bool toDispose; int opCount; - public readonly StorageSession storageSession; + public StorageSession storageSession; internal BasicGarnetApi basicGarnetApi; internal LockableGarnetApi lockableGarnetApi; readonly IGarnetAuthenticator _authenticator; + internal int activeDbId; + + readonly bool allowMultiDb; + internal ExpandableMap databaseSessions; + + GarnetDatabaseSession activeDatabaseSession; + /// /// The user currently authenticated in this session /// @@ -213,12 +220,6 @@ public RespServerSession( // Initialize session-local scratch buffer of size 64 bytes, used for constructing arguments in GarnetApi this.scratchBufferManager = new ScratchBufferManager(); - // Create storage session and API - this.storageSession = new StorageSession(storeWrapper, scratchBufferManager, sessionMetrics, LatencyMetrics, logger); - - this.basicGarnetApi = new BasicGarnetApi(storageSession, storageSession.basicContext, storageSession.objectStoreBasicContext); - this.lockableGarnetApi = new LockableGarnetApi(storageSession, storageSession.lockableContext, storageSession.objectStoreLockableContext); - this.storeWrapper = storeWrapper; this.subscribeBroker = subscribeBroker; this._authenticator = authenticator ?? storeWrapper.serverOptions.AuthSettings?.CreateAuthenticator(this.storeWrapper) ?? new GarnetNoAuthAuthenticator(); @@ -226,6 +227,17 @@ public RespServerSession( if (storeWrapper.serverOptions.EnableLua && enableScripts) sessionScriptCache = new(storeWrapper, _authenticator, storeWrapper.luaTimeoutManager, logger); + var dbSession = CreateDatabaseSession(0); + var maxDbs = storeWrapper.serverOptions.MaxDatabases; + activeDbId = 0; + allowMultiDb = storeWrapper.serverOptions.AllowMultiDb; + + databaseSessions = new ExpandableMap(1, 0, maxDbs - 1); + if (!databaseSessions.TrySetValue(0, ref dbSession)) + throw new GarnetException("Failed to set initial database session in database sessions map"); + + SwitchActiveDatabaseSession(0, ref dbSession); + // Associate new session with default user and automatically authenticate, if possible this.AuthenticateUser(Encoding.ASCII.GetBytes(this.storeWrapper.accessControlList.GetDefaultUserHandle().User.Name)); @@ -251,6 +263,14 @@ public RespServerSession( } } + private GarnetDatabaseSession CreateDatabaseSession(int dbId) + { + var dbStorageSession = new StorageSession(storeWrapper, scratchBufferManager, sessionMetrics, LatencyMetrics, logger, dbId); + var dbGarnetApi = new BasicGarnetApi(dbStorageSession, dbStorageSession.basicContext, dbStorageSession.objectStoreBasicContext); + var dbLockableGarnetApi = new LockableGarnetApi(dbStorageSession, dbStorageSession.lockableContext, dbStorageSession.objectStoreLockableContext); + return new GarnetDatabaseSession(dbId, dbStorageSession, dbGarnetApi, dbLockableGarnetApi); + } + internal void SetUserHandle(UserHandle userHandle) { this._userHandle = userHandle; @@ -266,6 +286,10 @@ public override void Dispose() try { if (recvHandle.IsAllocated) recvHandle.Free(); } catch { } } + // Dispose all database sessions + foreach (var dbSession in databaseSessions.Map) + dbSession.Dispose(); + if (storeWrapper.serverOptions.MetricsSamplingFrequency > 0 || storeWrapper.serverOptions.LatencyMonitor) storeWrapper.monitor.AddMetricsHistorySessionDispose(sessionMetrics, latencyMetrics); @@ -671,6 +695,7 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st RespCommand.MSETNX => NetworkMSETNX(ref storageApi), RespCommand.UNLINK => NetworkDEL(ref storageApi), RespCommand.SELECT => NetworkSELECT(), + RespCommand.SWAPDB => NetworkSWAPDB(), RespCommand.WATCH => NetworkWATCH(), RespCommand.WATCHMS => NetworkWATCH_MS(), RespCommand.WATCHOS => NetworkWATCH_OS(), @@ -1208,7 +1233,7 @@ private void Send(byte* d) // Debug.WriteLine("SEND: [" + Encoding.UTF8.GetString(new Span(d, (int)(dcurr - d))).Replace("\n", "|").Replace("\r", "!") + "]"); if (waitForAofBlocking) { - var task = storeWrapper.appendOnlyFile.WaitForCommitAsync(); + var task = storeWrapper.WaitForCommitAsync(); if (!task.IsCompleted) task.AsTask().GetAwaiter().GetResult(); } int sendBytes = (int)(dcurr - d); @@ -1226,9 +1251,9 @@ private void DebugSend(byte* d) if ((int)(dcurr - d) > 0) { - if (storeWrapper.appendOnlyFile != null && storeWrapper.serverOptions.WaitForCommit) + if (storeWrapper.serverOptions.EnableAOF && storeWrapper.serverOptions.WaitForCommit) { - var task = storeWrapper.appendOnlyFile.WaitForCommitAsync(); + var task = storeWrapper.WaitForCommitAsync(); if (!task.IsCompleted) task.AsTask().GetAwaiter().GetResult(); } int sendBytes = (int)(dcurr - d); @@ -1279,5 +1304,89 @@ private unsafe ObjectOutputHeader ProcessOutputWithHeader(SpanByteAndMemory outp return header; } + + internal bool TrySwitchActiveDatabaseSession(int dbId) + { + if (!allowMultiDb) return false; + + ref var dbSession = ref TryGetOrSetDatabaseSession(dbId, out var success); + if (!success) return false; + + SwitchActiveDatabaseSession(dbId, ref dbSession); + return true; + } + + internal bool TrySwapDatabaseSessions(int dbId1, int dbId2) + { + if (!allowMultiDb) return false; + if (dbId1 == dbId2) return true; + + ref var dbSession1 = ref TryGetOrSetDatabaseSession(dbId1, out var success, dbId2); + if (!success) return false; + ref var dbSession2 = ref TryGetOrSetDatabaseSession(dbId2, out success, dbId1); + if (!success) return false; + + var tmp = dbSession1; + databaseSessions.Map[dbId1] = dbSession2; + databaseSessions.Map[dbId2] = tmp; + + databaseSessions.Map[dbId1].Id = dbId1; + databaseSessions.Map[dbId2].Id = dbId2; + + if (activeDbId == dbId1) + SwitchActiveDatabaseSession(dbId1, ref databaseSessions.Map[dbId1]); + else if (activeDbId == dbId2) + SwitchActiveDatabaseSession(dbId2, ref databaseSessions.Map[dbId2]); + + return true; + } + + private ref GarnetDatabaseSession TryGetOrSetDatabaseSession(int dbId, out bool success, int dbIdForSessionCreation = -1) + { + success = false; + if (dbIdForSessionCreation == -1) + dbIdForSessionCreation = dbId; + + var databaseSessionsMapSize = databaseSessions.ActualSize; + var databaseSessionsMapSnapshot = databaseSessions.Map; + + if (dbId >= 0 && dbId < databaseSessionsMapSize && !databaseSessionsMapSnapshot[dbId].IsDefault()) + { + success = true; + return ref databaseSessionsMapSnapshot[dbId]; + } + + databaseSessions.mapLock.WriteLock(); + + try + { + if (dbId >= 0 && dbId < databaseSessionsMapSize && !databaseSessionsMapSnapshot[dbId].IsDefault()) + { + success = true; + return ref databaseSessionsMapSnapshot[dbId]; + } + + var dbSession = CreateDatabaseSession(dbIdForSessionCreation); + if (!databaseSessions.TrySetValueUnsafe(dbId, ref dbSession, false)) + return ref GarnetDatabaseSession.Empty; + + success = true; + databaseSessionsMapSnapshot = databaseSessions.Map; + return ref databaseSessionsMapSnapshot[dbId]; + } + finally + { + databaseSessions.mapLock.WriteUnlock(); + } + } + + private void SwitchActiveDatabaseSession(int dbId, ref GarnetDatabaseSession dbSession) + { + this.activeDbId = dbId; + this.activeDatabaseSession = dbSession; + this.storageSession = dbSession.StorageSession; + this.basicGarnetApi = dbSession.GarnetApi; + this.lockableGarnetApi = dbSession.LockableGarnetApi; + } } } \ No newline at end of file diff --git a/libs/server/ServerConfig.cs b/libs/server/ServerConfig.cs index 634114bb31..25a1e77f5d 100644 --- a/libs/server/ServerConfig.cs +++ b/libs/server/ServerConfig.cs @@ -89,7 +89,7 @@ private bool NetworkCONFIG_GET() ReadOnlySpan GetDatabases() { - var databases = storeWrapper.databaseNum.ToString(); + var databases = storeWrapper.serverOptions.MaxDatabases.ToString(); return Encoding.ASCII.GetBytes($"$9\r\ndatabases\r\n${databases.Length}\r\n{databases}\r\n"); } diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 399578e82d..9ac51eb926 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -446,6 +446,74 @@ public class GarnetServerOptions : ServerOptions /// public UnixFileMode UnixSocketPermission { get; set; } + /// + /// Max number of logical databases allowed + /// + public int MaxDatabases = 16; + + /// + /// Allow more than one logical database in server + /// + public bool AllowMultiDb => !EnableCluster && MaxDatabases > 1; + + /// + /// Gets the base directory for storing checkpoints + /// + public string CheckpointBaseDirectory => (CheckpointDir ?? LogDir) ?? string.Empty; + + /// + /// Gets the base directory for storing main-store checkpoints + /// + public string MainStoreCheckpointBaseDirectory => Path.Combine(CheckpointBaseDirectory, "Store"); + + /// + /// Gets the base directory for storing object-store checkpoints + /// + public string ObjectStoreCheckpointBaseDirectory => Path.Combine(CheckpointBaseDirectory, "ObjectStore"); + + /// + /// Get the directory name for database checkpoints + /// + /// Database Id + /// Directory name + public string GetCheckpointDirectoryName(int dbId) => $"checkpoints{(dbId == 0 ? string.Empty : $"_{dbId}")}"; + + /// + /// Get the directory for main-store database checkpoints + /// + /// Database Id + /// Directory + public string GetMainStoreCheckpointDirectory(int dbId) => + Path.Combine(MainStoreCheckpointBaseDirectory, GetCheckpointDirectoryName(dbId)); + + /// + /// Get the directory for object-store database checkpoints + /// + /// Database Id + /// Directory + public string GetObjectStoreCheckpointDirectory(int dbId) => + Path.Combine(ObjectStoreCheckpointBaseDirectory, GetCheckpointDirectoryName(dbId)); + + /// + /// Gets the base directory for storing AOF commits + /// + public string AppendOnlyFileBaseDirectory => CheckpointDir ?? string.Empty; + + /// + /// Get the directory name for database AOF commits + /// + /// Database Id + /// Directory name + public string GetAppendOnlyFileDirectoryName(int dbId) => $"AOF{(dbId == 0 ? string.Empty : $"_{dbId}")}"; + + /// + /// Get the directory for database AOF commits + /// + /// Database Id + /// Directory + public string GetAppendOnlyFileDirectory(int dbId) => + Path.Combine(AppendOnlyFileBaseDirectory, GetAppendOnlyFileDirectoryName(dbId)); + /// /// Constructor /// @@ -730,14 +798,15 @@ public KVSettings GetObjectStoreSettings(ILogger logger, /// /// Get AOF settings /// - /// - public void GetAofSettings(out TsavoriteLogSettings tsavoriteLogSettings) + /// DB ID + /// Tsavorite log settings + public void GetAofSettings(int dbId, out TsavoriteLogSettings tsavoriteLogSettings) { tsavoriteLogSettings = new TsavoriteLogSettings { MemorySizeBits = AofMemorySizeBits(), PageSizeBits = AofPageSizeBits(), - LogDevice = GetAofDevice(), + LogDevice = GetAofDevice(dbId), TryRecoverLatest = false, SafeTailRefreshFrequencyMs = EnableCluster ? AofReplicationRefreshFrequencyMs : -1, FastCommitMode = EnableFastCommit, @@ -749,9 +818,11 @@ public void GetAofSettings(out TsavoriteLogSettings tsavoriteLogSettings) logger?.LogError("AOF Page size cannot be more than the AOF memory size."); throw new Exception("AOF Page size cannot be more than the AOF memory size."); } + + var aofDir = GetAppendOnlyFileDirectory(dbId); tsavoriteLogSettings.LogCommitManager = new DeviceLogCommitCheckpointManager( FastAofTruncate ? new NullNamedDeviceFactoryCreator() : DeviceFactoryCreator, - new DefaultCheckpointNamingScheme(CheckpointDir + "/AOF"), + new DefaultCheckpointNamingScheme(aofDir), removeOutdated: true, fastCommitThrottleFreq: EnableFastCommit ? FastCommitThrottleFreq : 0); } @@ -835,12 +906,14 @@ public int ObjectStoreSegmentSizeBits() /// Get device for AOF /// /// - IDevice GetAofDevice() + IDevice GetAofDevice(int dbId) { if (UseAofNullDevice && EnableCluster && !FastAofTruncate) throw new Exception("Cannot use null device for AOF when cluster is enabled and you are not using main memory replication"); if (UseAofNullDevice) return new NullDevice(); - else return GetInitializedDeviceFactory(CheckpointDir).Get(new FileDescriptor("AOF", "aof.log")); + + return GetInitializedDeviceFactory(AppendOnlyFileBaseDirectory) + .Get(new FileDescriptor(GetAppendOnlyFileDirectoryName(dbId), "aof.log")); } } } \ No newline at end of file diff --git a/libs/server/Servers/StoreApi.cs b/libs/server/Servers/StoreApi.cs index f9d67cc366..c0cbc3076e 100644 --- a/libs/server/Servers/StoreApi.cs +++ b/libs/server/Servers/StoreApi.cs @@ -26,32 +26,29 @@ public StoreApi(StoreWrapper storeWrapper) /// Commit AOF /// /// - public void CommitAOF(bool spinWait = false) => storeWrapper.appendOnlyFile?.Commit(spinWait); + public void CommitAOF(bool spinWait = false) => storeWrapper.CommitAOF(spinWait); /// /// Wait for commit /// - public ValueTask WaitForCommitAsync(CancellationToken token = default) => storeWrapper.appendOnlyFile != null ? storeWrapper.appendOnlyFile.WaitForCommitAsync(token: token) : ValueTask.CompletedTask; + public ValueTask WaitForCommitAsync(CancellationToken token = default) => storeWrapper.WaitForCommitAsync(token: token); /// /// Wait for commit /// - public void WaitForCommit() => storeWrapper.appendOnlyFile?.WaitForCommit(); + public void WaitForCommit() => storeWrapper.WaitForCommit(); /// /// Commit AOF /// - public ValueTask CommitAOFAsync(CancellationToken token) => storeWrapper.appendOnlyFile != null ? storeWrapper.appendOnlyFile.CommitAsync(null, token) : ValueTask.CompletedTask; + public ValueTask CommitAOFAsync(CancellationToken token) => storeWrapper.CommitAOFAsync(token: token); /// /// Flush DB (delete all keys) /// Optionally truncate log on disk. This is a destructive operation. Instead take a checkpoint if you are using checkpointing, as /// that will safely truncate the log on disk after the checkpoint. /// - public void FlushDB(bool unsafeTruncateLog = false) - { - storeWrapper.store.Log.ShiftBeginAddress(storeWrapper.store.Log.TailAddress, truncateLog: unsafeTruncateLog); - storeWrapper.objectStore?.Log.ShiftBeginAddress(storeWrapper.objectStore.Log.TailAddress, truncateLog: unsafeTruncateLog); - } + public void FlushDB(int dbId = 0, bool unsafeTruncateLog = false) => + storeWrapper.FlushDatabase(unsafeTruncateLog, dbId); } } \ No newline at end of file diff --git a/libs/server/Storage/Session/StorageSession.cs b/libs/server/Storage/Session/StorageSession.cs index 989dd89536..86c2437c14 100644 --- a/libs/server/Storage/Session/StorageSession.cs +++ b/libs/server/Storage/Session/StorageSession.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Diagnostics; using Microsoft.Extensions.Logging; using Tsavorite.core; @@ -57,7 +58,8 @@ public StorageSession(StoreWrapper storeWrapper, ScratchBufferManager scratchBufferManager, GarnetSessionMetrics sessionMetrics, GarnetLatencyMetricsSession LatencyMetrics, - ILogger logger = null) + ILogger logger = null, + int dbId = 0) { this.sessionMetrics = sessionMetrics; this.LatencyMetrics = LatencyMetrics; @@ -67,13 +69,17 @@ public StorageSession(StoreWrapper storeWrapper, parseState.Initialize(); - functionsState = storeWrapper.CreateFunctionsState(); + functionsState = storeWrapper.CreateFunctionsState(dbId); var functions = new MainSessionFunctions(functionsState); - var session = storeWrapper.store.NewSession(functions); - var objstorefunctions = new ObjectSessionFunctions(functionsState); - var objectStoreSession = storeWrapper.objectStore?.NewSession(objstorefunctions); + var dbFound = storeWrapper.TryGetDatabase(dbId, out var db); + Debug.Assert(dbFound); + + var session = db.MainStore.NewSession(functions); + + var objectStoreFunctions = new ObjectSessionFunctions(functionsState); + var objectStoreSession = db.ObjectStore?.NewSession(objectStoreFunctions); basicContext = session.BasicContext; lockableContext = session.LockableContext; @@ -83,7 +89,7 @@ public StorageSession(StoreWrapper storeWrapper, objectStoreLockableContext = objectStoreSession.LockableContext; } - HeadAddress = storeWrapper.store.Log.HeadAddress; + HeadAddress = db.MainStore.Log.HeadAddress; ObjectScanCountLimit = storeWrapper.serverOptions.ObjectScanCountLimit; } diff --git a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs index 0da622e776..fe226a500f 100644 --- a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs +++ b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs @@ -25,8 +25,9 @@ public class CacheSizeTracker public long TargetSize; public long ReadCacheTargetSize; + int isStarted = 0; private const int deltaFraction = 10; // 10% of target size - private TsavoriteKV store; + TsavoriteKV store; internal bool Stopped => (mainLogTracker == null || mainLogTracker.Stopped) && (readCacheTracker == null || readCacheTracker.Stopped); @@ -89,6 +90,10 @@ public CacheSizeTracker(TsavoriteKV + /// If tracker has not started, prevent it from starting + /// + /// True if tracker hasn't previously started + public bool TryPreventStart() + { + var prevStarted = Interlocked.CompareExchange(ref isStarted, 1, 0); + return prevStarted == 0; + } } } \ No newline at end of file diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 0def6ff075..13463de779 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -35,61 +35,63 @@ public sealed class StoreWrapper internal readonly long startupTime; /// - /// Store + /// Default database (DB 0) /// - public readonly TsavoriteKV store; + public ref GarnetDatabase DefaultDatabase => ref databaseManager.DefaultDatabase; /// - /// Object store + /// Store (of DB 0) /// - public readonly TsavoriteKV objectStore; + public TsavoriteKV store => databaseManager.MainStore; /// - /// Server options + /// Object store (of DB 0) /// - public readonly GarnetServerOptions serverOptions; + public TsavoriteKV objectStore => databaseManager.ObjectStore; /// - /// Subscribe broker + /// AOF (of DB 0) /// - public readonly SubscribeBroker subscribeBroker; + public TsavoriteLog appendOnlyFile => databaseManager.AppendOnlyFile; - internal readonly IClusterProvider clusterProvider; + /// + /// Last save time (of DB 0) + /// + public DateTimeOffset lastSaveTime => databaseManager.LastSaveTime; /// - /// Get server + /// Object store size tracker (of DB 0) /// - public GarnetServerTcp TcpServer => (GarnetServerTcp)server; + public CacheSizeTracker objectStoreSizeTracker => databaseManager.ObjectStoreSizeTracker; /// - /// Access control list governing all commands + /// Server options /// - public readonly AccessControlList accessControlList; + public readonly GarnetServerOptions serverOptions; /// - /// AOF + /// Subscribe broker + /// + public readonly SubscribeBroker subscribeBroker; + + /// + /// Get server /// - public readonly TsavoriteLog appendOnlyFile; + public GarnetServerTcp TcpServer => server as GarnetServerTcp; /// - /// Last save time + /// Access control list governing all commands /// - public DateTimeOffset lastSaveTime; - internal long lastSaveStoreTailAddress; - internal long lastSaveObjectStoreTailAddress; + public readonly AccessControlList accessControlList; /// /// Logger factory /// public readonly ILoggerFactory loggerFactory; - internal readonly CollectionItemBroker itemBroker; - internal readonly CustomCommandManager customCommandManager; - internal readonly GarnetServerMonitor monitor; - internal readonly WatchVersionMap versionMap; - - internal readonly CacheSizeTracker objectStoreSizeTracker; - + /// + /// Object serializer + /// public readonly GarnetObjectSerializer GarnetObjectSerializer; /// @@ -97,32 +99,49 @@ public sealed class StoreWrapper /// public readonly ILogger logger; - internal readonly ILogger sessionLogger; - readonly CancellationTokenSource ctsCommit; + /// + /// Lua script cache + /// + public readonly ConcurrentDictionary storeScriptCache; - internal long SafeAofAddress = -1; + /// + /// Logging frequency + /// + public readonly TimeSpan loggingFrequency; - // Standalone instance node_id - internal readonly string run_id; - private SingleWriterMultiReaderLock _checkpointTaskLock; - internal readonly SlowLogContainer slowLogContainer; + /// + /// Definition for delegate creating a new logical database + /// + public delegate GarnetDatabase DatabaseCreatorDelegate(int dbId); /// - /// Lua script cache + /// Number of active databases /// - public readonly ConcurrentDictionary storeScriptCache; + public int DatabaseCount => databaseManager.DatabaseCount; /// /// Shared timeout manager for all across all sessions. /// internal readonly LuaTimeoutManager luaTimeoutManager; - public readonly TimeSpan loggingFrequncy; + private IDatabaseManager databaseManager; + SingleWriterMultiReaderLock databaseManagerLock; - /// - /// NOTE: For now we support only a single database - /// - public readonly int databaseNum = 1; + internal readonly CollectionItemBroker itemBroker; + internal readonly CustomCommandManager customCommandManager; + internal readonly GarnetServerMonitor monitor; + internal readonly IClusterProvider clusterProvider; + internal readonly SlowLogContainer slowLogContainer; + internal readonly ILogger sessionLogger; + internal long safeAofAddress = -1; + + // Standalone instance node_id + internal readonly string runId; + + readonly CancellationTokenSource ctsCommit; + + // True if StoreWrapper instance is disposed + bool disposed; /// /// Constructor @@ -131,42 +150,39 @@ public StoreWrapper( string version, string redisProtocolVersion, IGarnetServer server, - TsavoriteKV store, - TsavoriteKV objectStore, - CacheSizeTracker objectStoreSizeTracker, CustomCommandManager customCommandManager, - TsavoriteLog appendOnlyFile, GarnetServerOptions serverOptions, SubscribeBroker subscribeBroker, AccessControlList accessControlList = null, + DatabaseCreatorDelegate createDatabaseDelegate = null, + IDatabaseManager databaseManager = null, IClusterFactory clusterFactory = null, - ILoggerFactory loggerFactory = null - ) + ILoggerFactory loggerFactory = null) { this.version = version; this.redisProtocolVersion = redisProtocolVersion; this.server = server; this.startupTime = DateTimeOffset.UtcNow.Ticks; - this.store = store; - this.objectStore = objectStore; - this.appendOnlyFile = appendOnlyFile; this.serverOptions = serverOptions; this.subscribeBroker = subscribeBroker; - lastSaveTime = DateTimeOffset.FromUnixTimeSeconds(0); this.customCommandManager = customCommandManager; - this.monitor = serverOptions.MetricsSamplingFrequency > 0 ? new GarnetServerMonitor(this, serverOptions, server, loggerFactory?.CreateLogger("GarnetServerMonitor")) : null; - this.objectStoreSizeTracker = objectStoreSizeTracker; + this.databaseManager = databaseManager ?? DatabaseManagerFactory.CreateDatabaseManager(serverOptions, createDatabaseDelegate, this); + this.monitor = serverOptions.MetricsSamplingFrequency > 0 + ? new GarnetServerMonitor(this, serverOptions, server, + loggerFactory?.CreateLogger("GarnetServerMonitor")) + : null; this.loggerFactory = loggerFactory; this.logger = loggerFactory?.CreateLogger("StoreWrapper"); this.sessionLogger = loggerFactory?.CreateLogger("Session"); - // TODO Change map size to a reasonable number - this.versionMap = new WatchVersionMap(1 << 16); this.accessControlList = accessControlList; this.GarnetObjectSerializer = new GarnetObjectSerializer(this.customCommandManager); - this.loggingFrequncy = TimeSpan.FromSeconds(serverOptions.LoggingFrequency); + this.loggingFrequency = TimeSpan.FromSeconds(serverOptions.LoggingFrequency); + + // If cluster mode is off and more than one database allowed multi-db mode is turned on if (serverOptions.SlowLogThreshold > 0) this.slowLogContainer = new SlowLogContainer(serverOptions.SlowLogMaxEntries); + if (!serverOptions.DisableObjects) this.itemBroker = new CollectionItemBroker(); @@ -186,15 +202,19 @@ public StoreWrapper( // If ACL authentication is enabled, initiate access control list // NOTE: This is a temporary workflow. ACL should always be initiated and authenticator // should become a parameter of AccessControlList. - if ((this.serverOptions.AuthSettings != null) && (this.serverOptions.AuthSettings.GetType().BaseType == typeof(AclAuthenticationSettings))) + if ((this.serverOptions.AuthSettings != null) && (this.serverOptions.AuthSettings.GetType().BaseType == + typeof(AclAuthenticationSettings))) { // Create a new access control list and register it with the authentication settings - AclAuthenticationSettings aclAuthenticationSettings = (AclAuthenticationSettings)this.serverOptions.AuthSettings; + var aclAuthenticationSettings = + (AclAuthenticationSettings)this.serverOptions.AuthSettings; if (!string.IsNullOrEmpty(aclAuthenticationSettings.AclConfigurationFile)) { - logger?.LogInformation("Reading ACL configuration file '{filepath}'", aclAuthenticationSettings.AclConfigurationFile); - this.accessControlList = new AccessControlList(aclAuthenticationSettings.DefaultPassword, aclAuthenticationSettings.AclConfigurationFile); + logger?.LogInformation("Reading ACL configuration file '{filepath}'", + aclAuthenticationSettings.AclConfigurationFile); + this.accessControlList = new AccessControlList(aclAuthenticationSettings.DefaultPassword, + aclAuthenticationSettings.AclConfigurationFile); } else { @@ -211,7 +231,25 @@ public StoreWrapper( if (clusterFactory != null) clusterProvider = clusterFactory.CreateClusterProvider(this); ctsCommit = new(); - run_id = Generator.CreateHexId(); + runId = Generator.CreateHexId(); + } + + /// + /// Copy Constructor + /// + /// Source instance + /// Enable AOF in database manager + public StoreWrapper(StoreWrapper storeWrapper, bool recordToAof) : this(storeWrapper.version, + storeWrapper.redisProtocolVersion, + storeWrapper.server, + storeWrapper.customCommandManager, + storeWrapper.serverOptions, + storeWrapper.subscribeBroker, + storeWrapper.accessControlList, + databaseManager: storeWrapper.databaseManager.Clone(recordToAof), + clusterFactory: null, + loggerFactory: storeWrapper.loggerFactory) + { } /// @@ -244,9 +282,6 @@ public string GetIp() return localEndpoint.Address.ToString(); } - internal FunctionsState CreateFunctionsState() - => new(appendOnlyFile, versionMap, customCommandManager, null, objectStoreSizeTracker, GarnetObjectSerializer); - internal void Recover() { if (serverOptions.EnableCluster) @@ -268,122 +303,162 @@ internal void Recover() } /// - /// Caller will have to decide if recover is necessary, so we do not check if recover option is enabled + /// Take checkpoint of all active databases /// - public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) - { - long storeVersion = -1, objectStoreVersion = -1; - try - { - if (replicaRecover) - { - if (metadata.storeIndexToken != default && metadata.storeHlogToken != default) - { - storeVersion = !recoverMainStoreFromToken ? store.Recover() : store.Recover(metadata.storeIndexToken, metadata.storeHlogToken); - } + /// True if method can return before checkpoint is taken + /// Store type to checkpoint + /// Logger + /// Cancellation token + /// False if another checkpointing process is already in progress + public bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null, + CancellationToken token = default) => databaseManager.TakeCheckpoint(background, storeType, logger, token); - if (!serverOptions.DisableObjects) - { - if (metadata.objectStoreIndexToken != default && metadata.objectStoreHlogToken != default) - { - objectStoreVersion = !recoverObjectStoreFromToken ? objectStore.Recover() : objectStore.Recover(metadata.objectStoreIndexToken, metadata.objectStoreHlogToken); - } - } - } - else - { - storeVersion = store.Recover(); - if (objectStore != null) objectStoreVersion = objectStore.Recover(); - } - if (storeVersion > 0 || objectStoreVersion > 0) - lastSaveTime = DateTimeOffset.UtcNow; - } - catch (TsavoriteNoHybridLogException ex) - { - // No hybrid log being found is not the same as an error in recovery. e.g. fresh start - logger?.LogInformation(ex, "No Hybrid Log found for recovery; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion); - } - catch (Exception ex) - { - logger?.LogInformation(ex, "Error during recovery of store; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion); - if (serverOptions.FailOnRecoveryError) - throw; - } - } + /// + /// Recover checkpoint + /// + public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, + bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) + => databaseManager.RecoverCheckpoint(replicaRecover, recoverMainStoreFromToken, recoverObjectStoreFromToken, metadata); /// /// Recover AOF /// - public void RecoverAOF() + public void RecoverAOF() => databaseManager.RecoverAOF(); + + /// + /// When replaying AOF we do not want to write AOF records again. + /// + public long ReplayAOF(long untilAddress = -1) => this.databaseManager.ReplayAOF(); + + /// + /// Append a checkpoint commit to the AOF + /// + /// + /// + /// + public void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0) + { + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.EnqueueCommit)} with DB ID: {dbId}"); + + this.databaseManager.EnqueueCommit(entryType, version, dbId); + } + + internal FunctionsState CreateFunctionsState(int dbId = 0) { - if (appendOnlyFile == null) return; - appendOnlyFile.Recover(); - logger?.LogInformation("Recovered AOF: begin address = {beginAddress}, tail address = {tailAddress}", appendOnlyFile.BeginAddress, appendOnlyFile.TailAddress); + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.CreateFunctionsState)} with DB ID: {dbId}"); + + return databaseManager.CreateFunctionsState(dbId); } /// /// Reset /// - public void Reset() + /// Database ID + public void Reset(int dbId = 0) { - try - { - if (store.Log.TailAddress > 64) - store.Reset(); - if (objectStore?.Log.TailAddress > 64) - objectStore?.Reset(); - appendOnlyFile?.Reset(); - lastSaveTime = DateTimeOffset.FromUnixTimeSeconds(0); - } - catch (Exception ex) - { - logger?.LogError(ex, "Error during reset of store"); - } + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.Reset)} with DB ID: {dbId}"); + + databaseManager.Reset(dbId); } /// - /// When replaying AOF we do not want to write AOF records again. + /// Resets the revivification stats. /// - public long ReplayAOF(long untilAddress = -1) + public void ResetRevivificationStats() => databaseManager.ResetRevivificationStats(); + + /// + /// Get a snapshot of all active databases + /// + /// Array of active databases + public GarnetDatabase[] GetDatabasesSnapshot() => databaseManager.GetDatabasesSnapshot(); + + /// + /// Get database DB ID + /// + /// DB Id + /// Retrieved database + /// True if database was found + public bool TryGetDatabase(int dbId, out GarnetDatabase database) { - if (!serverOptions.EnableAOF) - return -1; - long replicationOffset = 0; - try + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) { - // When replaying AOF we do not want to write record again to AOF. - // So initialize local AofProcessor with recordToAof: false. - var aofProcessor = new AofProcessor(this, recordToAof: false, logger); - aofProcessor.Recover(untilAddress); - aofProcessor.Dispose(); - replicationOffset = aofProcessor.ReplicationOffset; - lastSaveTime = DateTimeOffset.UtcNow; + database = GarnetDatabase.Empty; + return false; } - catch (Exception ex) + + database = databaseManager.TryGetDatabase(dbId, out var success); + return success; + } + + /// + /// Try to get or add a new database + /// + /// Database ID + /// Retrieved or added database + /// True if database was added + /// True if database was found or added + public bool TryGetOrAddDatabase(int dbId, out GarnetDatabase database, out bool added) + { + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) { - logger?.LogError(ex, "Error during recovery of AofProcessor"); - if (serverOptions.FailOnRecoveryError) - throw; + database = GarnetDatabase.Empty; + added = false; + return false; } - return replicationOffset; + + database = databaseManager.TryGetOrAddDatabase(dbId, out var success, out added); + return success; } - async Task AutoCheckpointBasedOnAofSizeLimit(long AofSizeLimit, CancellationToken token = default, ILogger logger = null) + /// + /// Flush database with specified ID + /// + /// Truncate log + /// Database ID + public void FlushDatabase(bool unsafeTruncateLog, int dbId = 0) + { + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.FlushDatabase)} with DB ID: {dbId}"); + + databaseManager.FlushDatabase(unsafeTruncateLog, dbId); + } + + /// + /// Flush all active databases + /// + /// Truncate log + public void FlushAllDatabases(bool unsafeTruncateLog) + { + databaseManager.FlushAllDatabases(unsafeTruncateLog); + } + + /// + /// Try to swap between two database instances + /// + /// First database ID + /// Second database ID + /// True if swap successful + public bool TrySwapDatabases(int dbId1, int dbId2) + { + if (databaseManager is SingleDatabaseManager) return false; + + return this.databaseManager.TrySwapDatabases(dbId1, dbId2); + } + + async Task AutoCheckpointBasedOnAofSizeLimit(long aofSizeLimit, CancellationToken token = default, ILogger logger = null) { try { while (true) { - await Task.Delay(1000); + await Task.Delay(1000, token); if (token.IsCancellationRequested) break; - var currAofSize = appendOnlyFile.TailAddress - appendOnlyFile.BeginAddress; - if (currAofSize > AofSizeLimit) - { - logger?.LogInformation("Enforcing AOF size limit currentAofSize: {currAofSize} > AofSizeLimit: {AofSizeLimit}", currAofSize, AofSizeLimit); - TakeCheckpoint(false, logger: logger); - } + await databaseManager.TaskCheckpointBasedOnAofSizeLimitAsync(aofSizeLimit, token, logger); } } catch (Exception ex) @@ -407,14 +482,15 @@ async Task CommitTask(int commitFrequencyMs, ILogger logger = null, Cancellation } else { - await appendOnlyFile.CommitAsync(null, token); + await databaseManager.CommitToAofAsync(token, logger); + await Task.Delay(commitFrequencyMs, token); } } } catch (Exception ex) { - logger?.LogError(ex, "CommitTask exception received, AOF tail address = {tailAddress}; AOF committed until address = {commitAddress}; ", appendOnlyFile.TailAddress, appendOnlyFile.CommittedUntilAddress); + logger?.LogError(ex, "CommitTask exception received."); } } @@ -426,7 +502,9 @@ async Task CompactionTask(int compactionFrequencySecs, CancellationToken token = while (true) { if (token.IsCancellationRequested) return; - DoCompaction(serverOptions.CompactionMaxSegments, serverOptions.ObjectStoreCompactionMaxSegments, 1, serverOptions.CompactionType, serverOptions.CompactionForceDelete); + + databaseManager.DoCompaction(token, logger); + if (!serverOptions.CompactionForceDelete) logger?.LogInformation("NOTE: Take a checkpoint (SAVE/BGSAVE) in order to actually delete the older data segments (files) from disk"); else @@ -437,7 +515,7 @@ async Task CompactionTask(int compactionFrequencySecs, CancellationToken token = } catch (Exception ex) { - logger?.LogError(ex, "CompactionTask exception received, AOF tail address = {tailAddress}; AOF committed until address = {commitAddress}; ", appendOnlyFile.TailAddress, appendOnlyFile.CommittedUntilAddress); + logger?.LogError(ex, "CompactionTask exception received."); } } @@ -449,7 +527,7 @@ async Task HashCollectTask(int hashCollectFrequencySecs, CancellationToken token var scratchBufferManager = new ScratchBufferManager(); using var storageSession = new StorageSession(this, scratchBufferManager, null, null, logger); - if (objectStore is null) + if (serverOptions.DisableObjects) { logger?.LogWarning("HashCollectFrequencySecs option is configured but Object store is disabled. Stopping the background hash collect task."); return; @@ -484,137 +562,81 @@ static void ExecuteHashCollect(ScratchBufferManager scratchBufferManager, Storag } } - void DoCompaction() + /// + /// Commit AOF for all active databases + /// + /// True if should wait until all commits complete + internal void CommitAOF(bool spinWait) { - // Periodic compaction -> no need to compact before checkpointing - if (serverOptions.CompactionFrequencySecs > 0) return; + if (!serverOptions.EnableAOF) return; + + var task = databaseManager.CommitToAofAsync(); + if (!spinWait) return; - DoCompaction(serverOptions.CompactionMaxSegments, serverOptions.ObjectStoreCompactionMaxSegments, 1, serverOptions.CompactionType, serverOptions.CompactionForceDelete); + task.GetAwaiter().GetResult(); } /// - /// Append a checkpoint commit to the AOF + /// Wait for commits from all active databases /// - /// - /// - public void EnqueueCommit(AofEntryType entryType, long version) + internal void WaitForCommit() => + WaitForCommitAsync().GetAwaiter().GetResult(); + + /// + /// Asynchronously wait for commits from all active databases + /// + /// Cancellation token + /// ValueTask + internal async ValueTask WaitForCommitAsync(CancellationToken token = default) { - AofHeader header = new() - { - opType = entryType, - storeVersion = version, - sessionID = -1 - }; - appendOnlyFile?.Enqueue(header, out _); + if (!serverOptions.EnableAOF) return; + + await databaseManager.WaitForCommitToAofAsync(token); } - void DoCompaction(int mainStoreMaxSegments, int objectStoreMaxSegments, int numSegmentsToCompact, LogCompactionType compactionType, bool compactionForceDelete) + /// + /// Asynchronously wait for AOF commits on all active databases, + /// unless specific database ID specified (by default: -1 = all) + /// + /// Specific database ID to commit AOF for (optional) + /// Cancellation token + /// ValueTask + internal async ValueTask CommitAOFAsync(int dbId = -1, CancellationToken token = default) { - if (compactionType == LogCompactionType.None) return; - - long mainStoreMaxLogSize = (1L << serverOptions.SegmentSizeBits()) * mainStoreMaxSegments; + if (!serverOptions.EnableAOF) return; - if (store.Log.ReadOnlyAddress - store.Log.BeginAddress > mainStoreMaxLogSize) + if (dbId == -1) { - long readOnlyAddress = store.Log.ReadOnlyAddress; - long compactLength = (1L << serverOptions.SegmentSizeBits()) * (mainStoreMaxSegments - numSegmentsToCompact); - long untilAddress = readOnlyAddress - compactLength; - logger?.LogInformation("Begin main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", untilAddress, store.Log.BeginAddress, readOnlyAddress, store.Log.TailAddress); - - switch (compactionType) - { - case LogCompactionType.Shift: - store.Log.ShiftBeginAddress(untilAddress, true, compactionForceDelete); - break; - - case LogCompactionType.Scan: - store.Log.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Scan); - if (compactionForceDelete) - { - CompactionCommitAof(); - store.Log.Truncate(); - } - break; - - case LogCompactionType.Lookup: - store.Log.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Lookup); - if (compactionForceDelete) - { - CompactionCommitAof(); - store.Log.Truncate(); - } - break; - - default: - break; - } - - logger?.LogInformation("End main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", untilAddress, store.Log.BeginAddress, readOnlyAddress, store.Log.TailAddress); + await databaseManager.CommitToAofAsync(token, logger); + return; } - if (objectStore == null) return; - - long objectStoreMaxLogSize = (1L << serverOptions.ObjectStoreSegmentSizeBits()) * objectStoreMaxSegments; - - if (objectStore.Log.ReadOnlyAddress - objectStore.Log.BeginAddress > objectStoreMaxLogSize) - { - long readOnlyAddress = objectStore.Log.ReadOnlyAddress; - long compactLength = (1L << serverOptions.ObjectStoreSegmentSizeBits()) * (objectStoreMaxSegments - numSegmentsToCompact); - long untilAddress = readOnlyAddress - compactLength; - logger?.LogInformation("Begin object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", untilAddress, objectStore.Log.BeginAddress, readOnlyAddress, objectStore.Log.TailAddress); - - switch (compactionType) - { - case LogCompactionType.Shift: - objectStore.Log.ShiftBeginAddress(untilAddress, compactionForceDelete); - break; - - case LogCompactionType.Scan: - objectStore.Log.Compact>( - new SimpleSessionFunctions(), untilAddress, CompactionType.Scan); - if (compactionForceDelete) - { - CompactionCommitAof(); - objectStore.Log.Truncate(); - } - break; - - case LogCompactionType.Lookup: - objectStore.Log.Compact>( - new SimpleSessionFunctions(), untilAddress, CompactionType.Lookup); - if (compactionForceDelete) - { - CompactionCommitAof(); - objectStore.Log.Truncate(); - } - break; + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.CommitToAofAsync)} with DB ID: {dbId}"); - default: - break; - } - - logger?.LogInformation("End object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", untilAddress, store.Log.BeginAddress, readOnlyAddress, store.Log.TailAddress); - } + await databaseManager.CommitToAofAsync(dbId, token); } - void CompactionCommitAof() + /// + /// Take checkpoint of all active database IDs or a specified database ID + /// + /// True if method can return before checkpoint is taken + /// ID of database to checkpoint (default: -1 - checkpoint all active databases) + /// Store type to checkpoint + /// Logger + /// Cancellation token + /// False if another checkpointing process is already in progress + public bool TakeCheckpoint(bool background, int dbId = -1, StoreType storeType = StoreType.All, ILogger logger = null, CancellationToken token = default) { - // If we are the primary, we commit the AOF. - // If we are the replica, we commit the AOF only if fast commit is disabled - // because we do not want to clobber AOF addresses. - // TODO: replica should instead wait until the next AOF commit is done via primary - if (serverOptions.EnableAOF) + if (dbId == -1) { - if (serverOptions.EnableCluster && clusterProvider.IsReplica()) - { - if (!serverOptions.EnableFastCommit) - appendOnlyFile?.CommitAsync().ConfigureAwait(false).GetAwaiter().GetResult(); - } - else - { - appendOnlyFile?.CommitAsync().ConfigureAwait(false).GetAwaiter().GetResult(); - } + return databaseManager.TakeCheckpoint(background, storeType, logger, token); } + + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.TakeCheckpoint)} with DB ID: {dbId}"); + + return databaseManager.TakeCheckpoint(background, dbId, storeType, logger, token); } internal void Start() @@ -625,11 +647,11 @@ internal void Start() if (serverOptions.AofSizeLimit.Length > 0) { - var AofSizeLimitBytes = 1L << serverOptions.AofSizeLimitSizeBits(); - Task.Run(async () => await AutoCheckpointBasedOnAofSizeLimit(AofSizeLimitBytes, ctsCommit.Token, logger)); + var aofSizeLimitBytes = 1L << serverOptions.AofSizeLimitSizeBits(); + Task.Run(async () => await AutoCheckpointBasedOnAofSizeLimit(aofSizeLimitBytes, ctsCommit.Token, logger)); } - if (serverOptions.CommitFrequencyMs > 0 && appendOnlyFile != null) + if (serverOptions.CommitFrequencyMs > 0 && serverOptions.EnableAOF) { Task.Run(async () => await CommitTask(serverOptions.CommitFrequencyMs, logger, ctsCommit.Token)); } @@ -649,7 +671,7 @@ internal void Start() Task.Run(() => IndexAutoGrowTask(ctsCommit.Token)); } - objectStoreSizeTracker?.Start(ctsCommit.Token); + databaseManager.StartObjectSizeTrackers(ctsCommit.Token); } /// Grows indexes of both main store and object store if current size is too small. @@ -658,21 +680,15 @@ private async void IndexAutoGrowTask(CancellationToken token) { try { - bool indexMaxedOut = serverOptions.AdjustedIndexMaxCacheLines == 0; - bool objectStoreIndexMaxedOut = serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0; - while (!indexMaxedOut || !objectStoreIndexMaxedOut) + var allIndexesMaxedOut = false; + + while (!allIndexesMaxedOut) { if (token.IsCancellationRequested) break; await Task.Delay(TimeSpan.FromSeconds(serverOptions.IndexResizeFrequencySecs), token); - if (!indexMaxedOut) - indexMaxedOut = GrowIndexIfNeeded(StoreType.Main, serverOptions.AdjustedIndexMaxCacheLines, store.OverflowBucketAllocations, - () => store.IndexSize, async () => await store.GrowIndexAsync()); - - if (!objectStoreIndexMaxedOut) - objectStoreIndexMaxedOut = GrowIndexIfNeeded(StoreType.Object, serverOptions.AdjustedObjectStoreIndexMaxCacheLines, objectStore.OverflowBucketAllocations, - () => objectStore.IndexSize, async () => await objectStore.GrowIndexAsync()); + allIndexesMaxedOut = databaseManager.GrowIndexesIfNeeded(token); } } catch (Exception ex) @@ -681,221 +697,61 @@ private async void IndexAutoGrowTask(CancellationToken token) } } - /// - /// Grows index if current size is smaller than max size. - /// Decision is based on whether overflow bucket allocation is more than a threshold which indicates a contention - /// in the index leading many allocations to the same bucket. - /// - /// - /// - /// - /// - /// - /// True if index has reached its max size - private bool GrowIndexIfNeeded(StoreType storeType, long indexMaxSize, long overflowCount, Func indexSizeRetriever, Action growAction) - { - logger?.LogDebug($"{nameof(IndexAutoGrowTask)}[{{storeType}}]: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}", storeType, indexSizeRetriever(), indexMaxSize, overflowCount); - - if (indexSizeRetriever() < indexMaxSize && - overflowCount > (indexSizeRetriever() * serverOptions.IndexResizeThreshold / 100)) - { - logger?.LogInformation($"{nameof(IndexAutoGrowTask)}[{{storeType}}]: overflowCount {{overflowCount}} ratio more than threshold {{indexResizeThreshold}}%. Doubling index size...", storeType, overflowCount, serverOptions.IndexResizeThreshold); - growAction(); - } - - if (indexSizeRetriever() < indexMaxSize) return false; - - logger?.LogDebug($"{nameof(IndexAutoGrowTask)}[{{storeType}}]: index size {{indexSizeRetriever}} reached index max size {{indexMaxSize}}", storeType, indexSizeRetriever(), indexMaxSize); - return true; - } - /// /// Dispose /// public void Dispose() { - // Wait for checkpoints to complete and disable checkpointing - _checkpointTaskLock.WriteLock(); + if (disposed) return; + disposed = true; itemBroker?.Dispose(); monitor?.Dispose(); luaTimeoutManager?.Dispose(); ctsCommit?.Cancel(); - - while (objectStoreSizeTracker != null && !objectStoreSizeTracker.Stopped) - Thread.Yield(); + databaseManager.Dispose(); ctsCommit?.Dispose(); clusterProvider?.Dispose(); } /// - /// Mark the beginning of a checkpoint by taking and a lock to avoid concurrent checkpoint tasks + /// Mark the beginning of a checkpoint by taking and a lock to avoid concurrent checkpointing /// - /// - public bool TryPauseCheckpoints() - => _checkpointTaskLock.TryWriteLock(); + /// ID of database to lock + /// True if lock acquired + public bool TryPauseCheckpoints(int dbId = 0) + { + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.TryPauseCheckpoints)} with DB ID: {dbId}"); - /// - /// Release checkpoint task lock - /// - public void ResumeCheckpoints() - => _checkpointTaskLock.WriteUnlock(); + return databaseManager.TryPauseCheckpoints(dbId); + } /// - /// Take a checkpoint if no checkpoint was taken after the provided time offset + /// Release checkpoint task lock /// - /// - /// - public async Task TakeOnDemandCheckpoint(DateTimeOffset entryTime) + /// ID of database to unlock + public void ResumeCheckpoints(int dbId = 0) { - // Take lock to ensure no other task will be taking a checkpoint - while (!TryPauseCheckpoints()) - await Task.Yield(); - - // If an external task has taken a checkpoint beyond the provided entryTime return - if (this.lastSaveTime > entryTime) - { - ResumeCheckpoints(); - return; - } + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.ResumeCheckpoints)} with DB ID: {dbId}"); - // Necessary to take a checkpoint because the latest checkpoint is before entryTime - await CheckpointTask(StoreType.All, logger: logger); + databaseManager.ResumeCheckpoints(dbId); } /// - /// Take checkpoint + /// Take a checkpoint if no checkpoint was taken after the provided time offset /// - /// - /// - /// + /// + /// /// - public bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null) + public async Task TakeOnDemandCheckpoint(DateTimeOffset entryTime, int dbId = 0) { - // Prevent parallel checkpoint - if (!TryPauseCheckpoints()) return false; - if (background) - Task.Run(async () => await CheckpointTask(storeType, logger)); - else - CheckpointTask(storeType, logger).ConfigureAwait(false).GetAwaiter().GetResult(); - return true; - } + if (dbId != 0 && !CheckMultiDatabaseCompatibility()) + throw new GarnetException($"Unable to call {nameof(databaseManager.TakeOnDemandCheckpointAsync)} with DB ID: {dbId}"); - private async Task CheckpointTask(StoreType storeType, ILogger logger = null) - { - try - { - DoCompaction(); - var lastSaveStoreTailAddress = store.Log.TailAddress; - var lastSaveObjectStoreTailAddress = (objectStore?.Log.TailAddress).GetValueOrDefault(); - - var full = false; - if (this.lastSaveStoreTailAddress == 0 || lastSaveStoreTailAddress - this.lastSaveStoreTailAddress >= serverOptions.FullCheckpointLogInterval) - full = true; - if (objectStore != null && (this.lastSaveObjectStoreTailAddress == 0 || lastSaveObjectStoreTailAddress - this.lastSaveObjectStoreTailAddress >= serverOptions.FullCheckpointLogInterval)) - full = true; - - var tryIncremental = serverOptions.EnableIncrementalSnapshots; - if (store.IncrementalSnapshotTailAddress >= serverOptions.IncrementalSnapshotLogSizeLimit) - tryIncremental = false; - if (objectStore?.IncrementalSnapshotTailAddress >= serverOptions.IncrementalSnapshotLogSizeLimit) - tryIncremental = false; - - var checkpointType = serverOptions.UseFoldOverCheckpoints ? CheckpointType.FoldOver : CheckpointType.Snapshot; - await InitiateCheckpoint(full, checkpointType, tryIncremental, storeType, logger); - if (full) - { - if (storeType is StoreType.Main or StoreType.All) - this.lastSaveStoreTailAddress = lastSaveStoreTailAddress; - if (storeType is StoreType.Object or StoreType.All) - this.lastSaveObjectStoreTailAddress = lastSaveObjectStoreTailAddress; - } - lastSaveTime = DateTimeOffset.UtcNow; - } - catch (Exception ex) - { - logger?.LogError(ex, "Checkpointing threw exception"); - } - finally - { - ResumeCheckpoints(); - } - } - - private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, bool tryIncremental, StoreType storeType, ILogger logger = null) - { - logger?.LogInformation("Initiating checkpoint; full = {full}, type = {checkpointType}, tryIncremental = {tryIncremental}, storeType = {storeType}", full, checkpointType, tryIncremental, storeType); - - long CheckpointCoveredAofAddress = 0; - if (appendOnlyFile != null) - { - if (serverOptions.EnableCluster) - clusterProvider.OnCheckpointInitiated(out CheckpointCoveredAofAddress); - else - CheckpointCoveredAofAddress = appendOnlyFile.TailAddress; - - if (CheckpointCoveredAofAddress > 0) - logger?.LogInformation("Will truncate AOF to {tailAddress} after checkpoint (files deleted after next commit)", CheckpointCoveredAofAddress); - } - - (bool success, Guid token) storeCheckpointResult = default; - (bool success, Guid token) objectStoreCheckpointResult = default; - if (full) - { - if (storeType is StoreType.Main or StoreType.All) - { - storeCheckpointResult = await store.TakeFullCheckpointAsync(checkpointType); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); - } - - if (objectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) - { - objectStoreCheckpointResult = await objectStore.TakeFullCheckpointAsync(checkpointType); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, objectStore.CurrentVersion); - } - } - else - { - if (storeType is StoreType.Main or StoreType.All) - { - storeCheckpointResult = await store.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); - } - - if (objectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) - { - objectStoreCheckpointResult = await objectStore.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, objectStore.CurrentVersion); - } - } - - // If cluster is enabled the replication manager is responsible for truncating AOF - if (serverOptions.EnableCluster && serverOptions.EnableAOF) - { - clusterProvider.SafeTruncateAOF(storeType, full, CheckpointCoveredAofAddress, storeCheckpointResult.token, objectStoreCheckpointResult.token); - } - else - { - appendOnlyFile?.TruncateUntil(CheckpointCoveredAofAddress); - appendOnlyFile?.Commit(); - } - - if (objectStore != null) - { - // During the checkpoint, we may have serialized Garnet objects in (v) versions of objects. - // We can now safely remove these serialized versions as they are no longer needed. - using (var iter1 = objectStore.Log.Scan(objectStore.Log.ReadOnlyAddress, objectStore.Log.TailAddress, ScanBufferingMode.SinglePageBuffering, includeSealedRecords: true)) - { - while (iter1.GetNext(out _, out _, out var value)) - { - if (value != null) - ((GarnetObjectBase)value).serialized = null; - } - } - } - - logger?.LogInformation("Completed checkpoint"); + await databaseManager.TakeOnDemandCheckpointAsync(entryTime, dbId); } public bool HasKeysInSlots(List slots) @@ -916,9 +772,9 @@ public bool HasKeysInSlots(List slots) } } - if (!hasKeyInSlots && objectStore != null) + if (!hasKeyInSlots && !serverOptions.DisableObjects) { - var functionsState = CreateFunctionsState(); + var functionsState = databaseManager.CreateFunctionsState(); var objstorefunctions = new ObjectSessionFunctions(functionsState); var objectStoreSession = objectStore?.NewSession(objstorefunctions); var iter = objectStoreSession.Iterate(); @@ -938,5 +794,27 @@ public bool HasKeysInSlots(List slots) return false; } + + private bool CheckMultiDatabaseCompatibility() + { + if (databaseManager is MultiDatabaseManager) + return true; + + if (!serverOptions.AllowMultiDb) + return false; + + databaseManagerLock.WriteLock(); + try + { + if (databaseManager is SingleDatabaseManager singleDatabaseManager) + databaseManager = new MultiDatabaseManager(singleDatabaseManager); + + return true; + } + finally + { + databaseManagerLock.WriteUnlock(); + } + } } } \ No newline at end of file diff --git a/playground/CommandInfoUpdater/GarnetCommandsDocs.json b/playground/CommandInfoUpdater/GarnetCommandsDocs.json index bebf4f591a..3111c5269d 100644 --- a/playground/CommandInfoUpdater/GarnetCommandsDocs.json +++ b/playground/CommandInfoUpdater/GarnetCommandsDocs.json @@ -1,4 +1,29 @@ [ + { + "Command": "BGSAVE", + "Name": "BGSAVE", + "Summary": "Asynchronously saves the database(s) to disk.", + "Group": "Server", + "Complexity": "O(1)", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "SCHEDULE", + "DisplayText": "schedule", + "Type": "PureToken", + "Token": "SCHEDULE", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] + }, { "Command": "CLIENT_KILL", "Name": "CLIENT|KILL", @@ -140,7 +165,17 @@ "Command": "COMMITAOF", "Name": "COMMITAOF", "Group": "Server", - "Summary": "Commit to append-only file." + "Summary": "Commit to append-only file.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] }, { "Command": "COSCAN", @@ -519,6 +554,23 @@ } ] }, + { + "Command": "LASTSAVE", + "Name": "LASTSAVE", + "Summary": "Returns the Unix timestamp of the last successful save to disk.", + "Group": "Server", + "Complexity": "O(1)", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] + }, { "Command": "PURGEBP", "Name": "PURGEBP", @@ -617,6 +669,23 @@ } ] }, + { + "Command": "SAVE", + "Name": "SAVE", + "Summary": "Synchronously saves the database(s) to disk.", + "Group": "Server", + "Complexity": "O(N) where N is the total number of keys in all databases", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] + }, { "Command": "SET", "Name": "SET", diff --git a/playground/CommandInfoUpdater/GarnetCommandsInfo.json b/playground/CommandInfoUpdater/GarnetCommandsInfo.json index c62c8b278d..b7af50ed9f 100644 --- a/playground/CommandInfoUpdater/GarnetCommandsInfo.json +++ b/playground/CommandInfoUpdater/GarnetCommandsInfo.json @@ -640,6 +640,16 @@ } ] }, + { + "Command": "LASTSAVE", + "Name": "LASTSAVE", + "Arity": -1, + "Flags": "Fast, Loading, Stale", + "AclCategories": "Admin, Dangerous, Fast", + "Tips": [ + "nondeterministic_output" + ] + }, { "Command": "PURGEBP", "Name": "PURGEBP", @@ -729,6 +739,13 @@ } ] }, + { + "Command": "SAVE", + "Name": "SAVE", + "Arity": -1, + "Flags": "Admin, NoAsyncLoading, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow" + }, { "Command": "SETIFMATCH", "Name": "SETIFMATCH", diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index 2d59d8be80..1b0edc8218 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -292,6 +292,7 @@ public class SupportedCommand new("SINTER", RespCommand.SINTER), new("SINTERCARD", RespCommand.SINTERCARD), new("SINTERSTORE", RespCommand.SINTERSTORE), + new("SWAPDB", RespCommand.SWAPDB), new("TIME", RespCommand.TIME), new("TTL", RespCommand.TTL), new("TYPE", RespCommand.TYPE), diff --git a/test/BDNPerfTests/BDN_Benchmark_Config.json b/test/BDNPerfTests/BDN_Benchmark_Config.json index da7782f563..8de6a83d69 100644 --- a/test/BDNPerfTests/BDN_Benchmark_Config.json +++ b/test/BDNPerfTests/BDN_Benchmark_Config.json @@ -383,5 +383,13 @@ "expected_Publish_ACL": 800, "expected_Publish_AOF": 800, "expected_Publish_None": 800 + }, + "BDN.benchmark.Operations.ServerOperations.*": { + "expected_SelectUnselect_ACL": 0, + "expected_SelectUnselect_AOF": 0, + "expected_SelectUnselect_None": 0, + "expected_SwapDb_ACL": 0, + "expected_SwapDb_AOF": 0, + "expected_SwapDb_None": 0 } } \ No newline at end of file diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 7c4a685682..7c1cf135fd 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -67,9 +67,15 @@ public void TearDown() loggerFactory?.Dispose(); var timeoutSeconds = 5; if (!Task.Run(() => DisposeCluster()).Wait(TimeSpan.FromSeconds(timeoutSeconds))) + { logger?.LogError("Timed out waiting for DisposeCluster"); + Assert.Fail("Timed out waiting for DisposeCluster"); + } if (!Task.Run(() => TestUtils.DeleteDirectory(TestFolder, true)).Wait(TimeSpan.FromSeconds(timeoutSeconds))) - logger?.LogError("Timed out waiting for DisposeCluster"); + { + logger?.LogError("Timed out DeleteDirectory"); + Assert.Fail("Timed out DeleteDirectory"); + } } public void RegisterCustomTxn(string name, Func proc, RespCommandsInfo commandInfo = null, RespCommandDocs commandDocs = null) diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index d5b1ea83a4..354d034b20 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -2762,7 +2762,13 @@ public int GetStoreCurrentVersion(int nodeIndex, bool isMainStore, ILogger logge continue; var field = line.Trim().Split(':'); - if (!Enum.TryParse(field[0], ignoreCase: true, out StoreInfoItem type)) + // Remove 'db0.' prefix + var sepIdx = field[0].IndexOf('.'); + if (sepIdx == -1) + continue; + var key = field[0].Substring(sepIdx + 1); + + if (!Enum.TryParse(key, ignoreCase: true, out StoreInfoItem type)) continue; if (infoItems.Contains(type)) @@ -2790,7 +2796,7 @@ public string GetInfo(IPEndPoint endPoint, string section, string segment, ILogg var result = server.Info(section); ClassicAssert.AreEqual(1, result.Length, "section does not exist"); foreach (var item in result[0]) - if (item.Key.Equals(segment)) + if (item.Key.Equals($"db0.{segment}")) return item.Value; Assert.Fail($"Segment not available for {section} section"); return ""; diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs index 56d35c2284..c670b91695 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs @@ -754,6 +754,8 @@ public void ClusterReplicationCheckpointCleanupTest([Values] bool performRMW, [V if (!attachReplicaTask.Wait(TimeSpan.FromSeconds(30))) Assert.Fail("attachReplicaTask timeout"); + + context.clusterTestUtils.WaitForReplicaAofSync(primaryIndex: 0, secondaryIndex: 1, logger: context.logger); } [Test, Order(14)] diff --git a/test/Garnet.test/GarnetServerConfigTests.cs b/test/Garnet.test/GarnetServerConfigTests.cs index 500bf3ddba..66c1ed0ecd 100644 --- a/test/Garnet.test/GarnetServerConfigTests.cs +++ b/test/Garnet.test/GarnetServerConfigTests.cs @@ -182,10 +182,11 @@ public void ImportExportRedisConfigLocal() ClassicAssert.AreEqual("placeholder", options.CertPassword); ClassicAssert.AreEqual(10000, options.SlowLogThreshold); ClassicAssert.AreEqual(128, options.SlowLogMaxEntries); + ClassicAssert.AreEqual(32, options.MaxDatabases); // Import from redis.conf file, include command line args // Check values from import path override values from default.conf, and values from command line override values from default.conf and import path - args = ["--config-import-path", redisConfigPath, "--config-import-format", "RedisConf", "--config-export-path", garnetConfigPath, "-p", "12m", "--tls", "false", "--minthreads", "6", "--client-certificate-required", "true"]; + args = ["--config-import-path", redisConfigPath, "--config-import-format", "RedisConf", "--config-export-path", garnetConfigPath, "-p", "12m", "--tls", "false", "--minthreads", "6", "--client-certificate-required", "true", "--max-databases", "64"]; parseSuccessful = ServerSettingsManager.TryParseCommandLineArguments(args, out options, out invalidOptions, out _, silentMode: true); ClassicAssert.IsTrue(parseSuccessful); ClassicAssert.AreEqual(invalidOptions.Count, 0); @@ -200,6 +201,7 @@ public void ImportExportRedisConfigLocal() ClassicAssert.AreEqual("placeholder", options.CertPassword); ClassicAssert.AreEqual(10000, options.SlowLogThreshold); ClassicAssert.AreEqual(128, options.SlowLogMaxEntries); + ClassicAssert.AreEqual(64, options.MaxDatabases); ClassicAssert.IsTrue(File.Exists(garnetConfigPath)); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); diff --git a/test/Garnet.test/MultiDatabaseTests.cs b/test/Garnet.test/MultiDatabaseTests.cs new file mode 100644 index 0000000000..7b07ce0309 --- /dev/null +++ b/test/Garnet.test/MultiDatabaseTests.cs @@ -0,0 +1,1127 @@ +using System; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; +using Garnet.common; +using Garnet.server; +using NUnit.Framework; +using NUnit.Framework.Legacy; +using StackExchange.Redis; + +namespace Garnet.test +{ + [TestFixture] + public class MultiDatabaseTests + { + GarnetServer server; + + [SetUp] + public void Setup() + { + TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableAOF: true, lowMemory: true, commitFrequencyMs: 1000); + server.Start(); + } + + [Test] + public void MultiDatabaseBasicSelectTestSE() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + var db12Key1 = "db12:key1"; + var db12Key2 = "db12:key1"; + + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db1 = redis.GetDatabase(0); + + db1.StringSet(db1Key1, "db1:value1"); + db1.ListLeftPush(db1Key2, [new RedisValue("db1:val1"), new RedisValue("db1:val2")]); + + var db2 = redis.GetDatabase(1); + ClassicAssert.IsFalse(db2.KeyExists(db1Key1)); + ClassicAssert.IsFalse(db2.KeyExists(db1Key2)); + + db2.StringSet(db2Key1, "db2:value2"); + db2.SetAdd(db2Key2, [new RedisValue("db2:val2"), new RedisValue("db2:val2")]); + + ClassicAssert.IsTrue(db2.KeyExists(db2Key1)); + ClassicAssert.IsTrue(db2.KeyExists(db2Key2)); + + ClassicAssert.IsFalse(db1.KeyExists(db2Key1)); + ClassicAssert.IsFalse(db1.KeyExists(db2Key2)); + + var db12 = redis.GetDatabase(11); + ClassicAssert.IsFalse(db12.KeyExists(db1Key1)); + ClassicAssert.IsFalse(db12.KeyExists(db1Key2)); + + db2.StringSet(db12Key2, "db12:value2"); + db2.SetAdd(db12Key2, [new RedisValue("db12:val2"), new RedisValue("db12:val2")]); + + ClassicAssert.IsFalse(db12.KeyExists(db12Key1)); + ClassicAssert.IsFalse(db12.KeyExists(db12Key2)); + } + + [Test] + public void MultiDatabaseBasicSelectErroneousTestSE() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db1 = redis.GetDatabase(0); + + db1.StringSet(db1Key1, "db1:value1"); + db1.ListLeftPush(db1Key2, [new RedisValue("db1:val1"), new RedisValue("db1:val2")]); + + var db17 = redis.GetDatabase(17); + Assert.Throws(() => db17.StringSet(db1Key1, "db1:value1"), "The database does not exist on the server: 17"); + } + + [Test] + public void MultiDatabaseSameKeyTestSE() + { + var key1 = "key1"; + + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db1 = redis.GetDatabase(0); + db1.StringSet(key1, "db1:val1"); + + var db2 = redis.GetDatabase(1); + db2.SetAdd(key1, [new RedisValue("db2:val1"), new RedisValue("db2:val2")]); + + var db12 = redis.GetDatabase(11); + db12.ListLeftPush(key1, [new RedisValue("db12:val1"), new RedisValue("db12:val2")]); + + var db1val = db1.StringGet(key1); + ClassicAssert.AreEqual("db1:val1", db1val.ToString()); + + var db2val = db2.SetMembers(key1); + CollectionAssert.AreEquivalent(db2val, new[] { new RedisValue("db2:val1"), new RedisValue("db2:val2") }); + + var db12val = db12.ListLeftPop(key1); + ClassicAssert.AreEqual("db12:val2", db12val.ToString()); + } + + [Test] + public void MultiDatabaseFlushDatabasesTestSE() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + var db12Key1 = "db12:key1"; + var db12Key2 = "db12:key2"; + var db1data = new RedisValue[] { "db1:a", "db1:b", "db1:c", "db1:d" }; + var db2data = new RedisValue[] { "db2:a", "db2:b", "db2:c", "db2:d" }; + var db12data = new RedisValue[] { "db12:a", "db12:b", "db12:c", "db12:d" }; + + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + + var db1 = redis.GetDatabase(0); + var result = db1.StringSet(db1Key1, "db1:val1"); + ClassicAssert.IsTrue(result); + + var count = db1.ListLeftPush(db1Key2, db1data); + ClassicAssert.AreEqual(db1data.Length, count); + + var db2 = redis.GetDatabase(1); + result = db2.StringSet(db2Key1, "db2:val1"); + ClassicAssert.IsTrue(result); + + count = db2.ListLeftPush(db2Key2, db2data); + ClassicAssert.AreEqual(db2data.Length, count); + + var db12 = redis.GetDatabase(11); + result = db12.StringSet(db12Key1, "db12:val1"); + ClassicAssert.IsTrue(result); + + count = db12.ListLeftPush(db12Key2, db12data); + ClassicAssert.AreEqual(db12data.Length, count); + + var opResult = db1.Execute("FLUSHDB"); + ClassicAssert.AreEqual("OK", opResult.ToString()); + + ClassicAssert.IsFalse(db1.KeyExists(db1Key1)); + ClassicAssert.IsFalse(db1.KeyExists(db1Key2)); + + ClassicAssert.IsTrue(db2.KeyExists(db2Key1)); + ClassicAssert.IsTrue(db2.KeyExists(db2Key2)); + + ClassicAssert.IsTrue(db12.KeyExists(db12Key1)); + ClassicAssert.IsTrue(db12.KeyExists(db12Key2)); + + opResult = db1.Execute("FLUSHALL"); + ClassicAssert.AreEqual("OK", opResult.ToString()); + + ClassicAssert.IsFalse(db2.KeyExists(db2Key1)); + ClassicAssert.IsFalse(db2.KeyExists(db2Key2)); + + ClassicAssert.IsFalse(db12.KeyExists(db12Key1)); + ClassicAssert.IsFalse(db12.KeyExists(db12Key2)); + } + + [Test] + public void MultiDatabaseBasicSelectTestLC() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key1"; + + using var lightClientRequest = TestUtils.CreateRequest(); + + var response = lightClientRequest.SendCommand($"SET {db1Key1} db1:value1"); + var expectedResponse = "+OK\r\n"; + var actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"LPUSH {db1Key2} db1:val1 db1:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + lightClientRequest.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"EXISTS {db1Key1}"); + expectedResponse = ":0\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"EXISTS {db1Key2}"); + expectedResponse = ":0\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SET {db2Key1} db2:value1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SADD {db2Key2} db2:val1 db2:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + lightClientRequest.SendCommand($"SELECT 0"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"GET {db1Key1}", 2); + expectedResponse = "$10\r\ndb1:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"LPOP {db1Key2}", 2); + expectedResponse = "$8\r\ndb1:val2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + lightClientRequest.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"GET {db2Key1}", 2); + expectedResponse = "$10\r\ndb2:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SISMEMBER {db2Key2} db2:val2"); + expectedResponse = ":1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + } + + [Test] + public void MultiDatabaseSwapDatabasesTestLC() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + var db12Key1 = "db12:key1"; + var db12Key2 = "db12:key2"; + + using var lightClientRequest = TestUtils.CreateRequest(); + + // Add data to DB 0 + var response = lightClientRequest.SendCommand($"SET {db1Key1} db1:value1"); + var expectedResponse = "+OK\r\n"; + var actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"LPUSH {db1Key2} db1:val1 db1:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Add data to DB 1 + response = lightClientRequest.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SET {db2Key1} db2:value1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SADD {db2Key2} db2:val1 db2:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Add data to DB 11 + response = lightClientRequest.SendCommand($"SELECT 11"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SET {db12Key1} db12:value1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SADD {db12Key2} db12:val1 db12:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Swap DB 1 AND DB 11 (from DB 11 context) + response = lightClientRequest.SendCommand($"SWAPDB 1 11"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 11 is previous data from DB 1 + response = lightClientRequest.SendCommand($"GET {db2Key1}", 2); + expectedResponse = "$10\r\ndb2:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SISMEMBER {db2Key2} db2:val2"); + expectedResponse = ":1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 1 is previous data from DB 11 + response = lightClientRequest.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"GET {db12Key1}", 2); + expectedResponse = "$11\r\ndb12:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SISMEMBER {db12Key2} db12:val2"); + expectedResponse = ":1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Swap DB 11 AND DB 0 (from DB 1 context) + response = lightClientRequest.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SWAPDB 11 0"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 0 is previous data from DB 11 + response = lightClientRequest.SendCommand($"SELECT 0"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"GET {db2Key1}", 2); + expectedResponse = "$10\r\ndb2:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"SISMEMBER {db2Key2} db2:val2"); + expectedResponse = ":1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 11 is previous data from DB 0 + response = lightClientRequest.SendCommand($"SELECT 11"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"GET {db1Key1}", 2); + expectedResponse = "$10\r\ndb1:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest.SendCommand($"LPOP {db1Key2}", 2); + expectedResponse = "$8\r\ndb1:val2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + } + + [Test] + public void MultiDatabaseMultiSessionSwapDatabasesErrorTestLC() + { + // Ensure that SWAPDB returns an error when multiple clients are connected. + var db1Key1 = "db1:key1"; + var db2Key1 = "db2:key1"; + + using var lightClientRequest1 = TestUtils.CreateRequest(); // Session for DB 0 context + using var lightClientRequest2 = TestUtils.CreateRequest(); // Session for DB 1 context + + // Add data to DB 0 + var response = lightClientRequest1.SendCommand($"SET {db1Key1} db1:value1"); + var expectedResponse = "+OK\r\n"; + var actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Add data to DB 1 + response = lightClientRequest2.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest2.SendCommand($"SET {db2Key1} db2:value1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Swap DB 0 AND DB 1 (from DB 0 context) + response = lightClientRequest1.SendCommand($"SWAPDB 0 1"); + expectedResponse = $"-{Encoding.ASCII.GetString(CmdStrings.RESP_ERR_DBSWAP_UNSUPPORTED)}\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + } + + [Test] + [Ignore("SWAPDB is currently disallowed for more than one client session. This test should be enabled once that changes.")] + public void MultiDatabaseMultiSessionSwapDatabasesTestLC() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + + using var lightClientRequest1 = TestUtils.CreateRequest(); // Session for DB 0 context + using var lightClientRequest2 = TestUtils.CreateRequest(); // Session for DB 1 context + + // Add data to DB 0 + var response = lightClientRequest1.SendCommand($"SET {db1Key1} db1:value1"); + var expectedResponse = "+OK\r\n"; + var actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest1.SendCommand($"LPUSH {db1Key2} db1:val1 db1:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Add data to DB 1 + response = lightClientRequest2.SendCommand($"SELECT 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest2.SendCommand($"SET {db2Key1} db2:value1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest2.SendCommand($"SADD {db2Key2} db2:val1 db2:val2"); + expectedResponse = ":2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest2.SendCommand($"GET {db2Key1}", 2); + expectedResponse = "$10\r\ndb2:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Swap DB 0 AND DB 1 (from DB 0 context) + response = lightClientRequest1.SendCommand($"SWAPDB 0 1"); + expectedResponse = "+OK\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 0 is previous data from DB 1 + response = lightClientRequest1.SendCommand($"GET {db2Key1}", 2); + expectedResponse = "$10\r\ndb2:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest1.SendCommand($"SISMEMBER {db2Key2} db2:val2"); + expectedResponse = ":1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + // Verify data in DB 1 is previous data from DB 0 + response = lightClientRequest2.SendCommand($"GET {db1Key1}", 2); + expectedResponse = "$10\r\ndb1:value1\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + + response = lightClientRequest2.SendCommand($"LPOP {db1Key2}", 2); + expectedResponse = "$8\r\ndb1:val2\r\n"; + actualValue = Encoding.ASCII.GetString(response).Substring(0, expectedResponse.Length); + ClassicAssert.AreEqual(expectedResponse, actualValue); + } + + [Test] + [Ignore("")] + public void MultiDatabaseSelectMultithreadedTestLC() + { + var cts = new CancellationTokenSource(); + + // Create a set of tuples (db-id, key, value) + var dbCount = 16; + var keyCount = 8; + var tuples = GenerateDataset(dbCount, keyCount); + + // Create multiple LC request objects to be used + var lcRequests = new LightClientRequest[16]; + for (var i = 0; i < lcRequests.Length; i++) + lcRequests[i] = TestUtils.CreateRequest(countResponseType: CountResponseType.Bytes); + + // In parallel, add each (key, value) pair to a database of id db-id + var tasks = new Task[lcRequests.Length]; + var results = new bool[tuples.Length]; + var tupIdx = -1; + for (var i = 0; i < tasks.Length; i++) + { + var lcRequest = lcRequests[i]; + tasks[i] = Task.Run(() => + { + while (true) + { + var currTupIdx = Interlocked.Increment(ref tupIdx); + if (currTupIdx >= tuples.Length) break; + + var tup = tuples[currTupIdx]; + + var expectedResponse = "+OK\r\n+OK\r\n"; + var response = lcRequest.Execute($"SELECT {tup.Item1}", $"SET {tup.Item2} {tup.Item3}", expectedResponse.Length); + + results[currTupIdx] = response != null && expectedResponse == response; + } + }, cts.Token); + } + + // Wait for all tasks to finish + if (!Task.WhenAll(tasks).Wait(TimeSpan.FromSeconds(60))) + { + cts.Cancel(); + Assert.Fail("Items not inserted in allotted time."); + } + + // Check that all tasks successfully entered the data to the respective database + Assert.That(results, Is.All.True); + + cts = new CancellationTokenSource(); + + // In parallel, retrieve the actual value for each db-id and key + for (var i = 0; i < tasks.Length; i++) + { + var lcRequest = lcRequests[i]; + tasks[i] = Task.Run(() => + { + while (true) + { + var currTupIdx = Interlocked.Increment(ref tupIdx); + if (currTupIdx >= tuples.Length) break; + + var tup = tuples[currTupIdx]; + + var expectedResponse = $"+OK\r\n${tup.Item3.Length}\r\n{tup.Item3}\r\n"; + var response = lcRequest.Execute($"SELECT {tup.Item1}", $"GET {tup.Item2}", + expectedResponse.Length); + + results[currTupIdx] = response != null && expectedResponse == response; + } + + lcRequest.Dispose(); + }, cts.Token); + } + + // Wait for all tasks to finish + if (!Task.WhenAll(tasks).Wait(TimeSpan.FromSeconds(60))) + { + cts.Cancel(); + Assert.Fail("Items not retrieved in allotted time."); + } + + // Check that all the tasks retrieved the correct value successfully + Assert.That(results, Is.All.True); + } + + [Test] + public void MultiDatabaseSelectMultithreadedTestSE() + { + // Create a set of tuples (db-id, key, value) + var dbCount = 16; + var keyCount = 8; + var tuples = GenerateDataset(dbCount, keyCount); + + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var dbConnections = new IDatabase[dbCount]; + for (var i = 0; i < dbCount; i++) + { + dbConnections[i] = redis.GetDatabase(i); + } + + var cts = new CancellationTokenSource(TimeSpan.FromSeconds(60)); + + // In parallel, add each (key, value) pair to a database of id db-id + var tasks = new Task[tuples.Length]; + for (var i = 0; i < tasks.Length; i++) + { + var tup = tuples[i]; + tasks[i] = Task.Run(async () => + { + var db = dbConnections[tup.Item1]; + return await db.StringSetAsync(tup.Item3, tup.Item4).ConfigureAwait(false); + }, cts.Token); + } + + // Wait for all tasks to finish + if (!Task.WhenAll(tasks).Wait(TimeSpan.FromSeconds(60), cts.Token)) + Assert.Fail("Items not inserted in allotted time."); + + // Check that all tasks successfully entered the data to the respective database + Assert.That(tasks, Has.All.Matches>(t => t.IsCompletedSuccessfully && t.Result)); + + cts = new CancellationTokenSource(TimeSpan.FromSeconds(60)); + + // In parallel, retrieve the actual value for each db-id and key + for (var i = 0; i < tasks.Length; i++) + { + var tup = tuples[i]; + tasks[i] = Task.Run(async () => + { + var db = dbConnections[tup.Item1]; + var actualValue = await db.StringGetAsync(tup.Item3).ConfigureAwait(false); + return actualValue.ToString() == tup.Item4; + }, cts.Token); + } + + // Wait for all tasks to finish + if (!Task.WhenAll(tasks).Wait(TimeSpan.FromSeconds(60), cts.Token)) + Assert.Fail("Items not retrieved in allotted time."); + + // Check that (db-id, key, actual-value) tuples match original (db-id, key, value) tuples + Assert.That(tasks, Has.All.Matches>(t => t.IsCompletedSuccessfully && t.Result)); + } + + [Test] + public void MultiDatabaseSaveRecoverObjectTest() + { + var db1Key = "db1:key1"; + var db2Key = "db1:key1"; + var db1data = new RedisValue[] { "db1:a", "db1:b", "db1:c", "db1:d" }; + var db2data = new RedisValue[] { "db2:a", "db2:b", "db2:c", "db2:d" }; + RedisValue[] db1DataBeforeRecovery; + RedisValue[] db2DataBeforeRecovery; + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) + { + var db1 = redis.GetDatabase(0); + db1.ListLeftPush(db1Key, db1data); + db1data = db1data.Select(x => x).Reverse().ToArray(); + db1DataBeforeRecovery = db1.ListRange(db1Key); + ClassicAssert.AreEqual(db1data, db1DataBeforeRecovery); + + var db2 = redis.GetDatabase(1); + db2.SetAdd(db2Key, db2data); + db2DataBeforeRecovery = db2.SetMembers(db2Key); + ClassicAssert.AreEqual(db2data, db2DataBeforeRecovery); + + // Issue and wait for DB save + var garnetServer = redis.GetServer(TestUtils.EndPoint); + garnetServer.Save(SaveType.BackgroundSave); + while (garnetServer.LastSave().Ticks == DateTimeOffset.FromUnixTimeSeconds(0).Ticks) Thread.Sleep(10); + } + + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) + { + var db1 = redis.GetDatabase(0); + var db1ReturnedData = db1.ListRange(db1Key); + ClassicAssert.AreEqual(db1DataBeforeRecovery, db1ReturnedData); + ClassicAssert.AreEqual(db1data.Length, db1ReturnedData.Length); + ClassicAssert.AreEqual(db1data, db1ReturnedData); + + var db2 = redis.GetDatabase(1); + var db2ReturnedData = db2.SetMembers(db2Key); + ClassicAssert.AreEqual(db2DataBeforeRecovery, db2ReturnedData); + ClassicAssert.AreEqual(db2data.Length, db2ReturnedData.Length); + ClassicAssert.AreEqual(db2data, db2ReturnedData); + } + } + + [Test] + public void MultiDatabaseSaveRecoverRawStringTest() + { + var db1Key = "db1:key1"; + var db2Key = "db2:key1"; + var db1data = new RedisValue("db1:a"); + var db2data = new RedisValue("db2:a"); + RedisValue db1DataBeforeRecovery; + RedisValue db2DataBeforeRecovery; + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) + { + var db1 = redis.GetDatabase(0); + db1.StringSet(db1Key, db1data); + db1DataBeforeRecovery = db1.StringGet(db1Key); + ClassicAssert.AreEqual(db1data, db1DataBeforeRecovery); + + var db2 = redis.GetDatabase(1); + db2.StringSet(db2Key, db2data); + db2DataBeforeRecovery = db2.StringGet(db2Key); + ClassicAssert.AreEqual(db2data, db2DataBeforeRecovery); + + // Issue and wait for DB save + var garnetServer = redis.GetServer(TestUtils.EndPoint); + garnetServer.Save(SaveType.BackgroundSave); + while (garnetServer.LastSave().Ticks == DateTimeOffset.FromUnixTimeSeconds(0).Ticks) Thread.Sleep(10); + } + + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) + { + var db1 = redis.GetDatabase(0); + var db1ReturnedData = db1.StringGet(db1Key); + ClassicAssert.AreEqual(db1DataBeforeRecovery, db1ReturnedData); + ClassicAssert.AreEqual(db1data, db1ReturnedData); + + var db2 = redis.GetDatabase(1); + var db2ReturnedData = db2.StringGet(db2Key); + ClassicAssert.AreEqual(db2DataBeforeRecovery, db2ReturnedData); + ClassicAssert.AreEqual(db2data, db2ReturnedData); + } + } + + [Test] + public void MultiDatabaseAofRecoverRawStringTest() + { + var db1Key = "db1:key1"; + var db2Key = "db2:key1"; + var db1data = new RedisValue("db1:a"); + var db2data = new RedisValue("db2:a"); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db1 = redis.GetDatabase(0); + var result = db1.StringSet(db1Key, db1data); + ClassicAssert.IsTrue(result); + + var value = db1.StringGet(db1Key); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db1data, value.ToString()); + + var db2 = redis.GetDatabase(1); + result = db2.StringSet(db2Key, db2data); + ClassicAssert.IsTrue(result); + + value = db2.StringGet(db2Key); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2data, value.ToString()); + } + + server.Store.CommitAOF(true); + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, enableAOF: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db1 = redis.GetDatabase(0); + + var value = db1.StringGet(db1Key); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db1data, value.ToString()); + + var db2 = redis.GetDatabase(1); + + value = db2.StringGet(db2Key); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2data, value.ToString()); + } + } + + [Test] + public void MultiDatabaseAofRecoverObjectTest() + { + var db1Key = "db1:key1"; + var db2Key = "db2:key1"; + var db1data = new SortedSetEntry[] { new("db1:a", 1), new("db1:b", 2), new("db1:c", 3) }; + var db2data = new SortedSetEntry[] { new("db2:a", -1), new("db2:b", -2), new("db2:c", -3) }; + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db1 = redis.GetDatabase(0); + var added = db1.SortedSetAdd(db1Key, db1data); + ClassicAssert.AreEqual(3, added); + + var score = db1.SortedSetScore(db1Key, "db1:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(1, score.Value); + + var db2 = redis.GetDatabase(1); + added = db2.SortedSetAdd(db2Key, db2data); + ClassicAssert.AreEqual(3, added); + + score = db2.SortedSetScore(db2Key, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + } + + server.Store.CommitAOF(true); + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, enableAOF: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db1 = redis.GetDatabase(0); + + var score = db1.SortedSetScore(db1Key, "db1:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(1, score.Value); + + var db2 = redis.GetDatabase(1); + + score = db2.SortedSetScore(db2Key, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + } + } + + [Test] + [TestCase(false)] + [TestCase(true)] + public void MultiDatabaseSaveRecoverByDbIdTest(bool backgroundSave) + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + var db2Key3 = "db2:key3"; + var db2Key4 = "db2:key4"; + var db1val = new RedisValue("db1:a"); + var db2val1 = new RedisValue("db2:a"); + var db2val2 = new RedisValue("db2:b"); + var db1data = new SortedSetEntry[] { new("db1:a", 1), new("db1:b", 2), new("db1:c", 3) }; + var db2data1 = new SortedSetEntry[] { new("db2:a", -1), new("db2:b", -2), new("db2:c", -3) }; + var db2data2 = new SortedSetEntry[] { new("db2:d", 4), new("db2:e", 5), new("db2:f", 6) }; + long expectedLastSave; + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + // Add object & raw string data to DB 0 + var db1 = redis.GetDatabase(0); + var added = db1.SortedSetAdd(db1Key1, db1data); + ClassicAssert.AreEqual(3, added); + + var set = db1.StringSet(db1Key2, db1val); + ClassicAssert.IsTrue(set); + + // Add object & raw string data to DB 1 + var db2 = redis.GetDatabase(1); + added = db2.SortedSetAdd(db2Key1, db2data1); + ClassicAssert.AreEqual(3, added); + + set = db2.StringSet(db2Key2, db2val1); + ClassicAssert.IsTrue(set); + + // Issue DB SAVE for DB 1 + var res = db1.Execute(backgroundSave ? "BGSAVE" : "SAVE", "1"); + ClassicAssert.AreEqual(backgroundSave ? "Background saving started" : "OK", res.ToString()); + + var lastSave = 0L; + string lastSaveStr; + bool parsed; + var cts = new CancellationTokenSource(TimeSpan.FromSeconds(2)); + while (!cts.IsCancellationRequested) + { + // Verify DB 1 was saved by checking LASTSAVE + lastSaveStr = db1.Execute("LASTSAVE", "1").ToString(); + parsed = long.TryParse(lastSaveStr, out lastSave); + ClassicAssert.IsTrue(parsed); + if (lastSave != 0) + break; + Task.Delay(TimeSpan.FromMilliseconds(100), cts.Token); + } + + expectedLastSave = DateTimeOffset.UtcNow.ToUnixTimeSeconds(); + Assert.That(lastSave, Is.InRange(expectedLastSave - 2, expectedLastSave)); + + // Verify DB 0 was not saved + lastSaveStr = db1.Execute("LASTSAVE").ToString(); + parsed = long.TryParse(lastSaveStr, out lastSave); + ClassicAssert.IsTrue(parsed); + ClassicAssert.AreEqual(0, lastSave); + } + + // Restart server + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var lastSave = 0L; + string lastSaveStr; + bool parsed; + + // Verify that data was not recovered for DB 0 + var db1 = redis.GetDatabase(0); + + ClassicAssert.IsFalse(db1.KeyExists(db1Key1)); + ClassicAssert.IsFalse(db1.KeyExists(db1Key2)); + + // Verify that data was recovered for DB 1 + var db2 = redis.GetDatabase(1); + + var score = db2.SortedSetScore(db2Key1, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + + var value = db2.StringGet(db2Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2val1, value.ToString()); + + // Re-add object & raw string data to DB 0 + var added = db1.SortedSetAdd(db1Key1, db1data); + ClassicAssert.AreEqual(3, added); + + var set = db1.StringSet(db1Key2, db1val); + ClassicAssert.IsTrue(set); + + // Add new object & raw string data to DB 1 + added = db2.SortedSetAdd(db2Key3, db2data2); + ClassicAssert.AreEqual(3, added); + + set = db2.StringSet(db2Key4, db2val2); + ClassicAssert.IsTrue(set); + + // Issue DB SAVE for DB 0 + var res = db1.Execute(backgroundSave ? "BGSAVE" : "SAVE", "0"); + ClassicAssert.AreEqual(backgroundSave ? "Background saving started" : "OK", res.ToString()); + + // Verify DB 0 was saved by checking LASTSAVE + var cts = new CancellationTokenSource(TimeSpan.FromSeconds(2)); + while (!cts.IsCancellationRequested) + { + lastSaveStr = db1.Execute("LASTSAVE").ToString(); + parsed = long.TryParse(lastSaveStr, out lastSave); + ClassicAssert.IsTrue(parsed); + if (lastSave != 0) + break; + Task.Delay(TimeSpan.FromMilliseconds(100), cts.Token); + } + + var prevLastSave = expectedLastSave; + expectedLastSave = DateTimeOffset.UtcNow.ToUnixTimeSeconds(); + Assert.That(lastSave, Is.InRange(expectedLastSave - 2, expectedLastSave)); + + // Verify DB 1 was not saved + Thread.Sleep(TimeSpan.FromSeconds(2)); + lastSaveStr = db1.Execute("LASTSAVE", "1").ToString(); + parsed = long.TryParse(lastSaveStr, out lastSave); + ClassicAssert.IsTrue(parsed); + Assert.That(lastSave, Is.InRange(prevLastSave - 2, prevLastSave)); + } + + // Restart server + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + // Verify that data was recovered for DB 0 + var db1 = redis.GetDatabase(0); + + var score = db1.SortedSetScore(db1Key1, "db1:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(1, score.Value); + + var value = db1.StringGet(db1Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db1val, value.ToString()); + + // Verify that previous data was recovered for DB 1 + var db2 = redis.GetDatabase(1); + + score = db2.SortedSetScore(db2Key1, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + + value = db2.StringGet(db2Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2val1, value.ToString()); + + // Verify that new data was not recovered for DB 1 + ClassicAssert.IsFalse(db1.KeyExists(db2Key3)); + ClassicAssert.IsFalse(db1.KeyExists(db2Key4)); + } + } + + [Test] + public void MultiDatabaseAofRecoverByDbIdTest() + { + var db1Key1 = "db1:key1"; + var db1Key2 = "db1:key2"; + var db2Key1 = "db2:key1"; + var db2Key2 = "db2:key2"; + var db2Key3 = "db2:key3"; + var db2Key4 = "db2:key4"; + var db1val = new RedisValue("db1:a"); + var db2val1 = new RedisValue("db2:a"); + var db2val2 = new RedisValue("db2:b"); + var db1data = new SortedSetEntry[] { new("db1:a", 1), new("db1:b", 2), new("db1:c", 3) }; + var db2data1 = new SortedSetEntry[] { new("db2:a", -1), new("db2:b", -2), new("db2:c", -3) }; + var db2data2 = new SortedSetEntry[] { new("db2:d", 4), new("db2:e", 5), new("db2:f", 6) }; + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + // Add object & raw string data to DB 0 + var db1 = redis.GetDatabase(0); + var added = db1.SortedSetAdd(db1Key1, db1data); + ClassicAssert.AreEqual(3, added); + + var set = db1.StringSet(db1Key2, db1val); + ClassicAssert.IsTrue(set); + + // Add object & raw string data to DB 1 + var db2 = redis.GetDatabase(1); + added = db2.SortedSetAdd(db2Key1, db2data1); + ClassicAssert.AreEqual(3, added); + + set = db2.StringSet(db2Key2, db2val1); + ClassicAssert.IsTrue(set); + + // Issue COMMITAOF for DB 1 + var res = db1.Execute("COMMITAOF", "1"); + ClassicAssert.AreEqual("AOF file committed", res.ToString()); + } + + // Restart server + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, enableAOF: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + // Verify that data was not recovered for DB 0 + var db1 = redis.GetDatabase(0); + + ClassicAssert.IsFalse(db1.KeyExists(db1Key1)); + ClassicAssert.IsFalse(db1.KeyExists(db1Key2)); + + // Verify that data was recovered for DB 1 + var db2 = redis.GetDatabase(1); + + var score = db2.SortedSetScore(db2Key1, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + + var value = db2.StringGet(db2Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2val1, value.ToString()); + + // Re-add object & raw string data to DB 0 + var added = db1.SortedSetAdd(db1Key1, db1data); + ClassicAssert.AreEqual(3, added); + + var set = db1.StringSet(db1Key2, db1val); + ClassicAssert.IsTrue(set); + + // Add new object & raw string data to DB 1 + added = db2.SortedSetAdd(db2Key3, db2data2); + ClassicAssert.AreEqual(3, added); + + set = db2.StringSet(db2Key4, db2val2); + ClassicAssert.IsTrue(set); + + // Issue COMMITAOF for DB 0 + var res = db1.Execute("COMMITAOF", "0"); + ClassicAssert.AreEqual("AOF file committed", res.ToString()); + } + + // Restart server + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, enableAOF: true); + server.Start(); + + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + // Verify that data was recovered for DB 0 + var db1 = redis.GetDatabase(0); + + var score = db1.SortedSetScore(db1Key1, "db1:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(1, score.Value); + + var value = db1.StringGet(db1Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db1val, value.ToString()); + + // Verify that previous data was recovered for DB 1 + var db2 = redis.GetDatabase(1); + + score = db2.SortedSetScore(db2Key1, "db2:a"); + ClassicAssert.IsTrue(score.HasValue); + ClassicAssert.AreEqual(-1, score.Value); + + value = db2.StringGet(db2Key2); + ClassicAssert.IsTrue(value.HasValue); + ClassicAssert.AreEqual(db2val1, value.ToString()); + + // Verify that new data was not recovered for DB 1 + ClassicAssert.IsFalse(db1.KeyExists(db2Key3)); + ClassicAssert.IsFalse(db1.KeyExists(db2Key4)); + } + } + + [TearDown] + public void TearDown() + { + server.Dispose(); + TestUtils.DeleteDirectory(TestUtils.MethodTestDir); + } + + private (int, int, string, string)[] GenerateDataset(int dbCount, int keyCount) + { + var data = new (int, int, string, string)[dbCount * keyCount]; + + for (var dbId = 0; dbId < dbCount; dbId++) + { + for (var keyId = 0; keyId < keyCount; keyId++) + { + data[(keyCount * dbId) + keyId] = (dbId, keyId, $"key{keyId}", $"db{dbId}:val{keyId}"); + } + } + + return data; + } + } +} \ No newline at end of file diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index d721f9103b..2ebff69f04 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -7044,6 +7044,32 @@ async Task DoRestoreAsync(GarnetClient client) } } + [Test] + public async Task SwapDbACLsAsync() + { + await CheckCommandsAsync( + "SWAPDB", + [DoSwapDbAsync] + ); + + static async Task DoSwapDbAsync(GarnetClient client) + { + try + { + // Currently SWAPDB does not support calling the command when multiple clients are connected to the server. + await client.ExecuteForStringResultAsync("SWAPDB", ["0", "1"]); + Assert.Fail("Shouldn't reach here, calling SWAPDB should fail."); + } + catch (Exception ex) + { + if (ex.Message == Encoding.ASCII.GetString(CmdStrings.RESP_ERR_NOAUTH)) + throw; + + ClassicAssert.AreEqual(Encoding.ASCII.GetString(CmdStrings.RESP_ERR_DBSWAP_UNSUPPORTED), ex.Message); + } + } + } + [Test] public async Task TypeACLsAsync() { diff --git a/test/Garnet.test/RespAdminCommandsTests.cs b/test/Garnet.test/RespAdminCommandsTests.cs index 3e74e3c6a4..2be48a812d 100644 --- a/test/Garnet.test/RespAdminCommandsTests.cs +++ b/test/Garnet.test/RespAdminCommandsTests.cs @@ -598,7 +598,7 @@ public async Task SeFlushDbAndFlushAllTest2([Values(RespCommand.FLUSHALL, RespCo [TestCase("save", "")] [TestCase("appendonly", "no")] [TestCase("slave-read-only", "no")] - [TestCase("databases", "1")] + [TestCase("databases", "16")] [TestCase("cluster-node-timeout", "60")] public void SimpleConfigGet(string parameter, string parameterValue) { diff --git a/test/Garnet.test/RespCommandTests.cs b/test/Garnet.test/RespCommandTests.cs index 70b2e4ae9b..c0d90a60fd 100644 --- a/test/Garnet.test/RespCommandTests.cs +++ b/test/Garnet.test/RespCommandTests.cs @@ -427,6 +427,7 @@ public void AofIndependentCommandsTest() RespCommand.ASYNC, RespCommand.PING, RespCommand.SELECT, + RespCommand.SWAPDB, RespCommand.ECHO, RespCommand.MONITOR, RespCommand.MODULE_LOADCS, diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index bde2c6911b..b039d89ea2 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -2309,7 +2309,7 @@ public void CanSelectCommand() var db = redis.GetDatabase(0); var reply = db.Execute("SELECT", "0"); ClassicAssert.IsTrue(reply.ToString() == "OK"); - Assert.Throws(() => db.Execute("SELECT", "1")); + Assert.Throws(() => db.Execute("SELECT", "17")); //select again the def db db.Execute("SELECT", "0"); @@ -2321,7 +2321,7 @@ public void CanSelectCommandLC() using var lightClientRequest = TestUtils.CreateRequest(countResponseType: CountResponseType.Bytes); var expectedResponse = "-ERR invalid database index.\r\n+PONG\r\n"; - var response = lightClientRequest.Execute("SELECT 1", "PING", expectedResponse.Length); + var response = lightClientRequest.Execute("SELECT 17", "PING", expectedResponse.Length); ClassicAssert.AreEqual(expectedResponse, response); } diff --git a/test/Garnet.test/RespTlsTests.cs b/test/Garnet.test/RespTlsTests.cs index bfe58dd7a7..1d9f289b15 100644 --- a/test/Garnet.test/RespTlsTests.cs +++ b/test/Garnet.test/RespTlsTests.cs @@ -352,7 +352,7 @@ public void TlsCanSelectCommand() var db = redis.GetDatabase(0); var reply = db.Execute("SELECT", "0"); ClassicAssert.IsTrue(reply.ToString() == "OK"); - Assert.Throws(() => db.Execute("SELECT", "1")); + Assert.Throws(() => db.Execute("SELECT", "17")); //select again the def db db.Execute("SELECT", "0"); @@ -364,7 +364,7 @@ public void TlsCanSelectCommandLC() using var lightClientRequest = TestUtils.CreateRequest(useTLS: true, countResponseType: CountResponseType.Bytes); var expectedResponse = "-ERR invalid database index.\r\n+PONG\r\n"; - var response = lightClientRequest.Execute("SELECT 1", "PING", expectedResponse.Length); + var response = lightClientRequest.Execute("SELECT 17", "PING", expectedResponse.Length); ClassicAssert.AreEqual(expectedResponse, response); } diff --git a/test/Garnet.test/TestUtils.cs b/test/Garnet.test/TestUtils.cs index fc33c512f1..980b49d793 100644 --- a/test/Garnet.test/TestUtils.cs +++ b/test/Garnet.test/TestUtils.cs @@ -957,7 +957,7 @@ public static void CreateTestLibrary(string[] namespaces, string[] referenceFile } } - public static StoreAddressInfo GetStoreAddressInfo(IServer server, bool includeReadCache = false, bool isObjectStore = false) + public static StoreAddressInfo GetStoreAddressInfo(IServer server, int dbId = 0, bool includeReadCache = false, bool isObjectStore = false) { StoreAddressInfo result = default; var info = isObjectStore ? server.Info("OBJECTSTORE") : server.Info("STORE"); @@ -965,19 +965,19 @@ public static StoreAddressInfo GetStoreAddressInfo(IServer server, bool includeR { foreach (var entry in section) { - if (entry.Key.Equals("Log.BeginAddress")) + if (entry.Key.Equals($"db{dbId}.Log.BeginAddress")) result.BeginAddress = long.Parse(entry.Value); - else if (entry.Key.Equals("Log.HeadAddress")) + else if (entry.Key.Equals($"db{dbId}.Log.HeadAddress")) result.HeadAddress = long.Parse(entry.Value); - else if (entry.Key.Equals("Log.SafeReadOnlyAddress")) + else if (entry.Key.Equals($"db{dbId}.Log.SafeReadOnlyAddress")) result.ReadOnlyAddress = long.Parse(entry.Value); - else if (entry.Key.Equals("Log.TailAddress")) + else if (entry.Key.Equals($"db{dbId}.Log.TailAddress")) result.TailAddress = long.Parse(entry.Value); - else if (entry.Key.Equals("Log.MemorySizeBytes")) + else if (entry.Key.Equals($"db{dbId}.Log.MemorySizeBytes")) result.MemorySize = long.Parse(entry.Value); - else if (includeReadCache && entry.Key.Equals("ReadCache.BeginAddress")) + else if (includeReadCache && entry.Key.Equals($"db{dbId}.ReadCache.BeginAddress")) result.ReadCacheBeginAddress = long.Parse(entry.Value); - else if (includeReadCache && entry.Key.Equals("ReadCache.TailAddress")) + else if (includeReadCache && entry.Key.Equals($"db{dbId}.ReadCache.TailAddress")) result.ReadCacheTailAddress = long.Parse(entry.Value); } } diff --git a/test/Garnet.test/redis.conf b/test/Garnet.test/redis.conf index 450e402320..a4256a4338 100644 --- a/test/Garnet.test/redis.conf +++ b/test/Garnet.test/redis.conf @@ -377,7 +377,7 @@ logfile ./garnet-log # Set the number of databases. The default database is DB 0, you can select # a different one on a per-connection basis using SELECT where # dbid is a number between 0 and 'databases'-1 -# databases 16 +databases 32 # By default Redis shows an ASCII art logo only when started to log to the # standard output and if the standard output is a TTY and syslog logging is diff --git a/website/docs/commands/api-compatibility.md b/website/docs/commands/api-compatibility.md index 118d0122b4..2bf3588421 100644 --- a/website/docs/commands/api-compatibility.md +++ b/website/docs/commands/api-compatibility.md @@ -291,7 +291,7 @@ Note that this list is subject to change as we continue to expand our API comman | | [SAVE](checkpoint.md#save) | ➕ | | | | SHUTDOWN | ➖ | | | | [SLAVEOF](server.md#slaveof) | ➕ | (Deprecated) | -| | SWAPDB | ➖ | | +| | [SWAPDB](server.md#swapdb) | ➕ | | | | SYNC | ➖ | | | | [TIME](server.md#time) | ➕ | | | **SET** | [SADD](data-structures.md#sadd) | ➕ | | diff --git a/website/docs/commands/checkpoint.md b/website/docs/commands/checkpoint.md index 39983f4059..c4b66c02e2 100644 --- a/website/docs/commands/checkpoint.md +++ b/website/docs/commands/checkpoint.md @@ -9,10 +9,10 @@ slug: checkpoint #### Syntax ```bash -BGSAVE [SCHEDULE] +BGSAVE [SCHEDULE] [DBID] ``` -Save the DB in background. +Save all databases inside the Garnet instance in the background. If a DB ID is specified, save save only that specific database. #### Resp Reply @@ -28,10 +28,10 @@ One of the following: #### Syntax ```bash -SAVE +SAVE [DBID] ``` -The SAVE commands performs a synchronous save of the dataset producing a point in time snapshot of all the data inside the Garnet instance. +The SAVE commands performs a synchronous save of the dataset producing a point in time snapshot of all the data inside the Garnet instance. If a DB ID is specified, only the data inside of that database will be snapshotted. #### Resp Reply @@ -42,10 +42,10 @@ Simple string reply: OK. #### Syntax ```bash -LASTSAVE +LASTSAVE [DBID] ``` -Return the UNIX TIME of the last DB save executed with success. +Return the UNIX TIME of the last DB save executed with success for the current database or, if a DB ID is specified, the last DB save executed with success for the specified database. #### Resp Reply diff --git a/website/docs/commands/garnet-specific.md b/website/docs/commands/garnet-specific.md index 699db52412..b4cd7b5445 100644 --- a/website/docs/commands/garnet-specific.md +++ b/website/docs/commands/garnet-specific.md @@ -30,11 +30,10 @@ Simple string reply: OK. #### Syntax ```bash - COMMITAOF + COMMITAOF [DBID] ``` -Issues a manual commit of the append-only-file. This is useful when auto-commits are turned off, but you need the -system to commit at specific times. +Issues a manual commit of the append-only-file (for all active databases in the Garnet instance). This is useful when auto-commits are turned off, but you need the system to commit at specific times. If a DB ID is specified, a manual commit of the append-only-file of that specific database will be issues. #### Resp Reply diff --git a/website/docs/commands/server.md b/website/docs/commands/server.md index 62ba3bdb3c..aa0ed624dd 100644 --- a/website/docs/commands/server.md +++ b/website/docs/commands/server.md @@ -361,6 +361,22 @@ Simple string reply: OK. --- +### SWAPDB + +#### Syntax + +```bash +SWAPDB index1 index2 +``` + +This command swaps two Garnet databases, so that immediately all the clients connected to a given database will see the data of the other database, and the other way around. + +#### Resp Reply + +Simple string reply: OK. + +--- + ### TIME #### Syntax