From a6c98e864724e40d2e9a1b71bd53e71e53c3bd00 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Mon, 15 Sep 2025 22:05:30 -0700 Subject: [PATCH 01/28] wip --- .../ReplicaOps/ReplicaReceiveCheckpoint.cs | 2 +- libs/common/Metrics/InfoMetricsType.cs | 12 - libs/host/GarnetServer.cs | 68 ++--- libs/server/AOF/AofProcessor.cs | 23 +- libs/server/API/GarnetApi.cs | 11 +- libs/server/API/GarnetApiObjectCommands.cs | 11 +- libs/server/Databases/DatabaseManagerBase.cs | 281 +++++------------- .../Databases/DatabaseManagerFactory.cs | 2 +- libs/server/Databases/IDatabaseManager.cs | 20 +- libs/server/Databases/MultiDatabaseManager.cs | 49 +-- .../server/Databases/SingleDatabaseManager.cs | 65 ++-- libs/server/GarnetDatabase.cs | 87 ++---- libs/server/Metrics/Info/GarnetInfoMetrics.cs | 208 +++---------- libs/server/Metrics/Info/InfoHelp.cs | 3 - libs/server/Servers/GarnetServerOptions.cs | 149 +--------- libs/server/SessionParseStateExtensions.cs | 6 - .../Common/ArrayKeyIterationFunctions.cs | 26 +- .../Storage/Session/MainStore/MainStoreOps.cs | 107 ++++--- .../Session/ObjectStore/AdvancedOps.cs | 8 +- .../Storage/Session/ObjectStore/Common.cs | 26 +- .../Session/ObjectStore/CompletePending.cs | 6 +- .../Storage/Session/ObjectStore/HashOps.cs | 60 ++-- .../Storage/Session/ObjectStore/ListOps.cs | 38 +-- .../Storage/Session/ObjectStore/SetOps.cs | 42 +-- .../Session/ObjectStore/SortedSetGeoOps.cs | 12 +- .../Session/ObjectStore/SortedSetOps.cs | 86 +++--- libs/server/Storage/Session/StorageSession.cs | 28 +- .../Storage/SizeTracker/CacheSizeTracker.cs | 18 +- libs/server/StoreWrapper.cs | 34 +-- libs/server/Transaction/TransactionManager.cs | 28 +- libs/server/Transaction/TxnKeyEntry.cs | 11 +- .../Transaction/TxnKeyEntryComparison.cs | 15 +- .../Index/StoreFunctions/StoreFunctions.cs | 8 +- test/Garnet.test/CacheSizeTrackerTests.cs | 14 +- test/Garnet.test/GarnetObjectTests.cs | 8 +- test/Garnet.test/RespConfigTests.cs | 6 +- 36 files changed, 498 insertions(+), 1080 deletions(-) diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs index 36ef01d4520..4a1cde176a4 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs @@ -298,7 +298,7 @@ public long BeginReplicaRecover( errorMessage = []; UpdateLastPrimarySyncTime(); - logger?.LogInformation("Replica Recover MainStore: {storeVersion}>[{sIndexToken} {sHlogToken}]" + + logger?.LogInformation("Replica Recover Store: {storeVersion}>[{sIndexToken} {sHlogToken}]" + "\nObjectStore: {objectStoreVersion}>[{oIndexToken} {oHlogToken}]", remoteCheckpoint.metadata.storeVersion, remoteCheckpoint.metadata.storeIndexToken, diff --git a/libs/common/Metrics/InfoMetricsType.cs b/libs/common/Metrics/InfoMetricsType.cs index e309b4cd9b9..2e68f6798a7 100644 --- a/libs/common/Metrics/InfoMetricsType.cs +++ b/libs/common/Metrics/InfoMetricsType.cs @@ -39,26 +39,14 @@ public enum InfoMetricsType : byte /// STORE, /// - /// Object store info - /// - OBJECTSTORE, - /// /// Store hash table info /// STOREHASHTABLE, /// - /// Object store hash table info - /// - OBJECTSTOREHASHTABLE, - /// /// Store revivification info /// STOREREVIV, /// - /// Object store hash table info - /// - OBJECTSTOREREVIV, - /// /// Persistence information /// PERSISTENCE, diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index f8cc704fb89..a5226422700 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -20,11 +20,8 @@ namespace Garnet { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Implementation Garnet server @@ -49,7 +46,6 @@ static string GetVersion() private IGarnetServer[] servers; private SubscribeBroker subscribeBroker; private KVSettings kvSettings; - private KVSettings objKvSettings; private INamedDeviceFactory logFactory; private MemoryLogger initLogger; private ILogger logger; @@ -300,12 +296,11 @@ private void InitializeServer() private GarnetDatabase CreateDatabase(int dbId, GarnetServerOptions serverOptions, ClusterFactory clusterFactory, CustomCommandManager customCommandManager) { - var store = CreateMainStore(dbId, clusterFactory, out var epoch, out var stateMachineDriver); - var objectStore = CreateObjectStore(dbId, clusterFactory, customCommandManager, epoch, stateMachineDriver, out var objectStoreSizeTracker); + var store = CreateStore(dbId, clusterFactory, customCommandManager, out var epoch, out var stateMachineDriver, out var sizeTracker); var (aofDevice, aof) = CreateAOF(dbId); - return new GarnetDatabase(dbId, store, objectStore, epoch, stateMachineDriver, objectStoreSizeTracker, - aofDevice, aof, serverOptions.AdjustedIndexMaxCacheLines == 0, - serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0); + + return new GarnetDatabase(dbId, store, epoch, stateMachineDriver, sizeTracker, + aofDevice, aof, serverOptions.AdjustedIndexMaxCacheLines == 0); } private void LoadModules(CustomCommandManager customCommandManager) @@ -331,60 +326,36 @@ private void LoadModules(CustomCommandManager customCommandManager) } } - private TsavoriteKV CreateMainStore(int dbId, IClusterFactory clusterFactory, - out LightEpoch epoch, out StateMachineDriver stateMachineDriver) + private TsavoriteKV CreateStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager, + out LightEpoch epoch, out StateMachineDriver stateMachineDriver, out CacheSizeTracker sizeTracker) { + sizeTracker = null; + epoch = new LightEpoch(); stateMachineDriver = new StateMachineDriver(epoch, loggerFactory?.CreateLogger($"StateMachineDriver")); - kvSettings = opts.GetSettings(loggerFactory, epoch, stateMachineDriver, out logFactory); + kvSettings = opts.GetSettings(loggerFactory, epoch, stateMachineDriver, out logFactory, out var heapMemorySize, out var readCacheHeapMemorySize); // Run checkpoint on its own thread to control p99 kvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; - var baseName = opts.GetMainStoreCheckpointDirectory(dbId); + var baseName = opts.GetStoreCheckpointDirectory(dbId); var defaultNamingScheme = new DefaultCheckpointNamingScheme(baseName); kvSettings.CheckpointManager = opts.EnableCluster ? clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: true, logger) : new GarnetCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true); - return new(kvSettings - , StoreFunctions.Create() - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); - } - - private TsavoriteKV CreateObjectStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager, - LightEpoch epoch, StateMachineDriver stateMachineDriver, out CacheSizeTracker objectStoreSizeTracker) - { - objectStoreSizeTracker = null; - if (opts.DisableObjects) - return null; - - objKvSettings = opts.GetObjectStoreSettings(loggerFactory, epoch, stateMachineDriver, - 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 GarnetCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true); - - var objStore = new TsavoriteKV(objKvSettings - , StoreFunctions.Create(new SpanByteComparer(), + var store = new TsavoriteKV(kvSettings + , Tsavorite.core.StoreFunctions.Create(new SpanByteComparer(), () => new GarnetObjectSerializer(customCommandManager)) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); - if (objHeapMemorySize > 0 || objReadCacheHeapMemorySize > 0) - objectStoreSizeTracker = new CacheSizeTracker(objStore, objKvSettings, objHeapMemorySize, objReadCacheHeapMemorySize, + if (heapMemorySize > 0 || readCacheHeapMemorySize > 0) + sizeTracker = new CacheSizeTracker(store, kvSettings, heapMemorySize, readCacheHeapMemorySize, this.loggerFactory); - return objStore; - + return store; } private (IDevice, TsavoriteLog) CreateAOF(int dbId) @@ -454,11 +425,6 @@ private void InternalDispose() servers[i]?.Dispose(); subscribeBroker?.Dispose(); kvSettings.LogDevice?.Dispose(); - if (!opts.DisableObjects) - { - objKvSettings.LogDevice?.Dispose(); - objKvSettings.ObjectLogDevice?.Dispose(); - } opts.AuthSettings?.Dispose(); if (disposeLoggerFactory) loggerFactory?.Dispose(); diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index 679c83bd51c..8cc3bd02163 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -13,11 +13,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information @@ -42,12 +39,12 @@ public sealed unsafe partial class AofProcessor /// /// Session for main store /// - BasicContext basicContext; + BasicContext basicContext; /// /// Session for object store /// - BasicContext objectStoreBasicContext; + BasicContext objectStoreBasicContext; readonly Dictionary> inflightTxns; readonly byte[] buffer; @@ -397,7 +394,7 @@ private void SwitchActiveDatabaseContext(GarnetDatabase db, bool initialSetup = } } - static void StoreUpsert(BasicContext basicContext, + static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr) { var curr = ptr + sizeof(AofHeader); @@ -417,7 +414,7 @@ static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr) + static void StoreRMW(BasicContext basicContext, RawStringInput storeInput, byte* ptr) { var curr = ptr + sizeof(AofHeader); var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); @@ -436,13 +433,13 @@ static void StoreRMW(BasicContext basicContext, byte* ptr) + static void StoreDelete(BasicContext basicContext, byte* ptr) { var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); basicContext.Delete(key); } - static void ObjectStoreUpsert(BasicContext basicContext, + static void ObjectStoreUpsert(BasicContext basicContext, GarnetObjectSerializer garnetObjectSerializer, byte* ptr, byte* outputPtr, int outputLength) { var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); @@ -459,7 +456,7 @@ static void ObjectStoreUpsert(BasicContext basicContext, + static void ObjectStoreRMW(BasicContext basicContext, ObjectInput objectStoreInput, byte* ptr, byte* outputPtr, int outputLength) { var curr = ptr + sizeof(AofHeader); @@ -480,7 +477,7 @@ static void ObjectStoreRMW(BasicContext basicContext, byte* ptr) + static void ObjectStoreDelete(BasicContext basicContext, byte* ptr) { var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); basicContext.Delete(key); diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index 99886b74fd5..7ba856482a0 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -8,11 +8,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; // See TransactionManager.cs for aliases BasicGarnetApi and TransactionalGarnetApi @@ -20,8 +17,8 @@ namespace Garnet.server /// Garnet API implementation /// public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { readonly StorageSession storageSession; TContext context; diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs index 82ed66e00f6..f577c1a1cd3 100644 --- a/libs/server/API/GarnetApiObjectCommands.cs +++ b/libs/server/API/GarnetApiObjectCommands.cs @@ -7,18 +7,15 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Garnet API implementation /// public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { #region SortedSet Methods diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index 2c2e0278703..e50042b2617 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -10,11 +10,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Base class for logical database management @@ -84,7 +81,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, public abstract void ExpiredKeyDeletionScan(); /// - public abstract void StartObjectSizeTrackers(CancellationToken token = default); + public abstract void StartSizeTrackers(CancellationToken token = default); /// public abstract void Reset(int dbId = 0); @@ -120,10 +117,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, public abstract IDatabaseManager Clone(bool enableAof); /// - public TsavoriteKV MainStore => DefaultDatabase.MainStore; - - /// - public TsavoriteKV ObjectStore => DefaultDatabase.ObjectStore; + public TsavoriteKV Store => DefaultDatabase.Store; /// public TsavoriteLog AppendOnlyFile => DefaultDatabase.AppendOnlyFile; @@ -132,7 +126,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, public DateTimeOffset LastSaveTime => DefaultDatabase.LastSaveTime; /// - public CacheSizeTracker ObjectStoreSizeTracker => DefaultDatabase.ObjectStoreSizeTracker; + public CacheSizeTracker SizeTracker => DefaultDatabase.SizeTracker; /// public WatchVersionMap VersionMap => DefaultDatabase.VersionMap; @@ -174,37 +168,14 @@ protected DatabaseManagerBase(StoreWrapper.DatabaseCreatorDelegate createDatabas /// /// Database to recover /// Store version - /// Object store version - protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersion, out long objectStoreVersion) + protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersion) { storeVersion = 0; - objectStoreVersion = 0; - if (db.ObjectStore != null) - { - // Get store recover version - var currStoreVersion = db.MainStore.GetRecoverVersion(); - // Get object store recover version - var currObjectStoreVersion = db.ObjectStore.GetRecoverVersion(); - - // Choose the minimum common recover version for both stores - if (currStoreVersion < currObjectStoreVersion) - currObjectStoreVersion = currStoreVersion; - else if (objectStoreVersion > 0) // handle the case where object store was disabled at checkpointing time - currStoreVersion = currObjectStoreVersion; - - // Recover to the minimum common recover version - storeVersion = db.MainStore.Recover(recoverTo: currStoreVersion); - objectStoreVersion = db.ObjectStore.Recover(recoverTo: currObjectStoreVersion); - Logger?.LogInformation("Recovered store to version {storeVersion} and object store to version {objectStoreVersion}", storeVersion, objectStoreVersion); - } - else - { - storeVersion = db.MainStore.Recover(); - Logger?.LogInformation("Recovered store to version {storeVersion}", storeVersion); - } + storeVersion = db.Store.Recover(); + Logger?.LogInformation("Recovered store to version {storeVersion}", storeVersion); - if (storeVersion > 0 || objectStoreVersion > 0) + if (storeVersion > 0) { db.LastSaveTime = DateTimeOffset.UtcNow; } @@ -217,36 +188,31 @@ protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersio /// Logger /// Cancellation token /// Tuple of store tail address and object store tail address - protected async Task<(long?, long?)> TakeCheckpointAsync(GarnetDatabase db, ILogger logger = null, CancellationToken token = default) + protected async Task TakeCheckpointAsync(GarnetDatabase db, ILogger logger = null, CancellationToken token = default) { try { DoCompaction(db, isFromCheckpoint: true, logger); - var lastSaveStoreTailAddress = db.MainStore.Log.TailAddress; - var lastSaveObjectStoreTailAddress = (db.ObjectStore?.Log.TailAddress).GetValueOrDefault(); + var lastSaveStoreTailAddress = db.Store.Log.TailAddress; var full = db.LastSaveStoreTailAddress == 0 || - lastSaveStoreTailAddress - db.LastSaveStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval || - (db.ObjectStore != null && (db.LastSaveObjectStoreTailAddress == 0 || - lastSaveObjectStoreTailAddress - db.LastSaveObjectStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval)); + lastSaveStoreTailAddress - db.LastSaveStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval; var tryIncremental = StoreWrapper.serverOptions.EnableIncrementalSnapshots; - if (db.MainStore.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit) - tryIncremental = false; - if (db.ObjectStore?.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit) + if (db.Store.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit) tryIncremental = false; var checkpointType = StoreWrapper.serverOptions.UseFoldOverCheckpoints ? CheckpointType.FoldOver : CheckpointType.Snapshot; await InitiateCheckpointAsync(db, full, checkpointType, tryIncremental, logger); - return full ? new(lastSaveStoreTailAddress, lastSaveObjectStoreTailAddress) : (null, null); + return full ? lastSaveStoreTailAddress : null; } catch (Exception ex) { logger?.LogError(ex, "Checkpointing threw exception, DB ID: {id}", db.Id); } - return (null, null); + return null; } /// @@ -310,10 +276,8 @@ protected void ResetDatabase(GarnetDatabase db) { try { - if (db.MainStore.Log.TailAddress > 64) - db.MainStore.Reset(); - if (db.ObjectStore?.Log.TailAddress > 64) - db.ObjectStore?.Reset(); + if (db.Store.Log.TailAddress > 64) + db.Store.Reset(); db.AppendOnlyFile?.Reset(); var lastSave = DateTimeOffset.FromUnixTimeSeconds(0); @@ -353,8 +317,7 @@ protected static void EnqueueDatabaseCommit(GarnetDatabase db, AofEntryType entr /// Truncate AOF log protected static void FlushDatabase(GarnetDatabase db, bool unsafeTruncateLog, bool truncateAof = true) { - db.MainStore.Log.ShiftBeginAddress(db.MainStore.Log.TailAddress, truncateLog: unsafeTruncateLog); - db.ObjectStore?.Log.ShiftBeginAddress(db.ObjectStore.Log.TailAddress, truncateLog: unsafeTruncateLog); + db.Store.Log.ShiftBeginAddress(db.Store.Log.TailAddress, truncateLog: unsafeTruncateLog); if (truncateAof) db.AppendOnlyFile?.TruncateUntil(db.AppendOnlyFile.TailAddress); @@ -369,30 +332,13 @@ protected bool GrowIndexesIfNeeded(GarnetDatabase db) { var indexesMaxedOut = true; - if (!DefaultDatabase.MainStoreIndexMaxedOut) + if (!DefaultDatabase.StoreIndexMaxedOut) { - var dbMainStore = DefaultDatabase.MainStore; - if (GrowIndexIfNeeded(StoreType.Main, - StoreWrapper.serverOptions.AdjustedIndexMaxCacheLines, dbMainStore.OverflowBucketAllocations, + var dbMainStore = DefaultDatabase.Store; + if (GrowIndexIfNeeded(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; + db.StoreIndexMaxedOut = true; } else { @@ -410,15 +356,15 @@ protected bool GrowIndexesIfNeeded(GarnetDatabase db) /// Logger protected void ExecuteObjectCollection(GarnetDatabase db, ILogger logger = null) { - if (db.ObjectStoreCollectionDbStorageSession == null) + if (db.StoreCollectionDbStorageSession == null) { var scratchBufferManager = new ScratchBufferBuilder(); - db.ObjectStoreCollectionDbStorageSession = + db.StoreCollectionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger); } - ExecuteHashCollect(db.ObjectStoreCollectionDbStorageSession); - ExecuteSortedSetCollect(db.ObjectStoreCollectionDbStorageSession); + ExecuteHashCollect(db.StoreCollectionDbStorageSession); + ExecuteSortedSetCollect(db.StoreCollectionDbStorageSession); } /// @@ -427,12 +373,7 @@ protected void ExecuteObjectCollection(GarnetDatabase db, ILogger logger = null) /// Database protected void ExpiredKeyDeletionScan(GarnetDatabase db) { - _ = MainStoreExpiredKeyDeletionScan(db); - - if (StoreWrapper.serverOptions.DisableObjects) - return; - - _ = ObjectStoreExpiredKeyDeletionScan(db); + _ = StoreExpiredKeyDeletionScan(db); } /// @@ -466,32 +407,31 @@ protected void DoCompaction(GarnetDatabase db, bool isFromCheckpoint = false, IL /// 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) + protected bool GrowIndexIfNeeded(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); + $"IndexAutoGrowTask: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}", + 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); + $"IndexAutoGrowTask: overflowCount {{overflowCount}} ratio more than threshold {{indexResizeThreshold}}%. Doubling index size...", + 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); + $"IndexAutoGrowTask: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}", + indexSizeRetriever(), indexMaxSize, overflowCount); return true; } @@ -499,92 +439,47 @@ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objec { if (compactionType == LogCompactionType.None) return; - var mainStoreLog = db.MainStore.Log; + var storeLog = db.Store.Log; var mainStoreMaxLogSize = (1L << StoreWrapper.serverOptions.SegmentSizeBits()) * mainStoreMaxSegments; - if (mainStoreLog.ReadOnlyAddress - mainStoreLog.BeginAddress > mainStoreMaxLogSize) + if (storeLog.ReadOnlyAddress - storeLog.BeginAddress > mainStoreMaxLogSize) { - var readOnlyAddress = mainStoreLog.ReadOnlyAddress; + var readOnlyAddress = storeLog.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); + untilAddress, storeLog.BeginAddress, readOnlyAddress, storeLog.TailAddress); switch (compactionType) { case LogCompactionType.Shift: - mainStoreLog.ShiftBeginAddress(untilAddress, true, compactionForceDelete); + storeLog.ShiftBeginAddress(untilAddress, true, compactionForceDelete); break; case LogCompactionType.Scan: - mainStoreLog.Compact(untilAddress, CompactionType.Scan); + storeLog.Compact(untilAddress, CompactionType.Scan); if (compactionForceDelete) { CompactionCommitAof(db); - mainStoreLog.Truncate(); + storeLog.Truncate(); } break; case LogCompactionType.Lookup: - mainStoreLog.Compact(untilAddress, CompactionType.Lookup); + storeLog.Compact(untilAddress, CompactionType.Lookup); if (compactionForceDelete) { CompactionCommitAof(db); - mainStoreLog.Truncate(); + storeLog.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(untilAddress, CompactionType.Scan); - if (compactionForceDelete) - { - CompactionCommitAof(db); - objectStoreLog.Truncate(); - } - break; - - case LogCompactionType.Lookup: - objectStoreLog.Compact(untilAddress, CompactionType.Lookup); - if (compactionForceDelete) - { - CompactionCommitAof(db); - objectStoreLog.Truncate(); - } - break; - } - - Logger?.LogInformation( - "End object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", - untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress); + "End store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}", + untilAddress, storeLog.BeginAddress, readOnlyAddress, storeLog.TailAddress); } } @@ -642,28 +537,15 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo IStateMachine sm; if (full) { - sm = db.ObjectStore == null ? - Checkpoint.Full(db.MainStore, checkpointType, out checkpointResult.token) : - Checkpoint.Full(db.MainStore, db.ObjectStore, checkpointType, out checkpointResult.token); + sm = Checkpoint.Full(db.Store, checkpointType, out checkpointResult.token); } else { - tryIncremental = tryIncremental && db.MainStore.CanTakeIncrementalCheckpoint(checkpointType, out checkpointResult.token); - if (db.ObjectStore != null) - tryIncremental = tryIncremental && db.ObjectStore.CanTakeIncrementalCheckpoint(checkpointType, out var guid2) && checkpointResult.token == guid2; + tryIncremental = tryIncremental && db.Store.CanTakeIncrementalCheckpoint(checkpointType, out checkpointResult.token); - if (tryIncremental) - { - sm = db.ObjectStore == null ? - Checkpoint.IncrementalHybridLogOnly(db.MainStore, checkpointResult.token) : - Checkpoint.IncrementalHybridLogOnly(db.MainStore, db.ObjectStore, checkpointResult.token); - } - else - { - sm = db.ObjectStore == null ? - Checkpoint.HybridLogOnly(db.MainStore, checkpointType, out checkpointResult.token) : - Checkpoint.HybridLogOnly(db.MainStore, db.ObjectStore, checkpointType, out checkpointResult.token); - } + sm = tryIncremental + ? Checkpoint.IncrementalHybridLogOnly(db.Store, checkpointResult.token) + : Checkpoint.HybridLogOnly(db.Store, checkpointType, out checkpointResult.token); } checkpointResult.success = await db.StateMachineDriver.RunAsync(sm); @@ -680,18 +562,15 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo 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.Store.Log.Scan(db.Store.Log.ReadOnlyAddress, + db.Store.Log.TailAddress, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: true); + while (iter1.GetNext()) { - // 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, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: true); - while (iter1.GetNext()) - { - var valueObject = iter1.ValueObject; - if (valueObject != null) - ((GarnetObjectBase)iter1.ValueObject).serialized = null; - } + var valueObject = iter1.ValueObject; + if (valueObject != null) + ((GarnetObjectBase)iter1.ValueObject).serialized = null; } logger?.LogInformation("Completed checkpoint for DB ID: {id}", db.Id); @@ -716,34 +595,20 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession) /// public abstract (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId); - protected (long numExpiredKeysFound, long totalRecordsScanned) MainStoreExpiredKeyDeletionScan(GarnetDatabase db) + protected (long numExpiredKeysFound, long totalRecordsScanned) StoreExpiredKeyDeletionScan(GarnetDatabase db) { - if (db.MainStoreExpiredKeyDeletionDbStorageSession == null) + if (db.StoreExpiredKeyDeletionDbStorageSession == null) { var scratchBufferManager = new ScratchBufferBuilder(); - db.MainStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger); + db.StoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger); } var scanFrom = StoreWrapper.store.Log.ReadOnlyAddress; var scanUntil = StoreWrapper.store.Log.TailAddress; - (var deletedCount, var totalCount) = db.MainStoreExpiredKeyDeletionDbStorageSession.MainStoreExpiredKeyDeletionScan(scanFrom, scanUntil); - Logger?.LogDebug("Main Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}", deletedCount, totalCount, scanFrom, scanUntil, db.Id); - - return (deletedCount, totalCount); - } - - protected (long numExpiredKeysFound, long totalRecordsScanned) ObjectStoreExpiredKeyDeletionScan(GarnetDatabase db) - { - if (db.ObjectStoreExpiredKeyDeletionDbStorageSession == null) - { - var scratchBufferManager = new ScratchBufferBuilder(); - db.ObjectStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger); - } - - var scanFrom = StoreWrapper.objectStore.Log.ReadOnlyAddress; - var scanUntil = StoreWrapper.objectStore.Log.TailAddress; - (var deletedCount, var totalCount) = db.ObjectStoreExpiredKeyDeletionDbStorageSession.ObjectStoreExpiredKeyDeletionScan(scanFrom, scanUntil); - Logger?.LogDebug("Object Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}", deletedCount, totalCount, scanFrom, scanUntil, db.Id); + var (deletedCount, totalCount) = db.StoreExpiredKeyDeletionDbStorageSession.ExpiredKeyDeletionScan(scanFrom, scanUntil); + Logger?.LogDebug( + "Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}", + deletedCount, totalCount, scanFrom, scanUntil, db.Id); return (deletedCount, totalCount); } @@ -753,16 +618,12 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession) protected (HybridLogScanMetrics mainStore, HybridLogScanMetrics objectStore) CollectHybridLogStatsForDb(GarnetDatabase db) { - FunctionsState functionsState = CreateFunctionsState(); - MainSessionFunctions mainStoreSessionFuncs = new MainSessionFunctions(functionsState); - var mainStoreStats = CollectHybridLogStats(db, db.MainStore, mainStoreSessionFuncs); + var functionsState = CreateFunctionsState(); + var mainStoreSessionFunctions = new MainSessionFunctions(functionsState); + var mainStoreStats = CollectHybridLogStats(db, db.Store, mainStoreSessionFunctions); - HybridLogScanMetrics objectStoreStats = null; - if (ObjectStore != null) - { - ObjectSessionFunctions objectSessionFunctions = new ObjectSessionFunctions(functionsState); - objectStoreStats = CollectHybridLogStats(db, db.ObjectStore, objectSessionFunctions); - } + var objectSessionFunctions = new ObjectSessionFunctions(functionsState); + var objectStoreStats = CollectHybridLogStats(db, db.Store, objectSessionFunctions); return (mainStoreStats, objectStoreStats); } @@ -791,7 +652,7 @@ private HybridLogScanMetrics CollectHybridLogStats= db.MainStore.Log.ReadOnlyAddress ? "Mutable" : "Immutable"; + string region = iter.CurrentAddress >= db.Store.Log.ReadOnlyAddress ? "Mutable" : "Immutable"; string state = "Live"; if (iter.Info.IsSealed) { diff --git a/libs/server/Databases/DatabaseManagerFactory.cs b/libs/server/Databases/DatabaseManagerFactory.cs index a1a9bae89a8..12de2907e44 100644 --- a/libs/server/Databases/DatabaseManagerFactory.cs +++ b/libs/server/Databases/DatabaseManagerFactory.cs @@ -37,7 +37,7 @@ private static bool ShouldCreateMultipleDatabaseManager(GarnetServerOptions serv using (createDatabaseDelegate(0)) { // Check if there are multiple databases to recover from checkpoint - var checkpointParentDir = serverOptions.MainStoreCheckpointBaseDirectory; + var checkpointParentDir = serverOptions.StoreCheckpointBaseDirectory; var checkpointDirBaseName = serverOptions.GetCheckpointDirectoryName(0); if (MultiDatabaseManager.TryGetSavedDatabaseIds(checkpointParentDir, checkpointDirBaseName, diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs index a49ddc6a494..5b658113c0e 100644 --- a/libs/server/Databases/IDatabaseManager.cs +++ b/libs/server/Databases/IDatabaseManager.cs @@ -10,11 +10,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Interface for logical database management @@ -29,12 +26,7 @@ public interface IDatabaseManager : IDisposable /// /// Store (of DB 0) /// - public TsavoriteKV MainStore { get; } - - /// - /// Object store (of DB 0) - /// - public TsavoriteKV ObjectStore { get; } + public TsavoriteKV Store { get; } /// /// AOF (of DB 0) @@ -49,7 +41,7 @@ public interface IDatabaseManager : IDisposable /// /// Object store size tracker (of DB 0) /// - public CacheSizeTracker ObjectStoreSizeTracker { get; } + public CacheSizeTracker SizeTracker { get; } /// /// Version map (of DB 0) @@ -191,9 +183,9 @@ public Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, Cancellati public void ExpiredKeyDeletionScan(); /// - /// Start object size trackers for all active databases + /// Start size trackers for all active databases /// - public void StartObjectSizeTrackers(CancellationToken token = default); + public void StartSizeTrackers(CancellationToken token = default); /// /// Reset diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs index b5df1be96fb..4b7670609d9 100644 --- a/libs/server/Databases/MultiDatabaseManager.cs +++ b/libs/server/Databases/MultiDatabaseManager.cs @@ -93,7 +93,7 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover throw new GarnetException( $"Unexpected call to {nameof(MultiDatabaseManager)}.{nameof(RecoverCheckpoint)} with {nameof(replicaRecover)} == true."); - var checkpointParentDir = StoreWrapper.serverOptions.MainStoreCheckpointBaseDirectory; + var checkpointParentDir = StoreWrapper.serverOptions.StoreCheckpointBaseDirectory; var checkpointDirBaseName = StoreWrapper.serverOptions.GetCheckpointDirectoryName(0); int[] dbIdsToRecover; @@ -122,7 +122,7 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover try { - RecoverDatabaseCheckpoint(db, out storeVersion, out objectStoreVersion); + RecoverDatabaseCheckpoint(db, out storeVersion); } catch (TsavoriteNoHybridLogException ex) { @@ -139,14 +139,6 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover if (StoreWrapper.serverOptions.FailOnRecoveryError) throw; } - - // After recovery, we check if store versions match - if (db.ObjectStore != null && storeVersion != objectStoreVersion) - { - Logger?.LogInformation("Main store and object store checkpoint versions do not match; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion); - if (StoreWrapper.serverOptions.FailOnRecoveryError) - throw new GarnetException("Main store and object store checkpoint versions do not match"); - } } } @@ -210,9 +202,8 @@ public override bool TakeCheckpoint(bool background, int dbId, ILogger logger = { if (t.IsCompletedSuccessfully) { - var storeTailAddress = t.Result.Item1; - var objectStoreTailAddress = t.Result.Item2; - UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress); + var storeTailAddress = t.Result; + UpdateLastSaveData(dbId, storeTailAddress); } } finally @@ -249,9 +240,8 @@ public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, // Necessary to take a checkpoint because the latest checkpoint is before entryTime var result = await TakeCheckpointAsync(db, logger: Logger); - var storeTailAddress = result.Item1; - var objectStoreTailAddress = result.Item2; - UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress); + var storeTailAddress = result; + UpdateLastSaveData(dbId, storeTailAddress); } finally { @@ -575,7 +565,7 @@ public override void ExpiredKeyDeletionScan() } /// - public override void StartObjectSizeTrackers(CancellationToken token = default) + public override void StartSizeTrackers(CancellationToken token = default) { sizeTrackersStarted = true; @@ -595,7 +585,7 @@ public override void StartObjectSizeTrackers(CancellationToken token = default) var db = databasesMapSnapshot[dbId]; Debug.Assert(db != null); - db.ObjectStoreSizeTracker?.Start(token); + db.SizeTracker?.Start(token); } } finally @@ -624,8 +614,7 @@ public override void ResetRevivificationStats() for (var i = 0; i < activeDbIdsMapSize; i++) { var dbId = activeDbIdsMapSnapshot[i]; - databaseMapSnapshot[dbId].MainStore.ResetRevivificationStats(); - databaseMapSnapshot[dbId].ObjectStore?.ResetRevivificationStats(); + databaseMapSnapshot[dbId].Store.ResetRevivificationStats(); } } @@ -711,7 +700,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto if (!success) throw new GarnetException($"Database with ID {dbId} was not found."); - return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper, memoryPool: null, db.ObjectStoreSizeTracker, Logger, respProtocolVersion); + return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper, memoryPool: null, db.SizeTracker, Logger, respProtocolVersion); } /// @@ -931,7 +920,7 @@ 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 (sizeTrackersStarted) - db.ObjectStoreSizeTracker?.Start(StoreWrapper.ctsCommit.Token); + db.SizeTracker?.Start(StoreWrapper.ctsCommit.Token); activeDbIds.TryGetNextId(out var nextIdx); activeDbIds.TrySetValue(nextIdx, db.Id); @@ -1018,9 +1007,8 @@ private async Task TakeDatabasesCheckpointAsync(int dbIdsCount, ILogger lo if (!t.IsCompletedSuccessfully) return; - var storeTailAddress = t.Result.Item1; - var objectStoreTailAddress = t.Result.Item2; - UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress); + var storeTailAddress = t.Result; + UpdateLastSaveData(dbId, storeTailAddress); }, TaskContinuationOptions.ExecuteSynchronously); } @@ -1038,7 +1026,7 @@ private async Task TakeDatabasesCheckpointAsync(int dbIdsCount, ILogger lo return true; } - private void UpdateLastSaveData(int dbId, long? storeTailAddress, long? objectStoreTailAddress) + private void UpdateLastSaveData(int dbId, long? storeTailAddress) { var databasesMapSnapshot = databases.Map; @@ -1048,9 +1036,6 @@ private void UpdateLastSaveData(int dbId, long? storeTailAddress, long? objectSt if (storeTailAddress.HasValue) { db.LastSaveStoreTailAddress = storeTailAddress.Value; - - if (db.ObjectStore != null && objectStoreTailAddress.HasValue) - db.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value; } } @@ -1070,11 +1055,7 @@ public override void Dispose() } public override (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId) - { - var (k1, t1) = MainStoreExpiredKeyDeletionScan(GetDbById(dbId)); - var (k2, t2) = StoreWrapper.serverOptions.DisableObjects ? (0, 0) : ObjectStoreExpiredKeyDeletionScan(GetDbById(dbId)); - return (k1 + k2, t1 + t2); - } + => StoreExpiredKeyDeletionScan(GetDbById(dbId)); private GarnetDatabase GetDbById(int dbId) { diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs index 0a4b6053713..1fa136f33e8 100644 --- a/libs/server/Databases/SingleDatabaseManager.cs +++ b/libs/server/Databases/SingleDatabaseManager.cs @@ -4,7 +4,6 @@ using System; using System.Threading; using System.Threading.Tasks; -using Garnet.common; using Garnet.server.Metrics; using Microsoft.Extensions.Logging; using Tsavorite.core; @@ -56,7 +55,7 @@ public override GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, o /// public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) { - long storeVersion = 0, objectStoreVersion = 0; + long storeVersion = 0; try { if (replicaRecover) @@ -64,49 +63,33 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover // 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); + storeVersion = !recoverMainStoreFromToken ? Store.Recover() : Store.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) + if (storeVersion > 0) defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; } else { - RecoverDatabaseCheckpoint(defaultDatabase, out storeVersion, out objectStoreVersion); + RecoverDatabaseCheckpoint(defaultDatabase, out storeVersion); } } 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); + "No Hybrid Log found for recovery; storeVersion = {storeVersion};", + storeVersion); } catch (Exception ex) { Logger?.LogInformation(ex, - "Error during recovery of store; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", - storeVersion, objectStoreVersion); + "Error during recovery of store; storeVersion = {storeVersion};", + storeVersion); if (StoreWrapper.serverOptions.FailOnRecoveryError) throw; } - - // After recovery, we check if store versions match - if (ObjectStore != null && storeVersion != objectStoreVersion) - { - Logger?.LogInformation("Main store and object store checkpoint versions do not match; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion); - if (StoreWrapper.serverOptions.FailOnRecoveryError) - throw new GarnetException("Main store and object store checkpoint versions do not match"); - } } /// @@ -139,13 +122,10 @@ public override bool TakeCheckpoint(bool background, ILogger logger = null, Canc { if (t.IsCompletedSuccessfully) { - var storeTailAddress = t.Result.Item1; - var objectStoreTailAddress = t.Result.Item2; + var storeTailAddress = t.Result; if (storeTailAddress.HasValue) defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value; - if (ObjectStore != null && objectStoreTailAddress.HasValue) - defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value; defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; } @@ -189,14 +169,11 @@ public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, // Necessary to take a checkpoint because the latest checkpoint is before entryTime var result = await TakeCheckpointAsync(defaultDatabase, logger: Logger); - var storeTailAddress = result.Item1; - var objectStoreTailAddress = result.Item2; + var storeTailAddress = result; if (storeTailAddress.HasValue) defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value; - if (ObjectStore != null && objectStoreTailAddress.HasValue) - defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value; - + defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; } finally @@ -222,13 +199,10 @@ public override async Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLi { var result = await TakeCheckpointAsync(defaultDatabase, logger: logger, token: token); - var storeTailAddress = result.Item1; - var objectStoreTailAddress = result.Item2; + var storeTailAddress = result; if (storeTailAddress.HasValue) defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value; - if (ObjectStore != null && objectStoreTailAddress.HasValue) - defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value; defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; } @@ -307,8 +281,8 @@ public override void ExpiredKeyDeletionScan() => ExpiredKeyDeletionScan(defaultDatabase); /// - public override void StartObjectSizeTrackers(CancellationToken token = default) => - ObjectStoreSizeTracker?.Start(token); + public override void StartSizeTrackers(CancellationToken token = default) => + SizeTracker?.Start(token); /// public override void Reset(int dbId = 0) @@ -320,10 +294,7 @@ public override void Reset(int dbId = 0) /// public override void ResetRevivificationStats() - { - MainStore.ResetRevivificationStats(); - ObjectStore?.ResetRevivificationStats(); - } + => Store.ResetRevivificationStats(); /// public override void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0) @@ -379,7 +350,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto { ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); - return new(AppendOnlyFile, VersionMap, StoreWrapper, null, ObjectStoreSizeTracker, Logger, respProtocolVersion); + return new(AppendOnlyFile, VersionMap, StoreWrapper, null, SizeTracker, Logger, respProtocolVersion); } private async Task TryPauseCheckpointsContinuousAsync(int dbId, @@ -401,9 +372,7 @@ private async Task TryPauseCheckpointsContinuousAsync(int dbId, public override (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId) { ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); - var (k1, t1) = MainStoreExpiredKeyDeletionScan(DefaultDatabase); - var (k2, t2) = StoreWrapper.serverOptions.DisableObjects ? (0, 0) : ObjectStoreExpiredKeyDeletionScan(DefaultDatabase); - return (k1 + k2, t1 + t2); + return StoreExpiredKeyDeletionScan(DefaultDatabase); } public override (HybridLogScanMetrics mainStore, HybridLogScanMetrics objectStore)[] CollectHybridLogStats() => [CollectHybridLogStatsForDb(defaultDatabase)]; diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs index f09b1016681..502dee4660f 100644 --- a/libs/server/GarnetDatabase.cs +++ b/libs/server/GarnetDatabase.cs @@ -8,11 +8,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Represents a logical database in Garnet @@ -31,14 +28,9 @@ public class GarnetDatabase : IDisposable public int Id { get; } /// - /// Main Store - /// - public TsavoriteKV MainStore { get; } - - /// - /// Object Store + /// Store /// - public TsavoriteKV ObjectStore { get; } + public TsavoriteKV Store { get; } /// /// Epoch instance used by server @@ -51,9 +43,9 @@ public class GarnetDatabase : IDisposable public StateMachineDriver StateMachineDriver { get; } /// - /// Size Tracker for Object Store + /// Size Tracker /// - public CacheSizeTracker ObjectStoreSizeTracker { get; } + public CacheSizeTracker SizeTracker { get; } /// /// Device used for AOF logging @@ -71,29 +63,19 @@ public class GarnetDatabase : IDisposable public WatchVersionMap VersionMap { get; } /// - /// Tail address of main store log at last save + /// Tail address of 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 + /// True if database's store index has maxed-out /// - public bool MainStoreIndexMaxedOut; - - /// - /// True if database's object store index has maxed-out - /// - public bool ObjectStoreIndexMaxedOut; + public bool StoreIndexMaxedOut; /// /// Reader-Writer lock for database checkpointing @@ -103,59 +85,47 @@ public class GarnetDatabase : IDisposable /// /// Storage session intended for store-wide object collection operations /// - internal StorageSession ObjectStoreCollectionDbStorageSession; + internal StorageSession StoreCollectionDbStorageSession; /// - /// Storage session intended for main-store expired key deletion operations + /// Storage session intended for store expired key deletion operations /// - internal StorageSession MainStoreExpiredKeyDeletionDbStorageSession; - - /// - /// Storage session intended for object-store expired key deletion operations - /// - internal StorageSession ObjectStoreExpiredKeyDeletionDbStorageSession; - + internal StorageSession StoreExpiredKeyDeletionDbStorageSession; internal StorageSession HybridLogStatScanStorageSession; bool disposed = false; - public GarnetDatabase(int id, TsavoriteKV mainStore, - TsavoriteKV objectStore, + public GarnetDatabase(int id, TsavoriteKV store, LightEpoch epoch, StateMachineDriver stateMachineDriver, - CacheSizeTracker objectStoreSizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile, - bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut) : this() + CacheSizeTracker sizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile, + bool storeIndexMaxedOut) : this() { Id = id; - MainStore = mainStore; - ObjectStore = objectStore; + Store = store; Epoch = epoch; StateMachineDriver = stateMachineDriver; - ObjectStoreSizeTracker = objectStoreSizeTracker; + SizeTracker = sizeTracker; AofDevice = aofDevice; AppendOnlyFile = appendOnlyFile; - MainStoreIndexMaxedOut = mainStoreIndexMaxedOut; - ObjectStoreIndexMaxedOut = objectStoreIndexMaxedOut; + StoreIndexMaxedOut = storeIndexMaxedOut; } public GarnetDatabase(int id, GarnetDatabase srcDb, bool enableAof, bool copyLastSaveData = false) : this() { Id = id; - MainStore = srcDb.MainStore; - ObjectStore = srcDb.ObjectStore; + Store = srcDb.Store; Epoch = srcDb.Epoch; StateMachineDriver = srcDb.StateMachineDriver; - ObjectStoreSizeTracker = srcDb.ObjectStoreSizeTracker; + SizeTracker = srcDb.SizeTracker; AofDevice = enableAof ? srcDb.AofDevice : null; AppendOnlyFile = enableAof ? srcDb.AppendOnlyFile : null; - MainStoreIndexMaxedOut = srcDb.MainStoreIndexMaxedOut; - ObjectStoreIndexMaxedOut = srcDb.ObjectStoreIndexMaxedOut; + StoreIndexMaxedOut = srcDb.StoreIndexMaxedOut; if (copyLastSaveData) { LastSaveTime = srcDb.LastSaveTime; LastSaveStoreTailAddress = srcDb.LastSaveStoreTailAddress; - LastSaveObjectStoreTailAddress = srcDb.LastSaveObjectStoreTailAddress; } } @@ -163,7 +133,6 @@ public GarnetDatabase() { VersionMap = new WatchVersionMap(DefaultVersionMapSize); LastSaveStoreTailAddress = 0; - LastSaveObjectStoreTailAddress = 0; LastSaveTime = DateTimeOffset.FromUnixTimeSeconds(0); } @@ -177,20 +146,18 @@ public void Dispose() // Wait for checkpoints to complete and disable checkpointing CheckpointingLock.CloseLock(); - MainStore?.Dispose(); - ObjectStore?.Dispose(); + Store?.Dispose(); AofDevice?.Dispose(); AppendOnlyFile?.Dispose(); - ObjectStoreCollectionDbStorageSession?.Dispose(); - MainStoreExpiredKeyDeletionDbStorageSession?.Dispose(); - ObjectStoreExpiredKeyDeletionDbStorageSession?.Dispose(); + StoreCollectionDbStorageSession?.Dispose(); + StoreExpiredKeyDeletionDbStorageSession?.Dispose(); - if (ObjectStoreSizeTracker != null) + if (SizeTracker != null) { // If tracker has previously started, wait for it to stop - if (!ObjectStoreSizeTracker.TryPreventStart()) + if (!SizeTracker.TryPreventStart()) { - while (!ObjectStoreSizeTracker.Stopped) + while (!SizeTracker.Stopped) Thread.Yield(); } } diff --git a/libs/server/Metrics/Info/GarnetInfoMetrics.cs b/libs/server/Metrics/Info/GarnetInfoMetrics.cs index 59ea3a4150f..fec79cdeb57 100644 --- a/libs/server/Metrics/Info/GarnetInfoMetrics.cs +++ b/libs/server/Metrics/Info/GarnetInfoMetrics.cs @@ -18,9 +18,7 @@ class GarnetInfoMetrics .Where(e => e switch { InfoMetricsType.STOREHASHTABLE => false, - InfoMetricsType.OBJECTSTOREHASHTABLE => false, InfoMetricsType.STOREREVIV => false, - InfoMetricsType.OBJECTSTOREREVIV => false, InfoMetricsType.HLOGSCAN => false, _ => true })]; @@ -31,11 +29,8 @@ class GarnetInfoMetrics MetricsItem[] replicationInfo = null; MetricsItem[] statsInfo = null; MetricsItem[][] storeInfo = null; - MetricsItem[][] objectStoreInfo = null; MetricsItem[][] storeHashDistrInfo = null; - MetricsItem[][] objectStoreHashDistrInfo = null; MetricsItem[][] storeRevivInfo = null; - MetricsItem[][] objectStoreRevivInfo = null; MetricsItem[][] persistenceInfo = null; MetricsItem[] clientsInfo = null; MetricsItem[] keyspaceInfo = null; @@ -65,21 +60,13 @@ private void PopulateServerInfo(StoreWrapper storeWrapper) private void PopulateMemoryInfo(StoreWrapper storeWrapper) { - var main_store_index_size = 0L; - var main_store_log_memory_size = 0L; - var main_store_read_cache_size = 0L; - long total_main_store_size; - - var disableObj = storeWrapper.serverOptions.DisableObjects; - - var initialSize = disableObj ? -1L : 0L; - var object_store_index_size = initialSize; - var object_store_log_memory_size = initialSize; - var object_store_read_cache_log_memory_size = initialSize; - var object_store_heap_memory_target_size = initialSize; - var object_store_heap_memory_size = initialSize; - var object_store_read_cache_heap_memory_size = initialSize; - var total_object_store_size = initialSize; + var store_index_size = 0L; + var store_log_memory_size = 0L; + var store_read_cache_size = 0L; + var store_heap_memory_target_size = 0L; + var store_heap_memory_size = 0L; + var store_read_cache_heap_memory_size = 0L; + long total_store_size; var enableAof = storeWrapper.serverOptions.EnableAOF; var aof_log_memory_size = enableAof ? 0 : -1L; @@ -88,31 +75,19 @@ private void PopulateMemoryInfo(StoreWrapper storeWrapper) foreach (var db in databases) { - 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; + store_index_size += db.Store.IndexSize * 64; + store_log_memory_size += db.Store.Log.MemorySizeBytes; + store_read_cache_size += db.Store.ReadCache?.MemorySizeBytes ?? 0; aof_log_memory_size += db.AppendOnlyFile?.MemorySizeBytes ?? 0; - 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_target_size += db.ObjectStoreSizeTracker?.mainLogTracker.TargetSize ?? 0; - object_store_heap_memory_size += db.ObjectStoreSizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0; - object_store_read_cache_heap_memory_size += db.ObjectStoreSizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0; - } + store_heap_memory_target_size += db.SizeTracker?.mainLogTracker.TargetSize ?? 0; + store_heap_memory_size += db.SizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0; + store_read_cache_heap_memory_size += db.SizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0; } - total_main_store_size = main_store_index_size + main_store_log_memory_size + main_store_read_cache_size; - - if (!disableObj) - { - 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; - } + total_store_size = store_index_size + store_log_memory_size + store_read_cache_size + + store_heap_memory_size + store_read_cache_heap_memory_size; var gcMemoryInfo = GC.GetGCMemoryInfo(); var gcAvailableMemory = gcMemoryInfo.TotalCommittedBytes - gcMemoryInfo.HeapSizeBytes; @@ -144,17 +119,13 @@ private void PopulateMemoryInfo(StoreWrapper storeWrapper) new("gc_heap_bytes", gcMemoryInfo.HeapSizeBytes.ToString()), new("gc_managed_memory_bytes_excluding_heap", gcAvailableMemory.ToString()), new("gc_fragmented_bytes", gcMemoryInfo.FragmentedBytes.ToString()), - new("main_store_index_size", main_store_index_size.ToString()), - new("main_store_log_memory_size", main_store_log_memory_size.ToString()), - new("main_store_read_cache_size", main_store_read_cache_size.ToString()), - new("total_main_store_size", total_main_store_size.ToString()), - new("object_store_index_size", object_store_index_size.ToString()), - new("object_store_log_memory_size", object_store_log_memory_size.ToString()), - new("object_store_heap_memory_target_size", object_store_heap_memory_target_size.ToString()), - new("object_store_heap_memory_size", object_store_heap_memory_size.ToString()), - new("object_store_read_cache_log_memory_size", object_store_read_cache_log_memory_size.ToString()), - new("object_store_read_cache_heap_memory_size", object_store_read_cache_heap_memory_size.ToString()), - new("total_object_store_size", total_object_store_size.ToString()), + new("store_index_size", store_index_size.ToString()), + new("store_log_memory_size", store_log_memory_size.ToString()), + new("store_read_cache_size", store_read_cache_size.ToString()), + new("total_main_store_size", total_store_size.ToString()), + new("store_heap_memory_target_size", store_heap_memory_target_size.ToString()), + new("store_heap_memory_size", store_heap_memory_size.ToString()), + new("store_read_cache_heap_memory_size", store_read_cache_heap_memory_size.ToString()), new("aof_memory_size", aof_log_memory_size.ToString()) ]; } @@ -179,8 +150,6 @@ private void PopulateReplicationInfo(StoreWrapper storeWrapper) new("second_repl_offset", "N/A"), new("store_current_safe_aof_address", "N/A"), new("store_recovered_safe_aof_address", "N/A"), - new("object_store_current_safe_aof_address", "N/A"), - new("object_store_recovered_safe_aof_address", "N/A") ]; } else @@ -244,60 +213,25 @@ private void PopulateStoreStats(StoreWrapper storeWrapper) private MetricsItem[] GetDatabaseStoreStats(StoreWrapper storeWrapper, GarnetDatabase db) => [ - new($"CurrentVersion", db.MainStore.CurrentVersion.ToString()), - new($"LastCheckpointedVersion", db.MainStore.LastCheckpointedVersion.ToString()), - new($"SystemState", db.MainStore.SystemState.ToString()), - new($"IndexSize", db.MainStore.IndexSize.ToString()), - new($"LogDir", storeWrapper.serverOptions.LogDir), - new($"Log.BeginAddress", db.MainStore.Log.BeginAddress.ToString()), - new($"Log.BufferSize", db.MainStore.Log.BufferSize.ToString()), - new($"Log.EmptyPageCount", db.MainStore.Log.EmptyPageCount.ToString()), - new($"Log.MinEmptyPageCount", db.MainStore.Log.MinEmptyPageCount.ToString()), - new($"Log.HeadAddress", db.MainStore.Log.HeadAddress.ToString()), - new($"Log.MemorySizeBytes", db.MainStore.Log.MemorySizeBytes.ToString()), - new($"Log.SafeReadOnlyAddress", db.MainStore.Log.SafeReadOnlyAddress.ToString()), - new($"Log.TailAddress", db.MainStore.Log.TailAddress.ToString()), - new($"ReadCache.BeginAddress", db.MainStore.ReadCache?.BeginAddress.ToString() ?? "N/A"), - new($"ReadCache.BufferSize", db.MainStore.ReadCache?.BufferSize.ToString() ?? "N/A"), - new($"ReadCache.EmptyPageCount", db.MainStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), - new($"ReadCache.HeadAddress", db.MainStore.ReadCache?.HeadAddress.ToString() ?? "N/A"), - new($"ReadCache.MemorySizeBytes", db.MainStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), - new($"ReadCache.TailAddress", db.MainStore.ReadCache?.TailAddress.ToString() ?? "N/A"), - ]; - - private void PopulateObjectStoreStats(StoreWrapper storeWrapper) - { - var databases = storeWrapper.GetDatabasesSnapshot(); - - objectStoreInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][]; - foreach (var db in databases) - { - var storeStats = GetDatabaseObjectStoreStats(storeWrapper, db); - objectStoreInfo[db.Id] = storeStats; - } - } - - private MetricsItem[] GetDatabaseObjectStoreStats(StoreWrapper storeWrapper, GarnetDatabase db) => - [ - new($"CurrentVersion", db.ObjectStore.CurrentVersion.ToString()), - new($"LastCheckpointedVersion", db.ObjectStore.LastCheckpointedVersion.ToString()), - new($"SystemState", db.ObjectStore.SystemState.ToString()), - new($"IndexSize", db.ObjectStore.IndexSize.ToString()), + new($"CurrentVersion", db.Store.CurrentVersion.ToString()), + new($"LastCheckpointedVersion", db.Store.LastCheckpointedVersion.ToString()), + new($"SystemState", db.Store.SystemState.ToString()), + new($"IndexSize", db.Store.IndexSize.ToString()), new($"LogDir", storeWrapper.serverOptions.LogDir), - new($"Log.BeginAddress", db.ObjectStore.Log.BeginAddress.ToString()), - new($"Log.BufferSize", db.ObjectStore.Log.BufferSize.ToString()), - new($"Log.EmptyPageCount", db.ObjectStore.Log.EmptyPageCount.ToString()), - new($"Log.MinEmptyPageCount", db.ObjectStore.Log.MinEmptyPageCount.ToString()), - new($"Log.HeadAddress", db.ObjectStore.Log.HeadAddress.ToString()), - new($"Log.MemorySizeBytes", db.ObjectStore.Log.MemorySizeBytes.ToString()), - new($"Log.SafeReadOnlyAddress", db.ObjectStore.Log.SafeReadOnlyAddress.ToString()), - new($"Log.TailAddress", db.ObjectStore.Log.TailAddress.ToString()), - new($"ReadCache.BeginAddress", db.ObjectStore.ReadCache?.BeginAddress.ToString() ?? "N/A"), - new($"ReadCache.BufferSize", db.ObjectStore.ReadCache?.BufferSize.ToString() ?? "N/A"), - new($"ReadCache.EmptyPageCount", db.ObjectStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), - new($"ReadCache.HeadAddress", db.ObjectStore.ReadCache?.HeadAddress.ToString() ?? "N/A"), - new($"ReadCache.MemorySizeBytes", db.ObjectStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), - new($"ReadCache.TailAddress", db.ObjectStore.ReadCache?.TailAddress.ToString() ?? "N/A"), + new($"Log.BeginAddress", db.Store.Log.BeginAddress.ToString()), + new($"Log.BufferSize", db.Store.Log.BufferSize.ToString()), + new($"Log.EmptyPageCount", db.Store.Log.EmptyPageCount.ToString()), + new($"Log.MinEmptyPageCount", db.Store.Log.MinEmptyPageCount.ToString()), + new($"Log.HeadAddress", db.Store.Log.HeadAddress.ToString()), + new($"Log.MemorySizeBytes", db.Store.Log.MemorySizeBytes.ToString()), + new($"Log.SafeReadOnlyAddress", db.Store.Log.SafeReadOnlyAddress.ToString()), + new($"Log.TailAddress", db.Store.Log.TailAddress.ToString()), + new($"ReadCache.BeginAddress", db.Store.ReadCache?.BeginAddress.ToString() ?? "N/A"), + new($"ReadCache.BufferSize", db.Store.ReadCache?.BufferSize.ToString() ?? "N/A"), + new($"ReadCache.EmptyPageCount", db.Store.ReadCache?.EmptyPageCount.ToString() ?? "N/A"), + new($"ReadCache.HeadAddress", db.Store.ReadCache?.HeadAddress.ToString() ?? "N/A"), + new($"ReadCache.MemorySizeBytes", db.Store.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"), + new($"ReadCache.TailAddress", db.Store.ReadCache?.TailAddress.ToString() ?? "N/A"), ]; private void PopulateStoreHashDistribution(StoreWrapper storeWrapper) @@ -307,18 +241,7 @@ private void PopulateStoreHashDistribution(StoreWrapper storeWrapper) storeHashDistrInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][]; foreach (var db in databases) { - storeHashDistrInfo[db.Id] = [new("", db.MainStore.DumpDistribution())]; - } - } - - private void PopulateObjectStoreHashDistribution(StoreWrapper storeWrapper) - { - var databases = storeWrapper.GetDatabasesSnapshot(); - - objectStoreHashDistrInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][]; - foreach (var db in databases) - { - objectStoreHashDistrInfo[db.Id] = [new("", db.ObjectStore.DumpDistribution())]; + storeHashDistrInfo[db.Id] = [new("", db.Store.DumpDistribution())]; } } @@ -329,18 +252,7 @@ private void PopulateStoreRevivInfo(StoreWrapper storeWrapper) storeRevivInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][]; foreach (var db in databases) { - storeRevivInfo[db.Id] = [new("", db.MainStore.DumpRevivificationStats())]; - } - } - - private void PopulateObjectStoreRevivInfo(StoreWrapper storeWrapper) - { - var databases = storeWrapper.GetDatabasesSnapshot(); - - objectStoreRevivInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][]; - foreach (var db in databases) - { - objectStoreRevivInfo[db.Id] = [new("", db.ObjectStore.DumpRevivificationStats())]; + storeRevivInfo[db.Id] = [new("", db.Store.DumpRevivificationStats())]; } } @@ -430,11 +342,8 @@ public static string GetSectionHeader(InfoMetricsType infoType, int dbId) InfoMetricsType.REPLICATION => "Replication", InfoMetricsType.STATS => "Stats", InfoMetricsType.STORE => $"MainStore_DB_{dbId}", - InfoMetricsType.OBJECTSTORE => $"ObjectStore_DB_{dbId}", InfoMetricsType.STOREHASHTABLE => $"MainStoreHashTableDistribution_DB_{dbId}", - InfoMetricsType.OBJECTSTOREHASHTABLE => $"ObjectStoreHashTableDistribution_DB_{dbId}", InfoMetricsType.STOREREVIV => $"MainStoreDeletedRecordRevivification_DB_{dbId}", - InfoMetricsType.OBJECTSTOREREVIV => $"ObjectStoreDeletedRecordRevivification_DB_{dbId}", InfoMetricsType.PERSISTENCE => $"Persistence_DB_{dbId}", InfoMetricsType.CLIENTS => "Clients", InfoMetricsType.KEYSPACE => "Keyspace", @@ -495,32 +404,14 @@ private void GetRespInfo(InfoMetricsType section, int dbId, StoreWrapper storeWr PopulateStoreStats(storeWrapper); GetSectionRespInfo(header, storeInfo[dbId], sbResponse); return; - case InfoMetricsType.OBJECTSTORE: - if (storeWrapper.serverOptions.DisableObjects) - return; - PopulateObjectStoreStats(storeWrapper); - GetSectionRespInfo(header, objectStoreInfo[dbId], sbResponse); - return; case InfoMetricsType.STOREHASHTABLE: PopulateStoreHashDistribution(storeWrapper); GetSectionRespInfo(header, storeHashDistrInfo[dbId], sbResponse); return; - case InfoMetricsType.OBJECTSTOREHASHTABLE: - if (storeWrapper.serverOptions.DisableObjects) - return; - PopulateObjectStoreHashDistribution(storeWrapper); - GetSectionRespInfo(header, objectStoreHashDistrInfo[dbId], sbResponse); - return; case InfoMetricsType.STOREREVIV: PopulateStoreRevivInfo(storeWrapper); GetSectionRespInfo(header, storeRevivInfo[dbId], sbResponse); return; - case InfoMetricsType.OBJECTSTOREREVIV: - if (storeWrapper.serverOptions.DisableObjects) - return; - PopulateObjectStoreRevivInfo(storeWrapper); - GetSectionRespInfo(header, objectStoreRevivInfo[dbId], sbResponse); - return; case InfoMetricsType.PERSISTENCE: if (!storeWrapper.serverOptions.EnableAOF) return; @@ -591,27 +482,12 @@ private MetricsItem[] GetMetricInternal(InfoMetricsType section, int dbId, Store case InfoMetricsType.STORE: PopulateStoreStats(storeWrapper); return storeInfo[dbId]; - case InfoMetricsType.OBJECTSTORE: - if (storeWrapper.serverOptions.DisableObjects) - return null; - PopulateObjectStoreStats(storeWrapper); - return objectStoreInfo[dbId]; case InfoMetricsType.STOREHASHTABLE: PopulateStoreHashDistribution(storeWrapper); return storeHashDistrInfo[dbId]; - case InfoMetricsType.OBJECTSTOREHASHTABLE: - if (storeWrapper.serverOptions.DisableObjects) - return null; - PopulateObjectStoreHashDistribution(storeWrapper); - return objectStoreHashDistrInfo[dbId]; case InfoMetricsType.STOREREVIV: PopulateStoreRevivInfo(storeWrapper); return storeRevivInfo[dbId]; - case InfoMetricsType.OBJECTSTOREREVIV: - if (storeWrapper.serverOptions.DisableObjects) - return null; - PopulateObjectStoreRevivInfo(storeWrapper); - return objectStoreRevivInfo[dbId]; case InfoMetricsType.PERSISTENCE: if (!storeWrapper.serverOptions.EnableAOF) return null; diff --git a/libs/server/Metrics/Info/InfoHelp.cs b/libs/server/Metrics/Info/InfoHelp.cs index 47cc24a3460..656f71c5015 100644 --- a/libs/server/Metrics/Info/InfoHelp.cs +++ b/libs/server/Metrics/Info/InfoHelp.cs @@ -23,11 +23,8 @@ public static List GetInfoTypeHelpMessage() $"{nameof(InfoMetricsType.REPLICATION)}: Replication info.", $"{nameof(InfoMetricsType.STATS)}: General server operational stats.", $"{nameof(InfoMetricsType.STORE)}: Main store operational information.", - $"{nameof(InfoMetricsType.OBJECTSTORE)}: Object store operational information.", $"{nameof(InfoMetricsType.STOREHASHTABLE)}: Hash table distribution info for main store (expensive, not returned by default).", - $"{nameof(InfoMetricsType.OBJECTSTOREHASHTABLE)}: Hash table distribution info for object store (expensive, not returned by default).", $"{nameof(InfoMetricsType.STOREREVIV)}: Revivification info for deleted records in main store (not returned by default).", - $"{nameof(InfoMetricsType.OBJECTSTOREREVIV)}: Record revivification info for deleted records in object store (not returned by default).", $"{nameof(InfoMetricsType.PERSISTENCE)}: Persistence related information (i.e. Checkpoint and AOF).", $"{nameof(InfoMetricsType.CLIENTS)}: Information related to client connections.", $"{nameof(InfoMetricsType.KEYSPACE)}: Database related statistics.", diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 86b9446dabb..d391136c7ca 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -504,12 +504,7 @@ public class GarnetServerOptions : ServerOptions /// /// 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"); + public string StoreCheckpointBaseDirectory => Path.Combine(CheckpointBaseDirectory, "Store"); /// /// Seconds between attempts to re-establish replication between a Primary and Replica if the replication connection @@ -535,20 +530,12 @@ public class GarnetServerOptions : ServerOptions 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 + /// Get the directory for database checkpoints /// /// Database Id /// Directory - public string GetObjectStoreCheckpointDirectory(int dbId) => - Path.Combine(ObjectStoreCheckpointBaseDirectory, GetCheckpointDirectoryName(dbId)); + public string GetStoreCheckpointDirectory(int dbId) => + Path.Combine(StoreCheckpointBaseDirectory, GetCheckpointDirectoryName(dbId)); /// /// Gets the base directory for storing AOF commits @@ -593,11 +580,15 @@ public void Initialize(ILoggerFactory loggerFactory = null) /// Epoch instance used by server /// Common state machine driver used by Garnet /// Tsavorite Log factory instance + /// Heap memory size + /// Read cache heap memory size /// /// public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, - out INamedDeviceFactory logFactory) + out INamedDeviceFactory logFactory, out long heapMemorySize, out long readCacheHeapMemorySize) { + readCacheHeapMemorySize = 0; + if (MutablePercent is < 10 or > 95) throw new Exception("MutablePercent must be between 10 and 95"); @@ -616,6 +607,7 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, St }; logger?.LogInformation("[Store] Using page size of {PageSize}", PrettySize(kvSettings.PageSize)); + logger?.LogInformation("[Store] Each page can hold ~{PageSize} key-value pairs of objects", kvSettings.PageSize / 24); kvSettings.MemorySize = 1L << MemorySizeBits(MemorySize, PageSize, out var storeEmptyPageCount); kvSettings.MinEmptyPageCount = storeEmptyPageCount; @@ -651,6 +643,9 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, St if (LatencyMonitor && MetricsSamplingFrequency == 0) throw new Exception("LatencyMonitor requires MetricsSamplingFrequency to be set"); + heapMemorySize = ParseSize(ObjectStoreHeapMemorySize, out _); + logger?.LogInformation("[Store] Heap memory size is {heapMemorySize}", heapMemorySize > 0 ? PrettySize(heapMemorySize) : "unlimited"); + // Read cache related settings if (EnableReadCache && !EnableStorageTier) { @@ -664,6 +659,9 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, St kvSettings.ReadCacheMemorySize = ParseSize(ReadCacheMemorySize, out _); logger?.LogInformation("[Store] Read cache enabled with page size of {ReadCachePageSize} and memory size of {ReadCacheMemorySize}", PrettySize(kvSettings.ReadCachePageSize), PrettySize(kvSettings.ReadCacheMemorySize)); + + readCacheHeapMemorySize = ParseSize(ObjectStoreReadCacheHeapMemorySize, out _); + logger?.LogInformation("[Store] Read cache heap memory size is {readCacheHeapMemorySize}", readCacheHeapMemorySize > 0 ? PrettySize(readCacheHeapMemorySize) : "unlimited"); } if (EnableStorageTier) @@ -751,121 +749,6 @@ public static int MemorySizeBits(string memorySize, string storePageSize, out in return (int)Math.Log(adjustedSize, 2); } - /// - /// Get KVSettings for the object store log - /// - public KVSettings GetObjectStoreSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, - out long objHeapMemorySize, out long objReadCacheHeapMemorySize) - { - objReadCacheHeapMemorySize = default; - - if (ObjectStoreMutablePercent is < 10 or > 95) - throw new Exception("ObjectStoreMutablePercent must be between 10 and 95"); - - var indexCacheLines = IndexSizeCachelines("object store hash index size", ObjectStoreIndexSize); - KVSettings kvSettings = new() - { - IndexSize = indexCacheLines * 64L, - PreallocateLog = false, - MutableFraction = ObjectStoreMutablePercent / 100.0, - PageSize = 1L << ObjectStorePageSizeBits(), - Epoch = epoch, - StateMachineDriver = stateMachineDriver, - loggerFactory = loggerFactory, - logger = loggerFactory?.CreateLogger("TsavoriteKV [obj]") - }; - - logger?.LogInformation("[Object Store] Using page size of {PageSize}", PrettySize(kvSettings.PageSize)); - logger?.LogInformation("[Object Store] Each page can hold ~{PageSize} key-value pairs of objects", kvSettings.PageSize / 24); - - kvSettings.MemorySize = 1L << MemorySizeBits(ObjectStoreLogMemorySize, ObjectStorePageSize, out var objectStoreEmptyPageCount); - kvSettings.MinEmptyPageCount = objectStoreEmptyPageCount; - - long effectiveSize = kvSettings.MemorySize - objectStoreEmptyPageCount * kvSettings.PageSize; - if (objectStoreEmptyPageCount == 0) - logger?.LogInformation("[Object Store] Using log memory size of {MemorySize}", PrettySize(kvSettings.MemorySize)); - else - logger?.LogInformation("[Object Store] Using log memory size of {MemorySize}, with {objectStoreEmptyPageCount} empty pages, for effective size of {effectiveSize}", PrettySize(kvSettings.MemorySize), objectStoreEmptyPageCount, PrettySize(effectiveSize)); - - logger?.LogInformation("[Object Store] This can hold ~{PageSize} key-value pairs of objects in memory total", effectiveSize / 24); - - logger?.LogInformation("[Object Store] There are {LogPages} log pages in memory", PrettySize(kvSettings.MemorySize / kvSettings.PageSize)); - - kvSettings.SegmentSize = 1L << ObjectStoreSegmentSizeBits(); - logger?.LogInformation("[Object Store] Using disk segment size of {SegmentSize}", PrettySize(kvSettings.SegmentSize)); - - logger?.LogInformation("[Object Store] Using hash index size of {IndexSize} ({indexCacheLines} cache lines)", PrettySize(kvSettings.IndexSize), PrettySize(indexCacheLines)); - logger?.LogInformation("[Object Store] Hash index size is optimized for up to ~{distinctKeys} distinct keys", PrettySize(indexCacheLines * 4L)); - - AdjustedObjectStoreIndexMaxCacheLines = ObjectStoreIndexMaxSize == string.Empty ? 0 : IndexSizeCachelines("hash index max size", ObjectStoreIndexMaxSize); - if (AdjustedObjectStoreIndexMaxCacheLines != 0 && AdjustedObjectStoreIndexMaxCacheLines < indexCacheLines) - throw new Exception($"Index size {IndexSize} should not be less than index max size {IndexMaxSize}"); - - if (AdjustedObjectStoreIndexMaxCacheLines > 0) - { - logger?.LogInformation("[Object Store] Using hash index max size of {MaxSize}, ({CacheLines} cache lines)", PrettySize(AdjustedObjectStoreIndexMaxCacheLines * 64L), PrettySize(AdjustedObjectStoreIndexMaxCacheLines)); - logger?.LogInformation("[Object Store] Hash index max size is optimized for up to ~{distinctKeys} distinct keys", PrettySize(AdjustedObjectStoreIndexMaxCacheLines * 4L)); - } - logger?.LogInformation("[Object Store] Using log mutable percentage of {ObjectStoreMutablePercent}%", ObjectStoreMutablePercent); - - objHeapMemorySize = ParseSize(ObjectStoreHeapMemorySize, out _); - logger?.LogInformation("[Object Store] Heap memory size is {objHeapMemorySize}", objHeapMemorySize > 0 ? PrettySize(objHeapMemorySize) : "unlimited"); - - // Read cache related settings - if (EnableObjectStoreReadCache && !EnableStorageTier) - { - throw new Exception("Read cache requires storage tiering to be enabled"); - } - - if (EnableObjectStoreReadCache) - { - kvSettings.ReadCacheEnabled = true; - kvSettings.ReadCachePageSize = ParseSize(ObjectStoreReadCachePageSize, out _); - kvSettings.ReadCacheMemorySize = ParseSize(ObjectStoreReadCacheLogMemorySize, out _); - logger?.LogInformation("[Object Store] Read cache enabled with page size of {ReadCachePageSize} and memory size of {ReadCacheMemorySize}", - PrettySize(kvSettings.ReadCachePageSize), PrettySize(kvSettings.ReadCacheMemorySize)); - - objReadCacheHeapMemorySize = ParseSize(ObjectStoreReadCacheHeapMemorySize, out _); - logger?.LogInformation("[Object Store] Read cache heap memory size is {objReadCacheHeapMemorySize}", objReadCacheHeapMemorySize > 0 ? PrettySize(objReadCacheHeapMemorySize) : "unlimited"); - } - - if (EnableStorageTier) - { - if (LogDir is null or "") - LogDir = Directory.GetCurrentDirectory(); - kvSettings.LogDevice = GetInitializedDeviceFactory(LogDir).Get(new FileDescriptor("ObjectStore", "hlog")); - kvSettings.ObjectLogDevice = GetInitializedDeviceFactory(LogDir).Get(new FileDescriptor("ObjectStore", "hlog.obj")); - } - else - { - if (LogDir != null) - throw new Exception("LogDir specified without enabling tiered storage (UseStorage)"); - kvSettings.LogDevice = kvSettings.ObjectLogDevice = new NullDevice(); - } - - if (ObjectStoreCopyReadsToTail) - kvSettings.ReadCopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.MainLog); - - if (RevivInChainOnly) - { - logger?.LogInformation("[Object Store] Using Revivification in-chain only"); - kvSettings.RevivificationSettings = RevivificationSettings.InChainOnly.Clone(); - } - else if (UseRevivBinsPowerOf2 || RevivBinRecordSizes?.Length > 0) - { - logger?.LogInformation("[Store] Using Revivification with power-of-2 bins"); - kvSettings.RevivificationSettings = RevivificationSettings.PowerOf2Bins.Clone(); - kvSettings.RevivificationSettings.NumberOfBinsToSearch = RevivNumberOfBinsToSearch; - kvSettings.RevivificationSettings.RevivifiableFraction = RevivifiableFraction; - } - else - { - logger?.LogInformation("[Object Store] Not using Revivification"); - } - - return kvSettings; - } - /// /// Get AOF settings /// diff --git a/libs/server/SessionParseStateExtensions.cs b/libs/server/SessionParseStateExtensions.cs index d2d06be8a60..5fddc1ebe2c 100644 --- a/libs/server/SessionParseStateExtensions.cs +++ b/libs/server/SessionParseStateExtensions.cs @@ -38,16 +38,10 @@ public static bool TryGetInfoMetricsType(this SessionParseState parseState, int value = InfoMetricsType.STATS; else if (sbArg.EqualsUpperCaseSpanIgnoringCase("STORE"u8)) value = InfoMetricsType.STORE; - else if (sbArg.EqualsUpperCaseSpanIgnoringCase("OBJECTSTORE"u8)) - value = InfoMetricsType.OBJECTSTORE; else if (sbArg.EqualsUpperCaseSpanIgnoringCase("STOREHASHTABLE"u8)) value = InfoMetricsType.STOREHASHTABLE; - else if (sbArg.EqualsUpperCaseSpanIgnoringCase("OBJECTSTOREHASHTABLE"u8)) - value = InfoMetricsType.OBJECTSTOREHASHTABLE; else if (sbArg.EqualsUpperCaseSpanIgnoringCase("STOREREVIV"u8)) value = InfoMetricsType.STOREREVIV; - else if (sbArg.EqualsUpperCaseSpanIgnoringCase("OBJECTSTOREREVIV"u8)) - value = InfoMetricsType.OBJECTSTOREREVIV; else if (sbArg.EqualsUpperCaseSpanIgnoringCase("PERSISTENCE"u8)) value = InfoMetricsType.PERSISTENCE; else if (sbArg.EqualsUpperCaseSpanIgnoringCase("CLIENTS"u8)) diff --git a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs index 1019608a042..3ab38e722f3 100644 --- a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs +++ b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs @@ -21,8 +21,7 @@ sealed partial class StorageSession : IDisposable private ArrayKeyIterationFunctions.ObjectStoreGetDBKeys objStoreDbScanFuncs; // Iterators for expired key deletion - private ArrayKeyIterationFunctions.MainStoreExpiredKeyDeletionScan mainStoreExpiredKeyDeletionScanFuncs; - private ArrayKeyIterationFunctions.ObjectStoreExpiredKeyDeletionScan objectStoreExpiredKeyDeletionScanFuncs; + private ArrayKeyIterationFunctions.MainStoreExpiredKeyDeletionScan expiredKeyDeletionScanFuncs; // Iterators for KEYS command private ArrayKeyIterationFunctions.MainStoreGetDBKeys mainStoreDbKeysFuncs; @@ -117,25 +116,14 @@ internal unsafe bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, } /// - /// Iterates over main store memory collecting expired records. + /// Iterates over store memory collecting expired records. /// - internal (long, long) MainStoreExpiredKeyDeletionScan(long fromAddress, long untilAddress) + internal (long, long) ExpiredKeyDeletionScan(long fromAddress, long untilAddress) { - mainStoreExpiredKeyDeletionScanFuncs ??= new(); - mainStoreExpiredKeyDeletionScanFuncs.Initialize(this); - _ = basicContext.Session.ScanCursor(ref fromAddress, untilAddress, mainStoreExpiredKeyDeletionScanFuncs); - return (mainStoreExpiredKeyDeletionScanFuncs.deletedCount, mainStoreExpiredKeyDeletionScanFuncs.totalCount); - } - - /// - /// Iterates over object store memory collecting expired records. - /// - internal (long, long) ObjectStoreExpiredKeyDeletionScan(long fromAddress, long untilAddress) - { - objectStoreExpiredKeyDeletionScanFuncs ??= new(); - objectStoreExpiredKeyDeletionScanFuncs.Initialize(this); - _ = objectStoreBasicContext.Session.ScanCursor(ref fromAddress, untilAddress, objectStoreExpiredKeyDeletionScanFuncs); - return (objectStoreExpiredKeyDeletionScanFuncs.deletedCount, objectStoreExpiredKeyDeletionScanFuncs.totalCount); + expiredKeyDeletionScanFuncs ??= new(); + expiredKeyDeletionScanFuncs.Initialize(this); + _ = basicContext.Session.ScanCursor(ref fromAddress, untilAddress, expiredKeyDeletionScanFuncs); + return (expiredKeyDeletionScanFuncs.deletedCount, expiredKeyDeletionScanFuncs.totalCount); } /// diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index b6f42dba8a3..0b2f14f7515 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -10,16 +10,13 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { public GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { long ctx = default; var status = context.Read(key.ReadOnlySpan, ref input, ref output, ctx); @@ -44,7 +41,7 @@ public GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, } public unsafe GarnetStatus ReadWithUnsafeContext(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long localHeadAddress, out bool epochChanged, ref TContext context) - where TContext : ITsavoriteContext, IUnsafeContext + where TContext : ITsavoriteContext, IUnsafeContext { epochChanged = false; var status = context.Read(key.ReadOnlySpan, ref Unsafe.AsRef(in input), ref output, userContext: default); @@ -77,7 +74,7 @@ public unsafe GarnetStatus ReadWithUnsafeContext(PinnedSpanByte key, r } public unsafe GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte value, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GET); value = default; @@ -101,7 +98,7 @@ public unsafe GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte } public unsafe GarnetStatus GET(PinnedSpanByte key, out MemoryResult value, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GET); @@ -113,7 +110,7 @@ public unsafe GarnetStatus GET(PinnedSpanByte key, out MemoryResult(PinnedSpanByte key, out GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { ObjectInput input = default; output = default; @@ -139,7 +136,7 @@ public GarnetStatus GET(PinnedSpanByte key, out GarnetObjectStor } public unsafe GarnetStatus GETEX(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.RMW(key.ReadOnlySpan, ref input, ref output); @@ -170,7 +167,7 @@ public unsafe GarnetStatus GETEX(PinnedSpanByte key, ref RawStringInpu /// Basic Context of the store /// Operation status public unsafe GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GETDEL); @@ -184,7 +181,7 @@ public unsafe GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndM } public unsafe GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.Read(key.ReadOnlySpan, ref input, ref output); @@ -221,8 +218,8 @@ public unsafe GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringI /// when true the command to execute is PTTL. /// public unsafe GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output, ref TContext context, ref TObjectContext objectContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var cmd = milliseconds ? RespCommand.PTTL : RespCommand.TTL; var input = new RawStringInput(cmd); @@ -275,8 +272,8 @@ public unsafe GarnetStatus TTL(PinnedSpanByte key, Sto /// when true the command to execute is PEXPIRETIME. /// Returns the absolute Unix timestamp (since January 1, 1970) in seconds or milliseconds at which the given key will expire. public unsafe GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output, ref TContext context, ref TObjectContext objectContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (storeType == StoreType.Main || storeType == StoreType.All) { @@ -316,14 +313,14 @@ public unsafe GarnetStatus EXPIRETIME(PinnedSpanByte k } public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { context.Upsert(key.ReadOnlySpan, value.ReadOnlySpan); return GarnetStatus.OK; } public GarnetStatus SET(PinnedSpanByte key, ref RawStringInput input, PinnedSpanByte value, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var output = new SpanByteAndMemory(); context.Upsert(key.ReadOnlySpan, ref input, value.ReadOnlySpan, ref output); @@ -331,7 +328,7 @@ public GarnetStatus SET(PinnedSpanByte key, ref RawStringInput input, } public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { byte* pbOutput = stackalloc byte[8]; var o = SpanByteAndMemory.FromPinnedPointer(pbOutput, 8); @@ -359,7 +356,7 @@ public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref Raw public unsafe GarnetStatus DEL_Conditional(PinnedSpanByte key, ref RawStringInput input, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { Debug.Assert(input.header.cmd is RespCommand.DELIFGREATER or RespCommand.DELIFEXPIM); @@ -390,7 +387,7 @@ public unsafe GarnetStatus DEL_Conditional(PinnedSpanByte key, ref Raw } public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.RMW(key.ReadOnlySpan, ref input, ref output); @@ -414,7 +411,7 @@ public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref Raw } internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref TContext ctx) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var error = false; var count = input.parseState.Count; @@ -465,14 +462,14 @@ internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref T } public GarnetStatus SET(PinnedSpanByte key, IGarnetObject value, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { objectContext.Upsert(key.ReadOnlySpan, value); return GarnetStatus.OK; } public GarnetStatus SET(PinnedSpanByte key, Memory value, ref TContext context) // TODO are memory overloads needed? - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { unsafe { @@ -483,8 +480,8 @@ public GarnetStatus SET(PinnedSpanByte key, Memory value, ref TC } public GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext where TSourceLogRecord : ISourceLogRecord { if (storeType == StoreType.Main) @@ -495,11 +492,11 @@ public GarnetStatus SET(in TSourceLo } public unsafe GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, PinnedSpanByte expiryMs, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => SETEX(key, value, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ToPointer())), ref context); public GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, TimeSpan expiry, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.APPEND, ref parseState, arg1: DateTimeOffset.UtcNow.Ticks + expiry.Ticks); return SET(key, ref input, value, ref context); @@ -515,7 +512,7 @@ public GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, Ti /// Store context /// Operation status public unsafe GarnetStatus APPEND(PinnedSpanByte key, PinnedSpanByte value, ref PinnedSpanByte output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var _output = new SpanByteAndMemory(output); @@ -535,7 +532,7 @@ public unsafe GarnetStatus APPEND(PinnedSpanByte key, PinnedSpanByte v /// Store context /// Operation status public unsafe GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) @@ -551,8 +548,8 @@ public unsafe GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInp } public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var found = false; @@ -785,8 +782,8 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne /// Object context for the object store. /// public GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = GarnetStatus.NOTFOUND; RawStringInput input = default; @@ -826,8 +823,8 @@ public GarnetStatus EXISTS(PinnedSpanByte key, StoreTy /// Object context for the object store. /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => EXPIRE(key, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ToPointer())), out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext); /// @@ -843,8 +840,8 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, /// Object context for the object store /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); @@ -905,8 +902,8 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, /// When true, is treated as milliseconds else seconds /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND public unsafe GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return EXPIRE(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext, milliseconds ? RespCommand.PEXPIREAT : RespCommand.EXPIREAT); } @@ -926,8 +923,8 @@ public unsafe GarnetStatus EXPIREAT(PinnedSpanByte key /// When true the command executed is PEXPIRE, expire by default. /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return EXPIRE(key, (long)(milliseconds ? expiry.TotalMilliseconds : expiry.TotalSeconds), out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext, milliseconds ? RespCommand.PEXPIRE : RespCommand.EXPIRE); @@ -948,8 +945,8 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, /// The current RESP command /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, long expiration, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, RespCommand respCommand) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); @@ -1002,8 +999,8 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, } public unsafe GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { GarnetStatus status = GarnetStatus.NOTFOUND; @@ -1057,7 +1054,7 @@ public unsafe GarnetStatus PERSIST(PinnedSpanByte key, /// Basic context for the main store /// public unsafe GarnetStatus SETRANGE(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { SpanByteAndMemory sbmOut = new(output); @@ -1072,7 +1069,7 @@ public unsafe GarnetStatus SETRANGE(PinnedSpanByte key, ref RawStringI } public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { SpanByteAndMemory _output = new(output); @@ -1085,7 +1082,7 @@ public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput i } public unsafe GarnetStatus Increment(PinnedSpanByte key, out long output, long increment, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var cmd = RespCommand.INCRBY; if (increment < 0) @@ -1116,8 +1113,8 @@ public unsafe GarnetStatus Increment(PinnedSpanByte key, out long outp public unsafe GarnetStatus SCAN(long cursor, PinnedSpanByte match, long count, ref TContext context) => GarnetStatus.OK; public GarnetStatus GetKeyType(PinnedSpanByte key, out string keyType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { keyType = "string"; // Check if key exists in Main store @@ -1156,8 +1153,8 @@ public GarnetStatus GetKeyType(PinnedSpanByte key, out } public GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, ref TContext context, ref TObjectContext objectContext, int samples = 0) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { memoryUsage = -1; @@ -1218,7 +1215,7 @@ public unsafe GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref Spa } private unsafe GarnetStatus LCSInternal(PinnedSpanByte key1, PinnedSpanByte key2, ref SpanByteAndMemory output, ref TContext context, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { PinnedSpanByte val1, val2; var status1 = GET(key1, out val1, ref context); diff --git a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs index db6a3a83362..1504908d1bb 100644 --- a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs @@ -6,13 +6,13 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { public GarnetStatus RMW_ObjectStore(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = objectStoreContext.RMW(key, ref input, ref output); @@ -30,7 +30,7 @@ public GarnetStatus RMW_ObjectStore(ReadOnlySpan key, ref } public GarnetStatus Read_ObjectStore(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = objectStoreContext.Read(key, ref input, ref output); diff --git a/libs/server/Storage/Session/ObjectStore/Common.cs b/libs/server/Storage/Session/ObjectStore/Common.cs index 98511f116c8..9c739070741 100644 --- a/libs/server/Storage/Session/ObjectStore/Common.cs +++ b/libs/server/Storage/Session/ObjectStore/Common.cs @@ -12,15 +12,15 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { #region Common ObjectStore Methods unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -37,7 +37,7 @@ unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan k unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -58,7 +58,7 @@ unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan k /// /// GarnetStatus RMWObjectStoreOperationWithOutput(ReadOnlySpan key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -80,7 +80,7 @@ GarnetStatus RMWObjectStoreOperationWithOutput(ReadOnlySpan /// GarnetStatus ReadObjectStoreOperationWithOutput(ReadOnlySpan key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -103,7 +103,7 @@ GarnetStatus ReadObjectStoreOperationWithOutput(ReadOnlySpan unsafe GarnetStatus ReadObjectStoreOperationWithOutput(ReadOnlySpan key, PinnedSpanByte input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -124,7 +124,7 @@ unsafe GarnetStatus ReadObjectStoreOperationWithOutput(ReadOnlyS /// The list of items for the response /// public unsafe GarnetStatus ObjectScan(GarnetObjectType objectType, PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { Debug.Assert(objectType is GarnetObjectType.Hash or GarnetObjectType.Set or GarnetObjectType.SortedSet); @@ -715,7 +715,7 @@ unsafe bool TryProcessRespSimple64IntOutput(GarnetObjectStoreOutput output, out /// /// unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -751,7 +751,7 @@ unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan /// /// unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -784,7 +784,7 @@ unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan /// /// public GarnetStatus ObjectScan(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); [MethodImpl(MethodImplOptions.NoInlining)] @@ -802,7 +802,7 @@ static void ThrowObjectStoreUninitializedException() /// /// private GarnetStatus CompletePendingAndGetGarnetStatus(Status status, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref output, ref objectStoreContext); @@ -827,7 +827,7 @@ private GarnetStatus CompletePendingAndGetGarnetStatus(Status st /// The context of the object store. /// The status of the operation. private GarnetStatus ObjectCollect(PinnedSpanByte searchKey, ReadOnlySpan typeObject, SingleWriterMultiReaderLock collectLock, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (!collectLock.TryWriteLock()) return GarnetStatus.NOTFOUND; diff --git a/libs/server/Storage/Session/ObjectStore/CompletePending.cs b/libs/server/Storage/Session/ObjectStore/CompletePending.cs index 932f5166e6a..a850e3dab0b 100644 --- a/libs/server/Storage/Session/ObjectStore/CompletePending.cs +++ b/libs/server/Storage/Session/ObjectStore/CompletePending.cs @@ -6,8 +6,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession { @@ -18,7 +18,7 @@ sealed partial class StorageSession /// /// static void CompletePendingForObjectStoreSession(ref Status status, ref GarnetObjectStoreOutput output, ref TContext objectContext) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { objectContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); var more = completedOutputs.Next(); diff --git a/libs/server/Storage/Session/ObjectStore/HashOps.cs b/libs/server/Storage/Session/ObjectStore/HashOps.cs index 1d8951e5379..b9c8c08c154 100644 --- a/libs/server/Storage/Session/ObjectStore/HashOps.cs +++ b/libs/server/Storage/Session/ObjectStore/HashOps.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Server API methods - HASH @@ -33,7 +33,7 @@ sealed partial class StorageSession : IDisposable /// /// public unsafe GarnetStatus HashSet(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -65,7 +65,7 @@ public unsafe GarnetStatus HashSet(PinnedSpanByte key, PinnedSpa /// /// public unsafe GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpanByte value)[] elements, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -101,7 +101,7 @@ public unsafe GarnetStatus HashSet(PinnedSpanByte key, (PinnedSp /// /// public GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte field, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => HashDelete(key, [field], out itemsDoneCount, ref objectStoreContext); /// @@ -114,7 +114,7 @@ public GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByt /// /// public unsafe GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte[] fields, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -144,7 +144,7 @@ public unsafe GarnetStatus HashDelete(PinnedSpanByte key, Pinned /// /// public unsafe GarnetStatus HashGet(PinnedSpanByte key, PinnedSpanByte field, out PinnedSpanByte value, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { value = default; @@ -179,7 +179,7 @@ public unsafe GarnetStatus HashGet(PinnedSpanByte key, PinnedSpa /// /// public unsafe GarnetStatus HashGetMultiple(PinnedSpanByte key, PinnedSpanByte[] fields, out PinnedSpanByte[] values, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { values = default; @@ -213,7 +213,7 @@ public unsafe GarnetStatus HashGetMultiple(PinnedSpanByte key, P /// /// public unsafe GarnetStatus HashGetAll(PinnedSpanByte key, out PinnedSpanByte[] values, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { values = default; @@ -245,7 +245,7 @@ public unsafe GarnetStatus HashGetAll(PinnedSpanByte key, out Pi /// /// public unsafe GarnetStatus HashLength(PinnedSpanByte key, out int items, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { items = 0; @@ -273,7 +273,7 @@ public unsafe GarnetStatus HashLength(PinnedSpanByte key, out in /// /// public unsafe GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out bool exists, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { exists = false; if (key.Length == 0) @@ -302,7 +302,7 @@ public unsafe GarnetStatus HashExists(PinnedSpanByte key, Pinned /// /// public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, out PinnedSpanByte field, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { field = default; @@ -340,7 +340,7 @@ public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, o /// /// public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, int count, bool withValues, out PinnedSpanByte[] fields, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { fields = default; @@ -377,7 +377,7 @@ public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, i /// /// public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -393,7 +393,7 @@ public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -406,7 +406,7 @@ public GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -419,7 +419,7 @@ public GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -432,7 +432,7 @@ public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref Obje /// /// public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -445,7 +445,7 @@ public GarnetStatus HashRandomField(PinnedSpanByte key, ref Obje /// /// public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -458,7 +458,7 @@ public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -471,7 +471,7 @@ public GarnetStatus HashStrLength(PinnedSpanByte key, ref Object /// /// public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -484,7 +484,7 @@ public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -497,7 +497,7 @@ public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -510,7 +510,7 @@ public GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -523,7 +523,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, input, out output, ref objectContext); /// @@ -537,7 +537,7 @@ public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpan /// /// public GarnetStatus HashIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -550,7 +550,7 @@ public GarnetStatus HashIncrement(PinnedSpanByte key, ref Object /// The object context for the operation. /// The status of the operation. public GarnetStatus HashExpire(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -565,7 +565,7 @@ public GarnetStatus HashExpire(PinnedSpanByte key, ref ObjectInp /// The object context for the operation. /// The status of the operation. public GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var innerInput = new ObjectInput(input.header, ref input.parseState, arg1: isMilliseconds ? 1 : 0, arg2: isTimestamp ? 1 : 0); @@ -582,7 +582,7 @@ public GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMi /// The object context for the operation. /// The status of the operation. public GarnetStatus HashPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -598,7 +598,7 @@ public GarnetStatus HashPersist(PinnedSpanByte key, ref ObjectIn /// Otherwise, the operation is performed on the specified keys. /// public unsafe GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (keys[0].ReadOnlySpan.SequenceEqual("*"u8)) return ObjectCollect(keys[0], CmdStrings.HASH, _hcollectTaskLock, ref input, ref objectContext); diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index c07d6a7651c..18281d09ee3 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -26,7 +26,7 @@ sealed partial class StorageSession : IDisposable /// /// public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSpanByte[] elements, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -61,7 +61,7 @@ public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSp /// /// public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSpanByte element, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -90,7 +90,7 @@ public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSp /// /// The popped element public GarnetStatus ListPop(PinnedSpanByte key, ListOperation lop, ref TObjectContext objectStoreContext, out PinnedSpanByte element) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = ListPop(key, 1, lop, ref objectStoreContext, out var elements); element = status == GarnetStatus.OK ? elements.FirstOrDefault() : default; @@ -109,7 +109,7 @@ public GarnetStatus ListPop(PinnedSpanByte key, ListOperation lo /// /// The count elements popped from the list public unsafe GarnetStatus ListPop(PinnedSpanByte key, int count, ListOperation lop, ref TObjectContext objectStoreContext, out PinnedSpanByte[] elements) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { // Prepare the input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = lop }; @@ -140,7 +140,7 @@ public unsafe GarnetStatus ListPop(PinnedSpanByte key, int count /// /// The count elements popped from the list public unsafe GarnetStatus ListPopMultiple(PinnedSpanByte[] keys, OperationDirection direction, int count, ref TObjectContext objectContext, out PinnedSpanByte key, out PinnedSpanByte[] elements) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { foreach (var k in keys) { @@ -175,7 +175,7 @@ public unsafe GarnetStatus ListPopMultiple(PinnedSpanByte[] keys /// /// public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref TObjectContext objectStoreContext, out int count) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { count = 0; @@ -335,7 +335,7 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio /// /// true when successful public unsafe bool ListTrim(PinnedSpanByte key, int start, int stop, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { // Prepare the input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LTRIM }; @@ -356,7 +356,7 @@ public unsafe bool ListTrim(PinnedSpanByte key, int start, int s /// /// public GarnetStatus ListPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); itemBroker.HandleCollectionUpdate(key.ToArray()); @@ -374,7 +374,7 @@ public GarnetStatus ListPush(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); } @@ -388,7 +388,7 @@ public GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectI /// /// public GarnetStatus ListTrim(PinnedSpanByte key, ref ObjectInput input, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out _, ref objectStoreContext); /// @@ -401,7 +401,7 @@ public GarnetStatus ListTrim(PinnedSpanByte key, ref ObjectInput /// /// public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -414,7 +414,7 @@ public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInpu /// /// public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); itemBroker.HandleCollectionUpdate(key.ToArray()); @@ -431,7 +431,7 @@ public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -445,7 +445,7 @@ public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInpu /// /// public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -459,7 +459,7 @@ public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInp /// /// public unsafe GarnetStatus ListPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -473,7 +473,7 @@ public unsafe GarnetStatus ListPop(PinnedSpanByte key, ref Objec /// /// public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -486,7 +486,7 @@ public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref Ob /// /// public unsafe GarnetStatus ListSet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); } } \ No newline at end of file diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index 35dc29b03f8..72a2f934a7d 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Server session for RESP protocol - SET @@ -28,7 +28,7 @@ sealed partial class StorageSession : IDisposable /// /// internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte member, out int saddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { saddCount = 0; @@ -57,7 +57,7 @@ internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSp /// /// internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int saddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { saddCount = 0; @@ -91,7 +91,7 @@ internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSp /// /// internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte member, out int sremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { sremCount = 0; @@ -121,7 +121,7 @@ internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, Pinne /// /// internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int sremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { sremCount = 0; @@ -150,7 +150,7 @@ internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, Pinne /// /// internal unsafe GarnetStatus SetLength(PinnedSpanByte key, out int count, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { count = 0; @@ -176,7 +176,7 @@ internal unsafe GarnetStatus SetLength(PinnedSpanByte key, out i /// /// internal unsafe GarnetStatus SetMembers(PinnedSpanByte key, out PinnedSpanByte[] members, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { members = default; @@ -206,7 +206,7 @@ internal unsafe GarnetStatus SetMembers(PinnedSpanByte key, out /// /// internal GarnetStatus SetPop(PinnedSpanByte key, out PinnedSpanByte element, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = SetPop(key, int.MinValue, out var elements, ref objectStoreContext); element = default; @@ -226,7 +226,7 @@ internal GarnetStatus SetPop(PinnedSpanByte key, out PinnedSpanB /// /// internal unsafe GarnetStatus SetPop(PinnedSpanByte key, int count, out PinnedSpanByte[] elements, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { elements = default; @@ -449,7 +449,7 @@ public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, private GarnetStatus SetIntersect(ReadOnlySpan keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { output = new HashSet(ByteArrayComparer.Instance); @@ -617,7 +617,7 @@ public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out } private GarnetStatus SetUnion(PinnedSpanByte[] keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { output = new HashSet(ByteArrayComparer.Instance); if (keys.Length == 0) @@ -652,7 +652,7 @@ private GarnetStatus SetUnion(PinnedSpanByte[] keys, ref TObject /// /// public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// @@ -667,7 +667,7 @@ public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput i /// /// public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// @@ -680,7 +680,7 @@ public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInpu /// /// public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// @@ -693,7 +693,7 @@ public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInpu /// /// public GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -706,7 +706,7 @@ public GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInp /// /// public GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -718,7 +718,7 @@ public GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectIn /// /// public unsafe GarnetStatus SetIsMember(PinnedSpanByte key, PinnedSpanByte[] members, out int[] result, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { result = default; @@ -753,7 +753,7 @@ public unsafe GarnetStatus SetIsMember(PinnedSpanByte key, Pinne /// /// public GarnetStatus SetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -770,7 +770,7 @@ public GarnetStatus SetPop(PinnedSpanByte key, ref ObjectInput i /// /// public GarnetStatus SetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -876,7 +876,7 @@ public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out } private GarnetStatus SetDiff(PinnedSpanByte[] keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { output = new HashSet(); if (keys.Length == 0) diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index c7241c3ca38..8860bd84b6a 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -24,7 +24,7 @@ sealed partial class StorageSession : IDisposable /// /// public GarnetStatus GeoAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -39,7 +39,7 @@ public GarnetStatus GeoAdd(PinnedSpanByte key, ref ObjectInput i /// /// public GarnetStatus GeoCommands(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -61,7 +61,7 @@ public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref Ge ref ObjectInput input, ref SpanByteAndMemory output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var createTransaction = false; @@ -118,7 +118,7 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi ref ObjectInput input, ref SpanByteAndMemory output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var createTransaction = false; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index c301eab0928..035e287edc5 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -12,8 +12,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -30,7 +30,7 @@ sealed partial class StorageSession : IDisposable /// /// public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, PinnedSpanByte score, PinnedSpanByte member, out int zaddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zaddCount = 0; if (key.Length == 0) @@ -65,7 +65,7 @@ public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, Pinn /// /// public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, (PinnedSpanByte score, PinnedSpanByte member)[] inputs, out int zaddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zaddCount = 0; @@ -108,7 +108,7 @@ public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, (Pin /// public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte member, out int zremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zremCount = 0; @@ -139,7 +139,7 @@ public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, P /// /// public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int zremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zremCount = 0; @@ -170,7 +170,7 @@ public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, P /// public unsafe GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, string min, string max, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -217,7 +217,7 @@ public unsafe GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanB /// public unsafe GarnetStatus SortedSetRemoveRangeByScore(PinnedSpanByte key, string min, string max, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -273,7 +273,7 @@ public unsafe GarnetStatus SortedSetRemoveRangeByScore(PinnedSpa /// public unsafe GarnetStatus SortedSetRemoveRangeByRank(PinnedSpanByte key, int start, int stop, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -330,7 +330,7 @@ public unsafe GarnetStatus SortedSetRemoveRangeByRank(PinnedSpan /// /// public unsafe GarnetStatus SortedSetPop(PinnedSpanByte key, int count, bool lowScoresFirst, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { pairs = default; if (key.Length == 0) @@ -366,7 +366,7 @@ public unsafe GarnetStatus SortedSetPop(PinnedSpanByte key, int /// public unsafe GarnetStatus SortedSetIncrement(PinnedSpanByte key, double increment, PinnedSpanByte member, out double newScore, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { newScore = 0; @@ -412,7 +412,7 @@ public unsafe GarnetStatus SortedSetIncrement(PinnedSpanByte key /// /// public unsafe GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zcardCount = 0; @@ -447,7 +447,7 @@ public unsafe GarnetStatus SortedSetLength(PinnedSpanByte key, o /// /// public unsafe GarnetStatus SortedSetRange(PinnedSpanByte key, PinnedSpanByte min, PinnedSpanByte max, SortedSetOrderOperation sortedSetOrderOperation, ref TObjectContext objectContext, out PinnedSpanByte[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { elements = default; error = default; @@ -631,7 +631,7 @@ public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, Read /// /// public unsafe GarnetStatus SortedSetRank(PinnedSpanByte key, PinnedSpanByte member, bool reverse, out long? rank, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { rank = null; if (key.Length == 0) @@ -678,7 +678,7 @@ public unsafe GarnetStatus SortedSetRank(PinnedSpanByte key, Pin /// /// public GarnetStatus SortedSetAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); itemBroker.HandleCollectionUpdate(key.ToArray()); @@ -696,7 +696,7 @@ public GarnetStatus SortedSetAdd(PinnedSpanByte key, ref ObjectI /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte dstKey, PinnedSpanByte srcKey, ref ObjectInput input, out int result, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (txnManager.ObjectStoreTransactionalContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -800,7 +800,7 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds /// /// public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -813,7 +813,7 @@ public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref Obje /// /// public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -828,7 +828,7 @@ public GarnetStatus SortedSetLength(PinnedSpanByte key, ref Obje /// /// public GarnetStatus SortedSetRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -842,7 +842,7 @@ public GarnetStatus SortedSetRange(PinnedSpanByte key, ref Objec /// /// public GarnetStatus SortedSetScore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -856,7 +856,7 @@ public GarnetStatus SortedSetScore(PinnedSpanByte key, ref Objec /// /// public GarnetStatus SortedSetScores(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -870,7 +870,7 @@ public GarnetStatus SortedSetScores(PinnedSpanByte key, ref Obje /// /// public GarnetStatus SortedSetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -884,7 +884,7 @@ public GarnetStatus SortedSetPop(PinnedSpanByte key, ref ObjectI /// /// public unsafe GarnetStatus SortedSetCount(PinnedSpanByte key, PinnedSpanByte minScore, PinnedSpanByte maxScore, out int numElements, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { numElements = 0; if (key.Length == 0) @@ -923,7 +923,7 @@ public unsafe GarnetStatus SortedSetCount(PinnedSpanByte key, Pi /// /// public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -937,7 +937,7 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref Objec /// /// public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// @@ -952,7 +952,7 @@ public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key /// /// public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// @@ -966,7 +966,7 @@ public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, r /// /// public GarnetStatus SortedSetIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// @@ -980,7 +980,7 @@ public GarnetStatus SortedSetIncrement(PinnedSpanByte key, ref O /// /// public GarnetStatus SortedSetRemoveRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -993,7 +993,7 @@ public GarnetStatus SortedSetRemoveRange(PinnedSpanByte key, ref /// /// public GarnetStatus SortedSetRank(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -1006,7 +1006,7 @@ public GarnetStatus SortedSetRank(PinnedSpanByte key, ref Object /// /// public GarnetStatus SortedSetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// @@ -1020,7 +1020,7 @@ public GarnetStatus SortedSetRandomMember(PinnedSpanByte key, re /// /// public GarnetStatus SortedSetScan(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out SortedSet<(double, byte[])> pairs) @@ -1125,7 +1125,7 @@ public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlyS private GarnetStatus SortedSetUnion(ReadOnlySpan keys, ref TObjectContext objectContext, out Dictionary pairs, double[] weights = null, SortedSetAggregateType aggregateType = SortedSetAggregateType.Sum) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { pairs = default; @@ -1204,7 +1204,7 @@ private GarnetStatus SortedSetUnion(ReadOnlySpan } private GarnetStatus SortedSetDifference(ReadOnlySpan keys, ref TObjectContext objectContext, out Dictionary pairs) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { pairs = default; @@ -1437,7 +1437,7 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double /// The resulting dictionary of intersected elements and their scores. /// private GarnetStatus SortedSetIntersection(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, ref TObjectContext objectContext, out Dictionary pairs) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { pairs = default; @@ -1535,7 +1535,7 @@ private GarnetStatus SortedSetIntersection(ReadOnlySpanThe object context for the operation. /// The status of the operation. public GarnetStatus SortedSetExpire(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); } @@ -1552,7 +1552,7 @@ public GarnetStatus SortedSetExpire(PinnedSpanByte key, ref Obje /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. public GarnetStatus SortedSetExpire(PinnedSpanByte key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { results = default; var expirationTimeInTicks = expireAt.UtcTicks; @@ -1585,7 +1585,7 @@ public GarnetStatus SortedSetExpire(PinnedSpanByte key, ReadOnly /// The object context for the operation. /// The status of the operation. public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); /// @@ -1598,7 +1598,7 @@ public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ReadOnlySpan members, out TimeSpan[] expireIn, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { expireIn = default; var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZTTL }; @@ -1626,7 +1626,7 @@ public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, Read /// The object context for the operation. /// The status of the operation. public GarnetStatus SortedSetPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); /// @@ -1639,7 +1639,7 @@ public GarnetStatus SortedSetPersist(PinnedSpanByte key, ref Obj /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. public GarnetStatus SortedSetPersist(PinnedSpanByte key, ReadOnlySpan members, out int[] results, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { results = default; var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZPERSIST }; @@ -1671,7 +1671,7 @@ public GarnetStatus SortedSetPersist(PinnedSpanByte key, ReadOnl /// Otherwise, the operation is performed on the specified keys. /// public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { if (keys[0].ReadOnlySpan.SequenceEqual("*"u8)) { @@ -1697,7 +1697,7 @@ public GarnetStatus SortedSetCollect(ReadOnlySpan public GarnetStatus SortedSetCollect(ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return SortedSetCollect([], ref objectContext); } @@ -1714,7 +1714,7 @@ public GarnetStatus SortedSetCollect(ref TObjectContext objectCo /// Otherwise, the operation is performed on the specified keys. /// public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZCOLLECT }; var innerInput = new ObjectInput(header); diff --git a/libs/server/Storage/Session/StorageSession.cs b/libs/server/Storage/Session/StorageSession.cs index 73da80a116f..e3df4bfc904 100644 --- a/libs/server/Storage/Session/StorageSession.cs +++ b/libs/server/Storage/Session/StorageSession.cs @@ -8,11 +8,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Storage Session - the internal layer that Garnet uses to perform storage operations @@ -26,8 +23,8 @@ sealed partial class StorageSession : IDisposable /// /// Session Contexts for main store /// - public BasicContext basicContext; - public TransactionalContext transactionalContext; + public BasicContext basicContext; + public TransactionalContext transactionalContext; SectorAlignedMemory sectorAlignedMemoryHll1; SectorAlignedMemory sectorAlignedMemoryHll2; @@ -39,8 +36,8 @@ sealed partial class StorageSession : IDisposable /// /// Session Contexts for object store /// - public BasicContext objectStoreBasicContext; - public TransactionalContext objectStoreTransactionalContext; + public BasicContext objectStoreBasicContext; + public TransactionalContext objectStoreTransactionalContext; public readonly ScratchBufferBuilder scratchBufferBuilder; public readonly FunctionsState functionsState; @@ -78,20 +75,17 @@ public StorageSession(StoreWrapper storeWrapper, Debug.Assert(dbFound); this.stateMachineDriver = db.StateMachineDriver; - var session = db.MainStore.NewSession(functions); + var session = db.Store.NewSession(functions); var objectStoreFunctions = new ObjectSessionFunctions(functionsState); - var objectStoreSession = db.ObjectStore?.NewSession(objectStoreFunctions); + var objectStoreSession = db.Store.NewSession(objectStoreFunctions); basicContext = session.BasicContext; transactionalContext = session.TransactionalContext; - if (objectStoreSession != null) - { - objectStoreBasicContext = objectStoreSession.BasicContext; - objectStoreTransactionalContext = objectStoreSession.TransactionalContext; - } + objectStoreBasicContext = objectStoreSession.BasicContext; + objectStoreTransactionalContext = objectStoreSession.TransactionalContext; - HeadAddress = db.MainStore.Log.HeadAddress; + HeadAddress = db.Store.Log.HeadAddress; ObjectScanCountLimit = storeWrapper.serverOptions.ObjectScanCountLimit; } diff --git a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs index d37abb9a399..a7afa5cb2d3 100644 --- a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs +++ b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Tracks the size of the main log and read cache. @@ -20,8 +20,8 @@ namespace Garnet.server /// public class CacheSizeTracker { - internal readonly LogSizeTracker mainLogTracker; - internal readonly LogSizeTracker readCacheTracker; + internal readonly LogSizeTracker mainLogTracker; + internal readonly LogSizeTracker readCacheTracker; private long targetSize; public long ReadCacheTargetSize; @@ -72,7 +72,7 @@ public readonly long CalculateRecordSize(in TSourceLogRecord l /// Total memory size target /// Target memory size for read cache /// - public CacheSizeTracker(TsavoriteKV store, KVSettings logSettings, + public CacheSizeTracker(TsavoriteKV store, KVSettings logSettings, long targetSize, long readCacheTargetSize, ILoggerFactory loggerFactory = null) { Debug.Assert(store != null); @@ -85,19 +85,19 @@ public CacheSizeTracker(TsavoriteKV if (targetSize > 0) { - this.mainLogTracker = new LogSizeTracker(store.Log, logSizeCalculator, + this.mainLogTracker = new LogSizeTracker(store.Log, logSizeCalculator, targetSize, targetSize / deltaFraction, loggerFactory?.CreateLogger("ObjSizeTracker")); store.Log.SubscribeEvictions(mainLogTracker); - store.Log.SubscribeDeserializations(new LogOperationObserver(mainLogTracker, LogOperationType.Deserialize)); + store.Log.SubscribeDeserializations(new LogOperationObserver(mainLogTracker, LogOperationType.Deserialize)); store.Log.IsSizeBeyondLimit = () => mainLogTracker.IsSizeBeyondLimit; } if (store.ReadCache != null && readCacheTargetSize > 0) { - this.readCacheTracker = new LogSizeTracker(store.ReadCache, logSizeCalculator, + this.readCacheTracker = new LogSizeTracker(store.ReadCache, logSizeCalculator, readCacheTargetSize, readCacheTargetSize / deltaFraction, loggerFactory?.CreateLogger("ObjReadCacheSizeTracker")); store.ReadCache.SubscribeEvictions(readCacheTracker); - store.ReadCache.SubscribeDeserializations(new LogOperationObserver(readCacheTracker, LogOperationType.Deserialize)); + store.ReadCache.SubscribeDeserializations(new LogOperationObserver(readCacheTracker, LogOperationType.Deserialize)); store.ReadCache.IsSizeBeyondLimit = () => readCacheTracker.IsSizeBeyondLimit; } } diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 2fd8d890715..5f60d8b0f42 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -19,11 +19,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information @@ -43,12 +40,7 @@ public sealed class StoreWrapper /// /// Store (of DB 0) /// - public TsavoriteKV store => databaseManager.MainStore; - - /// - /// Object store (of DB 0) - /// - public TsavoriteKV objectStore => databaseManager.ObjectStore; + public TsavoriteKV store => databaseManager.Store; /// /// AOF (of DB 0) @@ -63,7 +55,7 @@ public sealed class StoreWrapper /// /// Object store size tracker (of DB 0) /// - public CacheSizeTracker objectStoreSizeTracker => databaseManager.ObjectStoreSizeTracker; + public CacheSizeTracker objectStoreSizeTracker => databaseManager.SizeTracker; public IStoreFunctions mainStoreFunctions => store.StoreFunctions; public IStoreFunctions objectStoreFunctions => objectStore?.StoreFunctions; @@ -158,15 +150,10 @@ public sealed class StoreWrapper bool disposed; /// - /// Garnet checkpoint manager for main store + /// Garnet checkpoint manager /// public GarnetCheckpointManager StoreCheckpointManager => (GarnetCheckpointManager)store?.CheckpointManager; - /// - /// Garnet checkpoint manager for object store - /// - public GarnetCheckpointManager ObjectStoreCheckpointManager => (GarnetCheckpointManager)objectStore?.CheckpointManager; - /// /// Constructor /// @@ -268,11 +255,6 @@ public StoreWrapper( { StoreCheckpointManager.CurrentHistoryId = runId; } - - if (!serverOptions.DisableObjects && ObjectStoreCheckpointManager != null) - { - ObjectStoreCheckpointManager.CurrentHistoryId = runId; - } } } @@ -833,7 +815,7 @@ internal void Start() Task.Run(() => IndexAutoGrowTask(ctsCommit.Token)); } - databaseManager.StartObjectSizeTrackers(ctsCommit.Token); + databaseManager.StartSizeTrackers(ctsCommit.Token); } public bool HasKeysInSlots(List slots) @@ -855,8 +837,8 @@ public bool HasKeysInSlots(List slots) if (!hasKeyInSlots && !serverOptions.DisableObjects) { var functionsState = databaseManager.CreateFunctionsState(); - var objstorefunctions = new ObjectSessionFunctions(functionsState); - var objectStoreSession = objectStore?.NewSession(objstorefunctions); + var objStorefunctions = new ObjectSessionFunctions(functionsState); + var objectStoreSession = store?.NewSession(objStorefunctions); var iter = objectStoreSession.Iterate(); while (!hasKeyInSlots && iter.GetNext()) { diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 5f982565b8d..dec6afd16f2 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -14,18 +14,16 @@ namespace Garnet.server /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; using TransactionalGarnetApi = GarnetApi, SpanByteAllocator>>, TransactionalContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Transaction manager @@ -35,22 +33,22 @@ public sealed unsafe partial class TransactionManager /// /// Basic context for main store /// - readonly BasicContext basicContext; + readonly BasicContext basicContext; /// /// Transactional context for main store /// - readonly TransactionalContext transactionalContext; + readonly TransactionalContext transactionalContext; /// /// Basic context for object store /// - readonly BasicContext objectStoreBasicContext; + readonly BasicContext objectStoreBasicContext; /// /// Transactional context for object store /// - readonly TransactionalContext objectStoreTransactionalContext; + readonly TransactionalContext objectStoreTransactionalContext; // Not readonly to avoid defensive copy GarnetWatchApi garnetTxPrepareApi; @@ -80,11 +78,11 @@ public sealed unsafe partial class TransactionManager readonly ILogger logger; long txnVersion; - internal TransactionalContext TransactionalContext + internal TransactionalContext TransactionalContext => transactionalContext; - internal TransactionalUnsafeContext TransactionalUnsafeContext + internal TransactionalUnsafeContext TransactionalUnsafeContext => basicContext.Session.TransactionalUnsafeContext; - internal TransactionalContext ObjectStoreTransactionalContext + internal TransactionalContext ObjectStoreTransactionalContext => objectStoreTransactionalContext; /// diff --git a/libs/server/Transaction/TxnKeyEntry.cs b/libs/server/Transaction/TxnKeyEntry.cs index 5829deebd98..3b5f0d80cdd 100644 --- a/libs/server/Transaction/TxnKeyEntry.cs +++ b/libs/server/Transaction/TxnKeyEntry.cs @@ -8,11 +8,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Entry for a key to lock and unlock in transactions @@ -61,8 +58,8 @@ internal sealed class TxnKeyEntries public int phase; - internal TxnKeyEntries(int initialCount, TransactionalContext transactionalContext, - TransactionalContext objectStoreTransactionalContext) + internal TxnKeyEntries(int initialCount, TransactionalContext transactionalContext, + TransactionalContext objectStoreTransactionalContext) { keys = GC.AllocateArray(initialCount, pinned: true); // We sort a single array for speed, and the sessions use the same sorting logic, diff --git a/libs/server/Transaction/TxnKeyEntryComparison.cs b/libs/server/Transaction/TxnKeyEntryComparison.cs index bc2df1ffd29..6e87e610460 100644 --- a/libs/server/Transaction/TxnKeyEntryComparison.cs +++ b/libs/server/Transaction/TxnKeyEntryComparison.cs @@ -6,21 +6,18 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; internal sealed class TxnKeyComparison { - public TransactionalContext transactionalContext; - public TransactionalContext objectStoreTransactionalContext; + public TransactionalContext transactionalContext; + public TransactionalContext objectStoreTransactionalContext; public readonly Comparison comparisonDelegate; - internal TxnKeyComparison(TransactionalContext transactionalContext, - TransactionalContext objectStoreTransactionalContext) + internal TxnKeyComparison(TransactionalContext transactionalContext, + TransactionalContext objectStoreTransactionalContext) { this.transactionalContext = transactionalContext; this.objectStoreTransactionalContext = objectStoreTransactionalContext; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs index f9a85e79908..f32202e7b74 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs @@ -103,9 +103,9 @@ public static StoreFunctions Create /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. /// - public static StoreFunctions Create(TKeyComparer keyComparer, Func> valueSerializerCreator) + public static StoreFunctions Create(TKeyComparer keyComparer, Func> valueSerializerCreator) where TKeyComparer : IKeyComparer - => new(keyComparer, valueSerializerCreator, new DefaultRecordDisposer()); + => new(keyComparer, valueSerializerCreator, new SpanByteRecordDisposer()); /// /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. @@ -118,9 +118,9 @@ public static StoreFunctions Create /// Store functions that take only the /// - public static StoreFunctions Create(TKeyComparer keyComparer) + public static StoreFunctions Create(TKeyComparer keyComparer) where TKeyComparer : IKeyComparer - => new(keyComparer, valueSerializerCreator: null, DefaultRecordDisposer.Instance); + => new(keyComparer, valueSerializerCreator: null, SpanByteRecordDisposer.Instance); /// /// Store functions for Key and Value diff --git a/test/Garnet.test/CacheSizeTrackerTests.cs b/test/Garnet.test/CacheSizeTrackerTests.cs index 556b48e8cfd..df75236c0b2 100644 --- a/test/Garnet.test/CacheSizeTrackerTests.cs +++ b/test/Garnet.test/CacheSizeTrackerTests.cs @@ -11,14 +11,14 @@ namespace Garnet.test { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; [TestFixture] public class CacheSizeTrackerTests { GarnetServer server; - TsavoriteKV objStore; + TsavoriteKV store; CacheSizeTracker cacheSizeTracker; [SetUp] @@ -27,7 +27,7 @@ public void Setup() TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "2k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreHeapMemorySize: "3k"); server.Start(); - objStore = server.Provider.StoreWrapper.objectStore; + store = server.Provider.StoreWrapper.store; cacheSizeTracker = server.Provider.StoreWrapper.objectStoreSizeTracker; } @@ -84,7 +84,7 @@ public void IncreaseEmptyPageCountTest() ClassicAssert.AreEqual(NumRecords * MemorySizePerEntry, cacheSizeTracker.mainLogTracker.LogHeapSizeBytes); // Wait for up to 3x resize task delay for the resizing to happen - if (!epcEvent.Wait(TimeSpan.FromSeconds(3 * LogSizeTracker.ResizeTaskDelaySeconds))) + if (!epcEvent.Wait(TimeSpan.FromSeconds(3 * LogSizeTracker.ResizeTaskDelaySeconds))) Assert.Fail("Timeout occurred. Resizing did not happen within the specified time."); } @@ -94,7 +94,7 @@ public void ReadCacheIncreaseEmptyPageCountTest() server?.Dispose(); server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "1k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreReadCacheHeapMemorySize: "1k", enableObjectStoreReadCache: true); server.Start(); - objStore = server.Provider.StoreWrapper.objectStore; + store = server.Provider.StoreWrapper.store; cacheSizeTracker = server.Provider.StoreWrapper.objectStoreSizeTracker; var readCacheEmptyPageCountIncrements = 0; @@ -129,7 +129,7 @@ public void ReadCacheIncreaseEmptyPageCountTest() var info = TestUtils.GetStoreAddressInfo(redis.GetServer(TestUtils.EndPoint), includeReadCache: true, isObjectStore: true); ClassicAssert.AreEqual(64 + 32 * NumRecords, info.ReadCacheTailAddress); - if (!readCacheEpcEvent.Wait(TimeSpan.FromSeconds(3 * 3 * LogSizeTracker.ResizeTaskDelaySeconds))) + if (!readCacheEpcEvent.Wait(TimeSpan.FromSeconds(3 * 3 * LogSizeTracker.ResizeTaskDelaySeconds))) ClassicAssert.Fail("Timeout occurred. Resizing did not happen within the specified time."); ClassicAssert.AreEqual(1, readCacheEmptyPageCountIncrements); diff --git a/test/Garnet.test/GarnetObjectTests.cs b/test/Garnet.test/GarnetObjectTests.cs index be43ff83bb7..6254c8ebfcd 100644 --- a/test/Garnet.test/GarnetObjectTests.cs +++ b/test/Garnet.test/GarnetObjectTests.cs @@ -10,13 +10,13 @@ namespace Garnet.test { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; [TestFixture] public class GarnetObjectTests { - TsavoriteKV store; + TsavoriteKV store; IDevice logDevice, objectLogDevice; [SetUp] @@ -173,7 +173,7 @@ private void CreateStore() }; store = new(kvSettings - , StoreFunctions.Create(new SpanByteComparer(), () => new MyGarnetObjectSerializer()) + , Tsavorite.core.StoreFunctions.Create(new SpanByteComparer(), () => new MyGarnetObjectSerializer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); } } diff --git a/test/Garnet.test/RespConfigTests.cs b/test/Garnet.test/RespConfigTests.cs index 0b590cb2723..34547530700 100644 --- a/test/Garnet.test/RespConfigTests.cs +++ b/test/Garnet.test/RespConfigTests.cs @@ -13,8 +13,8 @@ namespace Garnet.test { - using ObjectStoreAllocator = ObjectAllocator>; - using ObjectStoreFunctions = StoreFunctions; + using StoreAllocator = SpanByteAllocator>; + using StoreFunctions = StoreFunctions; /// /// Test dynamically changing server configuration using CONFIG SET command. @@ -758,7 +758,7 @@ public void ConfigSetHeapSizeUtilizationTest(string largerSize) } // Wait for log size tracker - var sizeTrackerDelay = TimeSpan.FromSeconds(LogSizeTracker.ResizeTaskDelaySeconds + 2); + var sizeTrackerDelay = TimeSpan.FromSeconds(LogSizeTracker.ResizeTaskDelaySeconds + 2); Thread.Sleep(sizeTrackerDelay); // Verify that empty page count has increased From 4f1f4b5a24a23533e96fa4243a933360676d62b5 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Tue, 16 Sep 2025 12:44:15 -0700 Subject: [PATCH 02/28] wip --- ...arnetClientSessionReplicationExtensions.cs | 21 ++--- .../cluster/Server/ClusterManagerSlotState.cs | 4 +- libs/cluster/Server/ClusterProvider.cs | 22 +----- .../Server/Migration/MigrateOperation.cs | 34 ++------- .../Server/Migration/MigrateSessionSlots.cs | 32 +++----- .../Server/Replication/CheckpointEntry.cs | 35 +-------- .../Server/Replication/CheckpointFileType.cs | 47 ------------ .../Server/Replication/CheckpointStore.cs | 56 +++----------- .../GarnetClusterCheckpointManager.cs | 4 - .../DisklessReplication/ReplicaSyncSession.cs | 13 +--- .../ReplicationSnapshotIterator.cs | 49 +++--------- .../ReplicationSyncManager.cs | 13 +--- .../PrimaryOps/ReplicaSyncSession.cs | 76 ++----------------- .../ReplicaOps/ReplicaDisklessSync.cs | 5 +- .../ReplicaOps/ReplicaReceiveCheckpoint.cs | 42 ++-------- .../ReplicationCheckpointManagement.cs | 29 ------- .../Server/Replication/ReplicationManager.cs | 28 ++----- .../Server/Replication/SyncMetadata.cs | 10 --- libs/cluster/Session/ClusterSession.cs | 4 +- .../Session/RespClusterMigrateCommands.cs | 4 +- libs/host/GarnetServer.cs | 11 +-- libs/server/AOF/AofProcessor.cs | 39 +--------- libs/server/Cluster/CheckpointMetadata.cs | 19 +---- libs/server/Cluster/IClusterProvider.cs | 4 +- libs/server/Resp/GarnetDatabaseSession.cs | 8 +- libs/server/Resp/LocalServerSession.cs | 4 +- libs/server/Resp/Objects/HashCommands.cs | 9 --- libs/server/Resp/Objects/ListCommands.cs | 9 --- libs/server/Resp/Objects/SortedSetCommands.cs | 15 ---- libs/server/Resp/RespServerSession.cs | 8 +- libs/server/ServerConfig.cs | 74 ++++++------------ .../Storage/Functions/FunctionsState.cs | 4 +- .../Functions/ObjectStore/DeleteMethods.cs | 2 +- .../Functions/ObjectStore/RMWMethods.cs | 4 +- libs/server/StoreWrapper.cs | 7 +- test/Garnet.test/CacheSizeTrackerTests.cs | 4 +- test/Garnet.test/ExpiredKeyDeletionTests.cs | 2 - test/Garnet.test/IndexGrowthTests.cs | 20 ++--- test/Garnet.test/RespAdminCommandsTests.cs | 2 +- test/Garnet.test/RespConfigTests.cs | 70 +++++++---------- test/Garnet.test/RespInfoTests.cs | 7 +- test/Garnet.test/RespSortedSetTests.cs | 4 +- 42 files changed, 179 insertions(+), 675 deletions(-) diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs index 2f9f060c415..da8c2a9c6ff 100644 --- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs @@ -257,19 +257,18 @@ public Task ExecuteSendFileSegments(Memory fileTokenBytes, int fil /// Signal replica to recover /// /// - /// /// /// /// /// /// /// - public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool sendObjectStoreCheckpoint, bool replayAOF, string primary_replid, byte[] checkpointEntryData, long beginAddress, long tailAddress) + public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool replayAOF, string primary_replid, byte[] checkpointEntryData, long beginAddress, long tailAddress) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); tcsQueue.Enqueue(tcs); byte* curr = offset; - int arraySize = 9; + int arraySize = 8; while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end)) { @@ -303,14 +302,6 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se offset = curr; //4 - while (!RespWriteUtils.TryWriteBulkString(sendObjectStoreCheckpoint ? "1"u8 : "0"u8, ref curr, end)) - { - Flush(); - curr = offset; - } - offset = curr; - - //5 while (!RespWriteUtils.TryWriteBulkString(replayAOF ? "1"u8 : "0"u8, ref curr, end)) { Flush(); @@ -318,7 +309,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se } offset = curr; - //6 + //5 while (!RespWriteUtils.TryWriteAsciiBulkString(primary_replid, ref curr, end)) { Flush(); @@ -326,7 +317,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se } offset = curr; - //7 + //6 while (!RespWriteUtils.TryWriteBulkString(checkpointEntryData, ref curr, end)) { Flush(); @@ -334,7 +325,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se } offset = curr; - //8 + //7 while (!RespWriteUtils.TryWriteArrayItem(beginAddress, ref curr, end)) { Flush(); @@ -342,7 +333,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se } offset = curr; - //9 + //8 while (!RespWriteUtils.TryWriteArrayItem(tailAddress, ref curr, end)) { Flush(); diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs index 5041f656d9a..27356859b80 100644 --- a/libs/cluster/Server/ClusterManagerSlotState.cs +++ b/libs/cluster/Server/ClusterManagerSlotState.cs @@ -16,8 +16,8 @@ namespace Garnet.cluster /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Cluster manager diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs index 3c060d35273..8b36bed74cb 100644 --- a/libs/cluster/Server/ClusterProvider.cs +++ b/libs/cluster/Server/ClusterProvider.cs @@ -19,8 +19,8 @@ namespace Garnet.cluster /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Cluster provider @@ -157,7 +157,6 @@ public void FlushConfig() 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); } /// @@ -171,12 +170,6 @@ public void SafeTruncateAOF(bool full, long CheckpointCoveredAofAddress, Guid st entry.metadata.storeCheckpointCoveredAofAddress = CheckpointCoveredAofAddress; entry.metadata.storePrimaryReplId = replicationManager.PrimaryReplId; - entry.metadata.objectStoreVersion = serverOptions.DisableObjects ? -1 : storeWrapper.objectStore.CurrentVersion; - entry.metadata.objectStoreHlogToken = serverOptions.DisableObjects ? default : objectStoreCheckpointToken; - entry.metadata.objectStoreIndexToken = serverOptions.DisableObjects ? default : objectStoreCheckpointToken; - entry.metadata.objectCheckpointCoveredAofAddress = CheckpointCoveredAofAddress; - entry.metadata.objectStorePrimaryReplId = replicationManager.PrimaryReplId; - // Keep track of checkpoints for replica // Used to delete old checkpoints and cleanup and also cleanup during attachment to new primary replicationManager.AddCheckpointEntry(entry, full); @@ -241,8 +234,6 @@ public MetricsItem[] GetReplicationInfo() new("second_repl_offset", replication_offset2), new("store_current_safe_aof_address", clusterEnabled ? replicationManager.StoreCurrentSafeAofAddress.ToString() : "N/A"), new("store_recovered_safe_aof_address", clusterEnabled ? replicationManager.StoreRecoveredSafeAofTailAddress.ToString() : "N/A"), - new("object_store_current_safe_aof_address", clusterEnabled && !serverOptions.DisableObjects ? replicationManager.ObjectStoreCurrentSafeAofAddress.ToString() : "N/A"), - new("object_store_recovered_safe_aof_address", clusterEnabled && !serverOptions.DisableObjects ? replicationManager.ObjectStoreRecoveredSafeAofTailAddress.ToString() : "N/A"), new("recover_status", replicationManager.currentRecoveryStatus.ToString()), new("last_failover_state", !clusterEnabled ? FailoverUtils.GetFailoverStatus(FailoverStatus.NO_FAILOVER) : failoverManager.GetLastFailoverStatus()) }; @@ -427,15 +418,10 @@ public void ExtractKeySpecs(RespCommandsInfo commandInfo, RespCommand cmd, ref S public void ClusterPublish(RespCommand cmd, ref Span channel, ref Span message) => clusterManager.TryClusterPublish(cmd, ref channel, ref message); - internal GarnetClusterCheckpointManager GetReplicationLogCheckpointManager(StoreType storeType) + internal GarnetClusterCheckpointManager GetReplicationLogCheckpointManager() { Debug.Assert(serverOptions.EnableCluster); - return storeType switch - { - StoreType.Main => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager, - StoreType.Object => (GarnetClusterCheckpointManager)storeWrapper.objectStore?.CheckpointManager, - _ => throw new Exception($"GetCkptManager: unexpected state {storeType}") - }; + return (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager; } /// diff --git a/libs/cluster/Server/Migration/MigrateOperation.cs b/libs/cluster/Server/Migration/MigrateOperation.cs index dcdf0721bc7..3287f7f73ad 100644 --- a/libs/cluster/Server/Migration/MigrateOperation.cs +++ b/libs/cluster/Server/Migration/MigrateOperation.cs @@ -55,23 +55,17 @@ public void Dispose() /// /// Perform scan to gather keys and build sketch /// - /// /// /// - public void Scan(StoreType storeType, ref long currentAddress, long endAddress) - { - if (storeType == StoreType.Main) - _ = localServerSession.BasicGarnetApi.IterateMainStore(ref mss, ref currentAddress, endAddress, endAddress, includeTombstones: true); - else if (storeType == StoreType.Object) - _ = localServerSession.BasicGarnetApi.IterateObjectStore(ref oss, ref currentAddress, endAddress, endAddress, includeTombstones: true); - } + public void Scan(ref long currentAddress, long endAddress) + => _ = localServerSession.BasicGarnetApi.IterateMainStore(ref mss, ref currentAddress, endAddress, + endAddress, includeTombstones: true); /// /// Transmit gathered keys /// - /// /// - public bool TransmitSlots(StoreType storeType) + public bool TransmitSlots() { // Use this for both stores; main store will just use the SpanByteAndMemory directly. We want it to be outside iterations // so we can reuse the SpanByteAndMemory.Memory across iterations. @@ -79,23 +73,11 @@ public bool TransmitSlots(StoreType storeType) try { - if (storeType == StoreType.Main) - { - var input = new RawStringInput(RespCommandAccessor.MIGRATE); - foreach (var key in sketch.argSliceVector) - { - if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output.SpanByteAndMemory, out _)) - return false; - } - } - else + var input = new RawStringInput(RespCommandAccessor.MIGRATE); + foreach (var key in sketch.argSliceVector) { - var input = new ObjectInput(new RespInputHeader(GarnetObjectType.Migrate)); - foreach (var key in sketch.argSliceVector) - { - if (!session.WriteOrSendObjectStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output, out _)) - return false; - } + if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output.SpanByteAndMemory, out _)) + return false; } // Flush final data in client buffer diff --git a/libs/cluster/Server/Migration/MigrateSessionSlots.cs b/libs/cluster/Server/Migration/MigrateSessionSlots.cs index cc139281a99..5589d503680 100644 --- a/libs/cluster/Server/Migration/MigrateSessionSlots.cs +++ b/libs/cluster/Server/Migration/MigrateSessionSlots.cs @@ -6,7 +6,6 @@ #if DEBUG using Garnet.common; #endif -using Garnet.server; using Microsoft.Extensions.Logging; namespace Garnet.cluster @@ -28,33 +27,22 @@ public async Task MigrateSlotsDriverInline() ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Migration_Slot_End_Scan_Range_Acquisition).GetAwaiter().GetResult(); #endif - // Send main store + // Send store logger?.LogWarning("Store migrate scan range [{storeBeginAddress}, {storeTailAddress}]", storeBeginAddress, storeTailAddress); - var success = await CreateAndRunMigrateTasks(StoreType.Main, storeBeginAddress, storeTailAddress, mainStorePageSize); + var success = await CreateAndRunMigrateTasks(storeBeginAddress, storeTailAddress, mainStorePageSize); if (!success) return false; - // Send object store - if (!clusterProvider.serverOptions.DisableObjects) - { - var objectStoreBeginAddress = clusterProvider.storeWrapper.objectStore.Log.BeginAddress; - var objectStoreTailAddress = clusterProvider.storeWrapper.objectStore.Log.TailAddress; - var objectStorePageSize = 1 << clusterProvider.serverOptions.ObjectStorePageSizeBits(); - logger?.LogWarning("Object Store migrate scan range [{objectStoreBeginAddress}, {objectStoreTailAddress}]", objectStoreBeginAddress, objectStoreTailAddress); - success = await CreateAndRunMigrateTasks(StoreType.Object, objectStoreBeginAddress, objectStoreTailAddress, objectStorePageSize); - if (!success) return false; - } - return true; - async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress, long tailAddress, int pageSize) + async Task CreateAndRunMigrateTasks(long beginAddress, long tailAddress, int pageSize) { - logger?.LogTrace("{method} > [{storeType}] Scan in range ({BeginAddress},{TailAddress})", nameof(CreateAndRunMigrateTasks), storeType, beginAddress, tailAddress); + logger?.LogTrace("{method} > Scan in range ({BeginAddress},{TailAddress})", nameof(CreateAndRunMigrateTasks), beginAddress, tailAddress); var migrateOperationRunners = new Task[clusterProvider.serverOptions.ParallelMigrateTaskCount]; var i = 0; while (i < migrateOperationRunners.Length) { var idx = i; - migrateOperationRunners[idx] = Task.Run(() => ScanStoreTask(idx, storeType, beginAddress, tailAddress, pageSize)); + migrateOperationRunners[idx] = Task.Run(() => ScanStoreTask(idx, beginAddress, tailAddress, pageSize)); i++; } @@ -64,14 +52,14 @@ async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress } catch (Exception ex) { - logger?.LogError(ex, "{CreateAndRunMigrateTasks}: {storeType} {beginAddress} {tailAddress} {pageSize}", nameof(CreateAndRunMigrateTasks), storeType, beginAddress, tailAddress, pageSize); + logger?.LogError(ex, "{CreateAndRunMigrateTasks}: {beginAddress} {tailAddress} {pageSize}", nameof(CreateAndRunMigrateTasks), beginAddress, tailAddress, pageSize); _cts.Cancel(); return false; } return true; } - Task ScanStoreTask(int taskId, StoreType storeType, long beginAddress, long tailAddress, int pageSize) + Task ScanStoreTask(int taskId, long beginAddress, long tailAddress, int pageSize) { var migrateOperation = this.migrateOperation[taskId]; var range = (tailAddress - beginAddress) / clusterProvider.storeWrapper.serverOptions.ParallelMigrateTaskCount; @@ -84,13 +72,13 @@ Task ScanStoreTask(int taskId, StoreType storeType, long beginAddress, lon return Task.FromResult(false); var cursor = workerStartAddress; - logger?.LogWarning("<{StoreType}:{taskId}> migrate scan range [{workerStartAddress}, {workerEndAddress}]", storeType, taskId, workerStartAddress, workerEndAddress); + logger?.LogWarning("<{taskId}> migrate scan range [{workerStartAddress}, {workerEndAddress}]", taskId, workerStartAddress, workerEndAddress); while (true) { var current = cursor; // Build Sketch migrateOperation.sketch.SetStatus(SketchStatus.INITIALIZING); - migrateOperation.Scan(storeType, ref current, workerEndAddress); + migrateOperation.Scan(ref current, workerEndAddress); // Stop if no keys have been found if (migrateOperation.sketch.argSliceVector.IsEmpty) break; @@ -103,7 +91,7 @@ Task ScanStoreTask(int taskId, StoreType storeType, long beginAddress, lon WaitForConfigPropagation(); // Transmit all keys gathered - migrateOperation.TransmitSlots(storeType); + migrateOperation.TransmitSlots(); // Transition EPSM to DELETING migrateOperation.sketch.SetStatus(SketchStatus.DELETING); diff --git a/libs/cluster/Server/Replication/CheckpointEntry.cs b/libs/cluster/Server/Replication/CheckpointEntry.cs index a6ba395f633..9c57eb7c2af 100644 --- a/libs/cluster/Server/Replication/CheckpointEntry.cs +++ b/libs/cluster/Server/Replication/CheckpointEntry.cs @@ -22,22 +22,13 @@ public static void LogCheckpointEntry(this ILogger logger, LogLevel logLevel, st "storeHlogToken: {storeHlogToken}\n" + "storeIndexToken: {storeIndexToken}\n" + "storeCheckpointCoveredAofAddress: {storeCheckpointCoveredAofAddress}\n" + - "------------------------------------------------------------------------\n" + - "objectStoreVersion:{objectStoreVersion}\n" + - "objectStoreHlogToken:{objectStoreHlogToken}\n" + - "objectStoreIndexToken:{objectStoreIndexToken}\n" + - "objectCheckpointCoveredAofAddress:{objectCheckpointCoveredAofAddress}\n" + "------------------------------------------------------------------------\n", msg, entry._lock, entry.metadata.storeVersion, entry.metadata.storeHlogToken, entry.metadata.storeIndexToken, - entry.metadata.storeCheckpointCoveredAofAddress, - entry.metadata.objectStoreVersion, - entry.metadata.objectStoreHlogToken, - entry.metadata.objectStoreIndexToken, - entry.metadata.objectCheckpointCoveredAofAddress); + entry.metadata.storeCheckpointCoveredAofAddress); } } @@ -55,7 +46,7 @@ public CheckpointEntry() } public long GetMinAofCoveredAddress() - => Math.Max(Math.Min(metadata.storeCheckpointCoveredAofAddress, metadata.objectCheckpointCoveredAofAddress), LogAddress.FirstValidAddress); + => Math.Max(metadata.storeCheckpointCoveredAofAddress, LogAddress.FirstValidAddress); /// /// Indicate addition of new reader by trying to increment reader counter @@ -90,8 +81,6 @@ public bool ContainsSharedToken(CheckpointEntry entry, CheckpointFileType fileTy { CheckpointFileType.STORE_HLOG => metadata.storeHlogToken.Equals(entry.metadata.storeHlogToken), CheckpointFileType.STORE_INDEX => metadata.storeIndexToken.Equals(entry.metadata.storeIndexToken), - CheckpointFileType.OBJ_STORE_HLOG => metadata.objectStoreHlogToken.Equals(entry.metadata.objectStoreHlogToken), - CheckpointFileType.OBJ_STORE_INDEX => metadata.objectStoreIndexToken.Equals(entry.metadata.objectStoreIndexToken), _ => throw new Exception($"Option {fileType} not supported") }; } @@ -118,18 +107,6 @@ public byte[] ToByteArray() writer.Write(metadata.storePrimaryReplId == null ? 0 : 1); if (metadata.storePrimaryReplId != null) writer.Write(metadata.storePrimaryReplId); - // Write checkpoint entry data for object store - writer.Write(metadata.objectStoreVersion); - byteBuffer = metadata.objectStoreHlogToken.ToByteArray(); - writer.Write(byteBuffer.Length); - writer.Write(byteBuffer); - byteBuffer = metadata.objectStoreIndexToken.ToByteArray(); - writer.Write(byteBuffer.Length); - writer.Write(byteBuffer); - writer.Write(metadata.objectCheckpointCoveredAofAddress); - writer.Write(metadata.objectStorePrimaryReplId == null ? 0 : 1); - if (metadata.objectStorePrimaryReplId != null) writer.Write(metadata.objectStorePrimaryReplId); - var byteArray = ms.ToArray(); writer.Dispose(); ms.Dispose(); @@ -154,13 +131,7 @@ public static CheckpointEntry FromByteArray(byte[] serialized) storeHlogToken = new Guid(reader.ReadBytes(reader.ReadInt32())), storeIndexToken = new Guid(reader.ReadBytes(reader.ReadInt32())), storeCheckpointCoveredAofAddress = reader.ReadInt64(), - storePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default, - - objectStoreVersion = reader.ReadInt64(), - objectStoreHlogToken = new Guid(reader.ReadBytes(reader.ReadInt32())), - objectStoreIndexToken = new Guid(reader.ReadBytes(reader.ReadInt32())), - objectCheckpointCoveredAofAddress = reader.ReadInt64(), - objectStorePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default + storePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default } }; diff --git a/libs/cluster/Server/Replication/CheckpointFileType.cs b/libs/cluster/Server/Replication/CheckpointFileType.cs index da2ac9690ba..e20bb1e28fa 100644 --- a/libs/cluster/Server/Replication/CheckpointFileType.cs +++ b/libs/cluster/Server/Replication/CheckpointFileType.cs @@ -1,9 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; -using Garnet.server; - namespace Garnet.cluster { /// @@ -31,49 +28,5 @@ enum CheckpointFileType : byte /// Store Snapshot /// STORE_SNAPSHOT, - /// - /// Object Store Hybrid Log - Main - /// - OBJ_STORE_HLOG, - /// - /// Object Store Hybrid Log - Object - /// - OBJ_STORE_HLOG_OBJ, - /// - /// Object Store Delta Log - /// - OBJ_STORE_DLOG, - /// - /// Object Store Index - /// - OBJ_STORE_INDEX, - /// - /// Object Store Snapshot - Main - /// - OBJ_STORE_SNAPSHOT, - /// - /// Object Store Snapshot - Object - /// - OBJ_STORE_SNAPSHOT_OBJ, - } - - static class CheckpointFileTypeExtensions - { - public static StoreType ToStoreType(this CheckpointFileType type) - { - return type switch - { - CheckpointFileType.STORE_HLOG or - CheckpointFileType.STORE_DLOG or - CheckpointFileType.STORE_INDEX or - CheckpointFileType.STORE_SNAPSHOT => StoreType.Main, - CheckpointFileType.OBJ_STORE_HLOG or - CheckpointFileType.OBJ_STORE_DLOG or - CheckpointFileType.OBJ_STORE_INDEX or - CheckpointFileType.OBJ_STORE_SNAPSHOT or - CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ => StoreType.Object, - _ => throw new Exception($"ToStoreType: unexpected state {type}") - }; - } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/CheckpointStore.cs b/libs/cluster/Server/Replication/CheckpointStore.cs index e3d8cc44045..4bc55205f83 100644 --- a/libs/cluster/Server/Replication/CheckpointStore.cs +++ b/libs/cluster/Server/Replication/CheckpointStore.cs @@ -40,7 +40,7 @@ public void Initialize() { head = tail = GetLatestCheckpointEntryFromDisk(); - if (tail.metadata.storeVersion == -1 && tail.metadata.objectStoreVersion == -1) + if (tail.metadata.storeVersion == -1) { head = tail = null; } @@ -48,11 +48,6 @@ public void Initialize() { clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredSafeAofAddress = tail.metadata.storeCheckpointCoveredAofAddress; clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredHistoryId = tail.metadata.storePrimaryReplId; - if (!storeWrapper.serverOptions.DisableObjects) - { - clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredSafeAofAddress = tail.metadata.storeCheckpointCoveredAofAddress; - clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredHistoryId = tail.metadata.storePrimaryReplId; - } } // This purge does not check for active readers @@ -85,13 +80,11 @@ public void PurgeAllCheckpointsExceptEntry(CheckpointEntry entry = null) entry ??= GetLatestCheckpointEntryFromDisk(); if (entry == null) return; logger?.LogCheckpointEntry(LogLevel.Trace, nameof(PurgeAllCheckpointsExceptEntry), entry); - PurgeAllCheckpointsExceptTokens(StoreType.Main, entry.metadata.storeHlogToken, entry.metadata.storeIndexToken); - if (!clusterProvider.serverOptions.DisableObjects) - PurgeAllCheckpointsExceptTokens(StoreType.Object, entry.metadata.objectStoreHlogToken, entry.metadata.objectStoreIndexToken); - - void PurgeAllCheckpointsExceptTokens(StoreType storeType, Guid logToken, Guid indexToken) + PurgeAllCheckpointsExceptTokens(entry.metadata.storeHlogToken, entry.metadata.storeIndexToken); + + void PurgeAllCheckpointsExceptTokens(Guid logToken, Guid indexToken) { - var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(storeType); + var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(); // Delete log checkpoints foreach (var toDeletelogToken in ckptManager.GetLogCheckpointTokens()) @@ -129,7 +122,6 @@ public void AddCheckpointEntry(CheckpointEntry entry, bool fullCheckpoint = fals { var lastEntry = tail ?? throw new GarnetException($"Checkpoint history unavailable, need full checkpoint for {entry}"); entry.metadata.storeIndexToken = lastEntry.metadata.storeIndexToken; - entry.metadata.objectStoreIndexToken = lastEntry.metadata.objectStoreIndexToken; } _ = ValidateCheckpointEntry(entry); @@ -156,9 +148,6 @@ bool ValidateCheckpointEntry(CheckpointEntry entry) if (!clusterProvider.replicationManager.TryAcquireSettledMetadataForMainStore(entry, out _, out _)) throw new GarnetException("Failed to validate main store metadata at insertion"); - if (!clusterProvider.serverOptions.DisableObjects && !clusterProvider.replicationManager.TryAcquireSettledMetadataForObjectStore(entry, out _, out _)) - throw new GarnetException("Failed to validate object store metadata at insertion"); - return true; } catch (Exception ex) @@ -191,22 +180,11 @@ private void DeleteOutdatedCheckpoints() // Below check each checkpoint token separately if it is eligible for deletion if (!CanDeleteToken(curr, CheckpointFileType.STORE_HLOG)) break; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).DeleteLogCheckpoint(curr.metadata.storeHlogToken); + clusterProvider.GetReplicationLogCheckpointManager().DeleteLogCheckpoint(curr.metadata.storeHlogToken); if (!CanDeleteToken(curr, CheckpointFileType.STORE_INDEX)) break; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).DeleteIndexCheckpoint(curr.metadata.storeIndexToken); - - if (!clusterProvider.serverOptions.DisableObjects) - { - if (!CanDeleteToken(curr, CheckpointFileType.OBJ_STORE_HLOG)) - break; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).DeleteLogCheckpoint(curr.metadata.objectStoreHlogToken); - - if (!CanDeleteToken(curr, CheckpointFileType.OBJ_STORE_INDEX)) - break; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).DeleteIndexCheckpoint(curr.metadata.objectStoreIndexToken); - } + clusterProvider.GetReplicationLogCheckpointManager().DeleteIndexCheckpoint(curr.metadata.storeIndexToken); logger?.LogCheckpointEntry(LogLevel.Warning, "Deleting outdated checkpoint", curr); @@ -262,7 +240,6 @@ public bool TryGetLatestCheckpointEntryFromMemory(out CheckpointEntry cEntry) metadata = new() { storeCheckpointCoveredAofAddress = 0, - objectCheckpointCoveredAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : 0 } }; _ = cEntry.TryAddReader(); @@ -282,13 +259,8 @@ public bool TryGetLatestCheckpointEntryFromMemory(out CheckpointEntry cEntry) /// public CheckpointEntry GetLatestCheckpointEntryFromDisk() { - Guid objectStoreHLogToken = default; - Guid objectStoreIndexToken = default; - var objectStoreVersion = -1L; storeWrapper.store.GetLatestCheckpointTokens(out var storeHLogToken, out var storeIndexToken, out var storeVersion); - storeWrapper.objectStore?.GetLatestCheckpointTokens(out objectStoreHLogToken, out objectStoreIndexToken, out objectStoreVersion); - var (storeCheckpointCoveredAofAddress, storePrimaryReplId) = GetCheckpointCookieMetadata(StoreType.Main, storeHLogToken); - var (objectCheckpointCoveredAofAddress, objectStorePrimaryReplId) = objectStoreHLogToken == default ? (long.MaxValue, null) : GetCheckpointCookieMetadata(StoreType.Object, objectStoreHLogToken); + var (storeCheckpointCoveredAofAddress, storePrimaryReplId) = GetCheckpointCookieMetadata(storeHLogToken); CheckpointEntry entry = new() { @@ -299,21 +271,15 @@ public CheckpointEntry GetLatestCheckpointEntryFromDisk() storeIndexToken = storeIndexToken, storeCheckpointCoveredAofAddress = storeCheckpointCoveredAofAddress, storePrimaryReplId = storePrimaryReplId, - - objectStoreVersion = objectStoreVersion, - objectStoreHlogToken = objectStoreHLogToken, - objectStoreIndexToken = objectStoreIndexToken, - objectCheckpointCoveredAofAddress = objectCheckpointCoveredAofAddress, - objectStorePrimaryReplId = objectStorePrimaryReplId, } }; return entry; - (long RecoveredSafeAofAddress, string RecoveredReplicationId) GetCheckpointCookieMetadata(StoreType storeType, Guid fileToken) + (long RecoveredSafeAofAddress, string RecoveredReplicationId) GetCheckpointCookieMetadata(Guid fileToken) { if (fileToken == default) return (0, null); - var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(storeType); - var pageSizeBits = storeType == StoreType.Main ? clusterProvider.serverOptions.PageSizeBits() : clusterProvider.serverOptions.ObjectStorePageSizeBits(); + var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(); + var pageSizeBits = clusterProvider.serverOptions.PageSizeBits(); using (var deltaFileDevice = ckptManager.GetDeltaLogDevice(fileToken)) { if (deltaFileDevice is not null) diff --git a/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs b/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs index 3bf7ec5b5fe..36829e9abb1 100644 --- a/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs +++ b/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs @@ -58,10 +58,6 @@ public IDevice GetDevice(CheckpointFileType retStateType, Guid fileToken) CheckpointFileType.STORE_DLOG => GetDeltaLogDevice(fileToken), CheckpointFileType.STORE_INDEX => GetIndexDevice(fileToken), CheckpointFileType.STORE_SNAPSHOT => GetSnapshotLogDevice(fileToken), - CheckpointFileType.OBJ_STORE_DLOG => GetDeltaLogDevice(fileToken), - CheckpointFileType.OBJ_STORE_INDEX => GetIndexDevice(fileToken), - CheckpointFileType.OBJ_STORE_SNAPSHOT => GetSnapshotLogDevice(fileToken), - CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ => GetSnapshotObjectLogDevice(fileToken), _ => throw new Exception($"RetrieveCheckpointFile: unexpected state{retStateType}") }; return device; diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index c31f1b4b2bd..69df96fffca 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -30,8 +30,6 @@ internal sealed partial class ReplicaSyncSession public long currentStoreVersion; - public long currentObjectStoreVersion; - /// /// Pessimistic checkpoint covered AOF address /// @@ -192,7 +190,6 @@ public bool NeedToFullSync() var localPrimaryReplId = clusterProvider.replicationManager.PrimaryReplId; var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal); var sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion; - var sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion; var aofBeginAddress = clusterProvider.storeWrapper.appendOnlyFile.BeginAddress; var aofTailAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; @@ -202,11 +199,10 @@ public bool NeedToFullSync() // We need to stream checkpoint if any of the following conditions are met: // 1. Replica has different history than primary - // 2. Replica has different main store version than primary - // 3. Replica has different object store version than primary - // 4. Replica has truncated AOF - // 5. The AOF to be replayed in case of a partial sync is larger than the specified threshold - fullSync = sendMainStore || sendObjectStore || outOfRangeAof || aofTooLarge; + // 2. Replica has different store version than primary + // 3. Replica has truncated AOF + // 4. The AOF to be replayed in case of a partial sync is larger than the specified threshold + fullSync = sendMainStore || outOfRangeAof || aofTooLarge; return fullSync; } @@ -227,7 +223,6 @@ public async Task BeginAofSync() originNodeId: clusterProvider.clusterManager.CurrentConfig.LocalNodeId, currentPrimaryReplId: clusterProvider.replicationManager.PrimaryReplId, currentStoreVersion: currentStoreVersion, - currentObjectStoreVersion: currentObjectStoreVersion, currentAofBeginAddress: currentAofBeginAddress, currentAofTailAddress: currentAofTailAddress, currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset, diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index d22d1cc25d0..ae96ddd972a 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -13,12 +13,10 @@ namespace Garnet.cluster internal sealed unsafe class SnapshotIteratorManager { public readonly ReplicationSyncManager replicationSyncManager; - public readonly TimeSpan timeout; public readonly CancellationToken cancellationToken; public readonly ILogger logger; - public MainStoreSnapshotIterator mainStoreSnapshotIterator; - public ObjectStoreSnapshotIterator objectStoreSnapshotIterator; + public StoreSnapshotIterator StoreSnapshotIterator; // For serialization from LogRecord to DiskLogRecord SpanByteAndMemory serializationOutput; @@ -50,9 +48,7 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ca sessions[i].checkpointCoveredAofAddress = CheckpointCoveredAddress; } - mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); - if (!replicationSyncManager.ClusterProvider.serverOptions.DisableObjects) - objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this); + StoreSnapshotIterator = new StoreSnapshotIterator(this); memoryPool = MemoryPool.Shared; valueObjectSerializer = new(customCommandManager: default); @@ -76,7 +72,7 @@ public bool IsProgressing() } } - public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion, bool isMainStore) + public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion) { if (cancellationToken.IsCancellationRequested) { @@ -92,14 +88,11 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio if (!replicationSyncManager.IsActive(i)) continue; sessions[i].InitializeIterationBuffer(); - if (isMainStore) - sessions[i].currentStoreVersion = targetVersion; - else - sessions[i].currentObjectStoreVersion = targetVersion; + sessions[i].currentStoreVersion = targetVersion; } - logger?.LogTrace("{OnStart} {store} {token} {currentVersion} {targetVersion}", - nameof(OnStart), isMainStore ? "MAIN STORE" : "OBJECT STORE", checkpointToken, currentVersion, targetVersion); + logger?.LogTrace("{OnStart} {token} {currentVersion} {targetVersion}", + nameof(OnStart), checkpointToken, currentVersion, targetVersion); return true; } @@ -229,16 +222,15 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long } } - internal sealed unsafe class MainStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) : + internal sealed unsafe class StoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) : IStreamingSnapshotIteratorFunctions { - readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager; long targetVersion; public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion) { this.targetVersion = targetVersion; - return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: true); + return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion); } public bool Reader(in TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) @@ -246,32 +238,9 @@ public bool Reader(in TSourceLogRecord srcLogRecord, RecordMet => snapshotIteratorManager.StringReader(in srcLogRecord, recordMetadata, numberOfRecords); public void OnException(Exception exception, long numberOfRecords) - => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(MainStoreSnapshotIterator)}"); + => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(StoreSnapshotIterator)}"); public void OnStop(bool completed, long numberOfRecords) => snapshotIteratorManager.OnStop(completed, numberOfRecords, isMainStore: true, targetVersion); } - - internal sealed unsafe class ObjectStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) : - IStreamingSnapshotIteratorFunctions - { - readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager; - long targetVersion; - - public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion) - { - this.targetVersion = targetVersion; - return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: false); - } - - public bool Reader(in TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) - where TSourceLogRecord : ISourceLogRecord - => snapshotIteratorManager.ObjectReader(in srcLogRecord, recordMetadata, numberOfRecords); - - public void OnException(Exception exception, long numberOfRecords) - => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(ObjectStoreSnapshotIterator)}"); - - public void OnStop(bool completed, long numberOfRecords) - => snapshotIteratorManager.OnStop(completed, numberOfRecords, isMainStore: false, targetVersion); - } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index e57e0f37b33..86ebf79378d 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -242,7 +242,6 @@ async Task PrepareForSync() // Set store version to operate on Sessions[i].currentStoreVersion = ClusterProvider.storeWrapper.store.CurrentVersion; - Sessions[i].currentObjectStoreVersion = disableObjects ? -1 : ClusterProvider.storeWrapper.objectStore.CurrentVersion; // If checkpoint is not needed mark this sync session as complete // to avoid waiting for other replicas which may need to receive the latest checkpoint @@ -280,22 +279,12 @@ async Task TakeStreamingCheckpoint() // Iterate through main store var mainStoreCheckpointTask = ClusterProvider.storeWrapper.store. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.StoreSnapshotIterator); var result = await WaitOrDie(checkpointTask: mainStoreCheckpointTask, iteratorManager: manager); if (!result.success) throw new InvalidOperationException("Main store checkpoint stream failed!"); - if (!ClusterProvider.serverOptions.DisableObjects) - { - // Iterate through object store - var objectStoreCheckpointTask = ClusterProvider.storeWrapper.objectStore. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); - result = await WaitOrDie(checkpointTask: objectStoreCheckpointTask, iteratorManager: manager); - if (!result.success) - throw new InvalidOperationException("Object store checkpoint stream failed!"); - } - // Note: We do not truncate the AOF here as this was just a "virtual" checkpoint async ValueTask<(bool success, Guid token)> WaitOrDie(ValueTask<(bool success, Guid token)> checkpointTask, SnapshotIteratorManager iteratorManager) diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index 54ecd50a934..20b64d1e447 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -61,37 +61,25 @@ public bool ValidateMetadata( CheckpointEntry localEntry, out long index_size, out LogFileInfo hlog_size, - out long obj_index_size, - out LogFileInfo obj_hlog_size, - out bool skipLocalMainStoreCheckpoint, - out bool skipLocalObjectStoreCheckpoint) + out bool skipLocalMainStoreCheckpoint) { hlog_size = default; - obj_hlog_size = default; index_size = -1L; - obj_index_size = -1L; // Local and remote checkpoints are of same history if both of the following hold // 1. There is a checkpoint available at remote node // 2. Remote and local checkpoints contain the same PrimaryReplId var sameMainStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.storePrimaryReplId) && replicaCheckpointEntry.metadata.storePrimaryReplId.Equals(localEntry.metadata.storePrimaryReplId); - var sameObjectStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.objectStorePrimaryReplId) && replicaCheckpointEntry.metadata.objectStorePrimaryReplId.Equals(localEntry.metadata.objectStorePrimaryReplId); // We will not send the latest local checkpoint if any of the following hold // 1. Local node does not have any checkpoints // 2. Local checkpoint is of same version and history as the remote checkpoint skipLocalMainStoreCheckpoint = localEntry.metadata.storeHlogToken == default || (sameMainStoreCheckpointHistory && localEntry.metadata.storeVersion == replicaCheckpointEntry.metadata.storeVersion); - skipLocalObjectStoreCheckpoint = clusterProvider.serverOptions.DisableObjects || localEntry.metadata.objectStoreHlogToken == default || (sameObjectStoreCheckpointHistory && localEntry.metadata.objectStoreVersion == replicaCheckpointEntry.metadata.objectStoreVersion); // Acquire metadata for main store // If failed then this checkpoint is not usable because it is corrupted if (!skipLocalMainStoreCheckpoint && !clusterProvider.replicationManager.TryAcquireSettledMetadataForMainStore(localEntry, out hlog_size, out index_size)) return false; - // Acquire metadata for object store - // If failed then this checkpoint is not usable because it is corrupted - if (!skipLocalObjectStoreCheckpoint && !clusterProvider.replicationManager.TryAcquireSettledMetadataForObjectStore(localEntry, out obj_hlog_size, out obj_index_size)) - return false; - return true; } @@ -101,8 +89,7 @@ public bool ValidateMetadata( public async Task SendCheckpoint() { errorMsg = default; - var storeCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main); - var objectStoreCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object); + var storeCkptManager = clusterProvider.GetReplicationLogCheckpointManager(); var current = clusterProvider.clusterManager.CurrentConfig; var (address, port) = current.GetWorkerAddressFromNodeId(replicaNodeId); @@ -126,8 +113,8 @@ public async Task SendCheckpoint() try { - logger?.LogInformation("Replica replicaId:{replicaId} requesting checkpoint replicaStoreVersion:{replicaStoreVersion} replicaObjectStoreVersion:{replicaObjectStoreVersion}", - replicaNodeId, replicaCheckpointEntry.metadata.storeVersion, replicaCheckpointEntry.metadata.objectStoreVersion); + logger?.LogInformation("Replica replicaId:{replicaId} requesting checkpoint replicaStoreVersion:{replicaStoreVersion}", + replicaNodeId, replicaCheckpointEntry.metadata.storeVersion); logger?.LogInformation("Attempting to acquire checkpoint"); (localEntry, aofSyncTaskInfo) = await AcquireCheckpointEntry(); @@ -136,13 +123,10 @@ public async Task SendCheckpoint() gcs.Connect((int)storeWrapper.serverOptions.ReplicaSyncTimeout.TotalMilliseconds); long index_size = -1; - long obj_index_size = -1; var hlog_size = default(LogFileInfo); - var obj_hlog_size = default(LogFileInfo); var skipLocalMainStoreCheckpoint = false; - var skipLocalObjectStoreCheckpoint = false; var retryCount = validateMetadataMaxRetryCount; - while (!ValidateMetadata(localEntry, out index_size, out hlog_size, out obj_index_size, out obj_hlog_size, out skipLocalMainStoreCheckpoint, out skipLocalObjectStoreCheckpoint)) + while (!ValidateMetadata(localEntry, out index_size, out hlog_size, out skipLocalMainStoreCheckpoint)) { logger?.LogError("Failed to validate metadata. Retrying...."); await Task.Yield(); @@ -177,50 +161,10 @@ public async Task SendCheckpoint() await SendCheckpointMetadata(gcs, storeCkptManager, CheckpointFileType.STORE_SNAPSHOT, localEntry.metadata.storeHlogToken); } - if (!skipLocalObjectStoreCheckpoint) - { - logger?.LogInformation("Sending object store checkpoint {version} {objectStoreHlogToken} {objectStoreIndexToken} to replica", localEntry.metadata.objectStoreVersion, localEntry.metadata.objectStoreHlogToken, localEntry.metadata.objectStoreIndexToken); - - // 1. send hlog file segments - if (clusterProvider.serverOptions.EnableStorageTier && obj_hlog_size.hybridLogFileEndAddress > 24) - { - //send object hlog file segments - await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_HLOG, obj_hlog_size.hybridLogFileStartAddress, obj_hlog_size.hybridLogFileEndAddress); - - var hlogSegmentCount = ((obj_hlog_size.hybridLogFileEndAddress - obj_hlog_size.hybridLogFileStartAddress) >> clusterProvider.serverOptions.ObjectStoreSegmentSizeBits()) + 1; - await SendObjectFiles(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_HLOG_OBJ, (int)hlogSegmentCount); - } - - // 2. Send object store snapshot files - if (obj_hlog_size.snapshotFileEndAddress > 24) - { - //send snapshot file segments - await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_SNAPSHOT, 0, obj_hlog_size.snapshotFileEndAddress); - - //send snapshot.obj file segments - var snapshotSegmentCount = (obj_hlog_size.snapshotFileEndAddress >> clusterProvider.serverOptions.ObjectStoreSegmentSizeBits()) + 1; - await SendObjectFiles(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ, (int)snapshotSegmentCount); - } - - // 3. Send object store index file segments - if (obj_index_size > 0) - await SendFileSegments(gcs, localEntry.metadata.objectStoreIndexToken, CheckpointFileType.OBJ_STORE_INDEX, 0, obj_index_size); - - // 4. Send object store delta file segments - var obj_dlog_size = obj_hlog_size.deltaLogTailAddress; - if (obj_dlog_size > 0) - await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_DLOG, 0, obj_dlog_size); - - // 5. Send object store index metadata - await SendCheckpointMetadata(gcs, objectStoreCkptManager, CheckpointFileType.OBJ_STORE_INDEX, localEntry.metadata.objectStoreIndexToken); - - // 6. Send object store snapshot metadata - await SendCheckpointMetadata(gcs, objectStoreCkptManager, CheckpointFileType.OBJ_STORE_SNAPSHOT, localEntry.metadata.objectStoreHlogToken); - } #endregion #region startAofSync - var recoverFromRemote = !skipLocalMainStoreCheckpoint || !skipLocalObjectStoreCheckpoint; + var recoverFromRemote = !skipLocalMainStoreCheckpoint; var replayAOF = false; var checkpointAofBeginAddress = localEntry.GetMinAofCoveredAddress(); var beginAddress = checkpointAofBeginAddress; @@ -266,8 +210,7 @@ public async Task SendCheckpoint() } var sameMainStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.storePrimaryReplId) && replicaCheckpointEntry.metadata.storePrimaryReplId.Equals(localEntry.metadata.storePrimaryReplId); - var sameObjectStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.objectStorePrimaryReplId) && replicaCheckpointEntry.metadata.objectStorePrimaryReplId.Equals(localEntry.metadata.objectStorePrimaryReplId); - if (!sameMainStoreCheckpointHistory || !sameObjectStoreCheckpointHistory) + if (!sameMainStoreCheckpointHistory) { // If we are not in the same checkpoint history, we need to stream the AOF from the primary's beginning address checkpointAofBeginAddress = beginAddress; @@ -281,7 +224,6 @@ public async Task SendCheckpoint() // Make replica replayAOF if needed and replay from provided beginAddress to RecoveredReplication Address var resp = await gcs.ExecuteBeginReplicaRecover( !skipLocalMainStoreCheckpoint, - !skipLocalObjectStoreCheckpoint, replayAOF, clusterProvider.replicationManager.PrimaryReplId, localEntry.ToByteArray(), @@ -395,7 +337,7 @@ public async Task SendCheckpoint() // If there is possible AOF data loss and we need to take an on-demand checkpoint, // then we should take the checkpoint before we register the sync task, because // TryAddReplicationTask is guaranteed to return true in this scenario. - var validMetadata = ValidateMetadata(cEntry, out _, out _, out _, out _, out _, out _); + var validMetadata = ValidateMetadata(cEntry, out _, out _, out _); if (clusterProvider.serverOptions.OnDemandCheckpoint && (startAofAddress < clusterProvider.replicationManager.AofTruncatedUntil || !validMetadata)) { @@ -443,11 +385,9 @@ private async Task SendCheckpointMetadata(GarnetClientSession gcs, GarnetCluster switch (fileType) { case CheckpointFileType.STORE_SNAPSHOT: - case CheckpointFileType.OBJ_STORE_SNAPSHOT: checkpointMetadata = ckptManager.GetLogCheckpointMetadata(fileToken, null, true, -1); break; case CheckpointFileType.STORE_INDEX: - case CheckpointFileType.OBJ_STORE_INDEX: checkpointMetadata = ckptManager.GetIndexCheckpointMetadata(fileToken); break; } diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 01b577e5027..1aa8c3e63e8 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -113,7 +113,6 @@ async Task TryBeginReplicaSync(bool downgradeLock) originNodeId: current.LocalNodeId, currentPrimaryReplId: PrimaryReplId, currentStoreVersion: storeWrapper.store.CurrentVersion, - currentObjectStoreVersion: disableObjects ? -1 : storeWrapper.objectStore.CurrentVersion, currentAofBeginAddress: storeWrapper.appendOnlyFile.BeginAddress, currentAofTailAddress: storeWrapper.appendOnlyFile.TailAddress, currentReplicationOffset: ReplicationOffset, @@ -171,9 +170,7 @@ public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata, out ReadOnl // Set DB version storeWrapper.store.SetVersion(primarySyncMetadata.currentStoreVersion); - if (!clusterProvider.serverOptions.DisableObjects) - storeWrapper.objectStore.SetVersion(primarySyncMetadata.currentObjectStoreVersion); - + // Update replicationId to mark any subsequent checkpoints as part of this history logger?.LogInformation("Updating ReplicationId"); TryUpdateMyPrimaryReplId(primarySyncMetadata.currentPrimaryReplId); diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs index 4a1cde176a4..081beb6b377 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs @@ -9,7 +9,6 @@ using System.Threading.Tasks; using Garnet.client; using Garnet.cluster.Server.Replication; -using Garnet.server; using Microsoft.Extensions.Logging; using Tsavorite.core; @@ -185,20 +184,16 @@ public void ProcessCheckpointMetadata(Guid fileToken, CheckpointFileType fileTyp var ckptManager = fileType switch { CheckpointFileType.STORE_SNAPSHOT or - CheckpointFileType.STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main), - CheckpointFileType.OBJ_STORE_SNAPSHOT or - CheckpointFileType.OBJ_STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object), + CheckpointFileType.STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(), _ => throw new Exception($"Invalid checkpoint filetype {fileType}"), }; switch (fileType) { case CheckpointFileType.STORE_SNAPSHOT: - case CheckpointFileType.OBJ_STORE_SNAPSHOT: ckptManager.CommitLogCheckpointSendFromPrimary(fileToken, checkpointMetadata); break; case CheckpointFileType.STORE_INDEX: - case CheckpointFileType.OBJ_STORE_INDEX: ckptManager.CommitIndexCheckpoint(fileToken, checkpointMetadata); break; default: @@ -217,9 +212,7 @@ public static bool ShouldInitialize(CheckpointFileType type) return type switch { CheckpointFileType.STORE_HLOG or - CheckpointFileType.STORE_SNAPSHOT or - CheckpointFileType.OBJ_STORE_HLOG or - CheckpointFileType.OBJ_STORE_SNAPSHOT + CheckpointFileType.STORE_SNAPSHOT => true, _ => false, }; @@ -236,9 +229,7 @@ public IDevice GetInitializedSegmentFileDevice(Guid token, CheckpointFileType ty var device = type switch { CheckpointFileType.STORE_HLOG => GetStoreHLogDevice(), - CheckpointFileType.OBJ_STORE_HLOG => GetObjectStoreHLogDevice(false),//TODO: return device for object store hlog - CheckpointFileType.OBJ_STORE_HLOG_OBJ => GetObjectStoreHLogDevice(true), - _ => clusterProvider.GetReplicationLogCheckpointManager(type.ToStoreType()).GetDevice(type, token), + _ => clusterProvider.GetReplicationLogCheckpointManager().GetDevice(type, token), }; if (ShouldInitialize(type)) @@ -257,19 +248,6 @@ IDevice GetStoreHLogDevice() } return null; } - - IDevice GetObjectStoreHLogDevice(bool obj) - { - var opts = clusterProvider.serverOptions; - if (opts.EnableStorageTier) - { - var LogDir = opts.LogDir; - if (LogDir is null or "") LogDir = Directory.GetCurrentDirectory(); - var logFactory = opts.GetInitializedDeviceFactory(LogDir); - return obj ? logFactory.Get(new FileDescriptor("ObjectStore", "hlog.obj")) : logFactory.Get(new FileDescriptor("ObjectStore", "hlog")); - } - return null; - } } /// @@ -298,14 +276,10 @@ public long BeginReplicaRecover( errorMessage = []; UpdateLastPrimarySyncTime(); - logger?.LogInformation("Replica Recover Store: {storeVersion}>[{sIndexToken} {sHlogToken}]" + - "\nObjectStore: {objectStoreVersion}>[{oIndexToken} {oHlogToken}]", + logger?.LogInformation("Replica Recover Store: {storeVersion}>[{sIndexToken} {sHlogToken}]", remoteCheckpoint.metadata.storeVersion, remoteCheckpoint.metadata.storeIndexToken, - remoteCheckpoint.metadata.storeHlogToken, - remoteCheckpoint.metadata.objectStoreVersion, - remoteCheckpoint.metadata.objectStoreIndexToken, - remoteCheckpoint.metadata.objectStoreHlogToken); + remoteCheckpoint.metadata.storeHlogToken); storeWrapper.RecoverCheckpoint( replicaRecover: true, @@ -333,12 +307,6 @@ public long BeginReplicaRecover( cEntry.metadata.storeHlogToken = remoteCheckpoint.metadata.storeHlogToken; } - // If checkpoint for object store was send add its token here in preparation for purge later on - if (recoverObjectStoreFromToken) - { - cEntry.metadata.objectStoreIndexToken = remoteCheckpoint.metadata.objectStoreIndexToken; - cEntry.metadata.objectStoreHlogToken = remoteCheckpoint.metadata.objectStoreHlogToken; - } checkpointStore.PurgeAllCheckpointsExceptEntry(cEntry); // Initialize in-memory checkpoint store and delete outdated checkpoint entries diff --git a/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs b/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs index e22db3ad2b1..1bfceec27ef 100644 --- a/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs +++ b/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs @@ -45,29 +45,6 @@ public bool TryAcquireSettledMetadataForMainStore(CheckpointEntry entry, out Log } } - /// - /// Keep trying to acquire object store metadata until it settles - /// - /// CheckpointEntry to retrieve metadata for - /// LogFileInfo to return - /// Index size in bytes to return - public bool TryAcquireSettledMetadataForObjectStore(CheckpointEntry entry, out LogFileInfo hlog_size, out long index_size) - { - hlog_size = default; - index_size = -1; - try - { - hlog_size = storeWrapper.objectStore.GetLogFileSize(entry.metadata.objectStoreHlogToken); - index_size = storeWrapper.objectStore.GetIndexFileSize(entry.metadata.objectStoreIndexToken); - return true; - } - catch (Exception ex) - { - logger?.LogError(ex, "Waiting for object store metadata to settle"); - return false; - } - } - /// /// Add new checkpoint entry to the in-memory store /// @@ -90,10 +67,8 @@ public string GetLatestCheckpointFromDiskInfo() #endregion public long StoreCurrentSafeAofAddress => clusterProvider.storeWrapper.StoreCheckpointManager.CurrentSafeAofAddress; - public long ObjectStoreCurrentSafeAofAddress => clusterProvider.serverOptions.DisableObjects ? -1 : clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentSafeAofAddress; public long StoreRecoveredSafeAofTailAddress => clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredSafeAofAddress; - public long ObjectStoreRecoveredSafeAofTailAddress => clusterProvider.serverOptions.DisableObjects ? -1 : clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredSafeAofAddress; /// /// Update current aof address for pending commit. @@ -103,8 +78,6 @@ public string GetLatestCheckpointFromDiskInfo() public void UpdateCommitSafeAofAddress(long safeAofTailAddress) { clusterProvider.storeWrapper.StoreCheckpointManager.CurrentSafeAofAddress = safeAofTailAddress; - if (!clusterProvider.serverOptions.DisableObjects) - clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentSafeAofAddress = safeAofTailAddress; } /// @@ -114,8 +87,6 @@ public void UpdateCommitSafeAofAddress(long safeAofTailAddress) public void SetPrimaryReplicationId() { clusterProvider.storeWrapper.StoreCheckpointManager.CurrentHistoryId = PrimaryReplId; - if (!clusterProvider.serverOptions.DisableObjects) - clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentHistoryId = PrimaryReplId; } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index 7eecb80878e..bbbbe8af27b 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -86,27 +86,20 @@ public long ReplicationOffset2 public RecoveryStatus currentRecoveryStatus; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public GarnetClusterCheckpointManager GetCkptManager(StoreType storeType) - { - return storeType switch - { - StoreType.Main => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager, - StoreType.Object => (GarnetClusterCheckpointManager)storeWrapper.objectStore?.CheckpointManager, - _ => throw new Exception($"GetCkptManager: unexpected state {storeType}") - }; - } + public GarnetClusterCheckpointManager GetCkptManager() + => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager; public long GetRecoveredSafeAofAddress() { - var storeAofAddress = clusterProvider.replicationManager.GetCkptManager(StoreType.Main).RecoveredSafeAofAddress; - var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager(StoreType.Object).RecoveredSafeAofAddress; + var storeAofAddress = clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress; + var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress; return Math.Min(storeAofAddress, objectStoreAofAddress); } public long GetCurrentSafeAofAddress() { - var storeAofAddress = clusterProvider.replicationManager.GetCkptManager(StoreType.Main).CurrentSafeAofAddress; - var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager(StoreType.Object).CurrentSafeAofAddress; + var storeAofAddress = clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress; + var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress; return Math.Min(storeAofAddress, objectStoreAofAddress); } @@ -129,13 +122,8 @@ public ReplicationManager(ClusterProvider clusterProvider, ILogger logger = null ReplicationOffset = 0; // Set the appendOnlyFile field for all stores - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).checkpointVersionShiftStart = CheckpointVersionShiftStart; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).checkpointVersionShiftEnd = CheckpointVersionShiftEnd; - if (storeWrapper.objectStore != null) - { - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).checkpointVersionShiftStart = CheckpointVersionShiftStart; - clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).checkpointVersionShiftEnd = CheckpointVersionShiftEnd; - } + clusterProvider.GetReplicationLogCheckpointManager().checkpointVersionShiftStart = CheckpointVersionShiftStart; + clusterProvider.GetReplicationLogCheckpointManager().checkpointVersionShiftEnd = CheckpointVersionShiftEnd; // If this node starts as replica, it cannot serve requests until it is connected to primary if (clusterProvider.clusterManager.CurrentConfig.LocalNodeRole == NodeRole.REPLICA && clusterProvider.serverOptions.Recover && !BeginRecovery(RecoveryStatus.InitializeRecover, upgradeLock: false)) diff --git a/libs/cluster/Server/Replication/SyncMetadata.cs b/libs/cluster/Server/Replication/SyncMetadata.cs index 927f600e123..c85853ecf6d 100644 --- a/libs/cluster/Server/Replication/SyncMetadata.cs +++ b/libs/cluster/Server/Replication/SyncMetadata.cs @@ -26,7 +26,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m "originNodeId:{originNodeId}\n" + "currentPrimaryReplId:{currentPrimaryReplId}\n" + "currentStoreVersion:{currentStoreVersion}\n" + - "currentObjectStoreVersion:{currentObjectStoreVersion}\n" + "currentAofBeginAddress:{currentAofBeginAddress}\n" + "currentAofTailAddress:{currentAofTailAddress}\n" + "currentReplicationOffset:{currentReplicationOffset}\n" + @@ -37,7 +36,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m syncMetadata.originNodeId, syncMetadata.currentPrimaryReplId, syncMetadata.currentStoreVersion, - syncMetadata.currentObjectStoreVersion, syncMetadata.currentAofBeginAddress, syncMetadata.currentAofTailAddress, syncMetadata.currentReplicationOffset, @@ -63,7 +61,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m "originNodeId:{originNodeId}\n" + "currentPrimaryReplId:{currentPrimaryReplId}\n" + "currentStoreVersion:{currentStoreVersion}\n" + - "currentObjectStoreVersion:{currentObjectStoreVersion}\n" + "currentAofBeginAddress:{currentAofBeginAddress}\n" + "currentAofTailAddress:{currentAofTailAddress}\n" + "currentReplicationOffset:{currentReplicationOffset}\n" + @@ -74,7 +71,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m "recoverOriginNodeId:{originNodeId}\n" + "recoverCurrentPrimaryReplId:{currentPrimaryReplId}\n" + "recoverCurrentStoreVersion:{currentStoreVersion}\n" + - "recoverCurrentObjectStoreVersion:{currentObjectStoreVersion}\n" + "recoverCurrentAofBeginAddress:{currentAofBeginAddress}\n" + "recoverCurrentAofTailAddress:{currentAofTailAddress}\n" + "recoverCurrentReplicationOffset:{currentReplicationOffset}\n" + @@ -85,7 +81,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m origin.originNodeId, origin.currentPrimaryReplId, origin.currentStoreVersion, - origin.currentObjectStoreVersion, origin.currentAofBeginAddress, origin.currentAofTailAddress, origin.currentReplicationOffset, @@ -95,7 +90,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m local.originNodeId, local.currentPrimaryReplId, local.currentStoreVersion, - local.currentObjectStoreVersion, local.currentAofBeginAddress, local.currentAofTailAddress, local.currentReplicationOffset, @@ -109,7 +103,6 @@ internal sealed class SyncMetadata( string originNodeId, string currentPrimaryReplId, long currentStoreVersion, - long currentObjectStoreVersion, long currentAofBeginAddress, long currentAofTailAddress, long currentReplicationOffset, @@ -120,7 +113,6 @@ internal sealed class SyncMetadata( public readonly string originNodeId = originNodeId; public readonly string currentPrimaryReplId = currentPrimaryReplId; public readonly long currentStoreVersion = currentStoreVersion; - public readonly long currentObjectStoreVersion = currentObjectStoreVersion; public readonly long currentAofBeginAddress = currentAofBeginAddress; public readonly long currentAofTailAddress = currentAofTailAddress; public readonly long currentReplicationOffset = currentReplicationOffset; @@ -137,7 +129,6 @@ public byte[] ToByteArray() writer.Write(currentPrimaryReplId); writer.Write(currentStoreVersion); - writer.Write(currentObjectStoreVersion); writer.Write(currentAofBeginAddress); writer.Write(currentAofTailAddress); @@ -168,7 +159,6 @@ public static SyncMetadata FromByteArray(byte[] serialized) originNodeId: reader.ReadString(), currentPrimaryReplId: reader.ReadString(), currentStoreVersion: reader.ReadInt64(), - currentObjectStoreVersion: reader.ReadInt64(), currentAofBeginAddress: reader.ReadInt64(), currentAofTailAddress: reader.ReadInt64(), currentReplicationOffset: reader.ReadInt64(), diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs index 642ba634791..83e78f8e051 100644 --- a/libs/cluster/Session/ClusterSession.cs +++ b/libs/cluster/Session/ClusterSession.cs @@ -16,8 +16,8 @@ namespace Garnet.cluster /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession { diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 2f281032b4d..d11898350c4 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -16,8 +16,8 @@ namespace Garnet.cluster /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession { diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index a5226422700..cdbdda8c7f5 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -260,13 +260,10 @@ private void InitializeServer() 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); + (storeWrapper.appendOnlyFile?.MaxMemorySizeBytes ?? 0) + + (storeWrapper.sizeTracker?.TargetSize ?? 0) + + (storeWrapper.sizeTracker?.ReadCacheTargetSize ?? 0); + logger.LogInformation("Total configured memory limit: {configMemoryLimit}", configMemoryLimit); } diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index 8cc3bd02163..b48e0e853b8 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -292,7 +292,7 @@ public unsafe void ProcessAofRecordInternal(byte* ptr, int length, bool asReplic case AofEntryType.ObjectStoreStreamingCheckpointStartCommit: Debug.Assert(storeWrapper.serverOptions.ReplicaDisklessSync); if (asReplica && header.storeVersion > storeWrapper.store.CurrentVersion) - storeWrapper.objectStore.SetVersion(header.storeVersion); + storeWrapper.store.SetVersion(header.storeVersion); break; case AofEntryType.FlushAll: storeWrapper.FlushAllDatabases(unsafeTruncateLog: header.unsafeTruncateLog == 1); @@ -510,42 +510,9 @@ bool BufferNewVersionRecord(AofHeader header, byte* entryPtr, int length) } bool IsOldVersionRecord(AofHeader header) - { - var storeType = ToAofStoreType(header.opType); - - return storeType switch - { - AofStoreType.MainStoreType => header.storeVersion < storeWrapper.store.CurrentVersion, - AofStoreType.ObjectStoreType => header.storeVersion < storeWrapper.objectStore.CurrentVersion, - AofStoreType.TxnType => header.storeVersion < storeWrapper.objectStore.CurrentVersion, - _ => throw new GarnetException($"Unexpected AOF header store type {storeType}"), - }; - } + => header.storeVersion < storeWrapper.store.CurrentVersion; bool IsNewVersionRecord(AofHeader header) - { - var storeType = ToAofStoreType(header.opType); - return storeType switch - { - AofStoreType.MainStoreType => header.storeVersion > storeWrapper.store.CurrentVersion, - AofStoreType.ObjectStoreType => header.storeVersion > storeWrapper.objectStore.CurrentVersion, - AofStoreType.TxnType => header.storeVersion > storeWrapper.objectStore.CurrentVersion, - _ => throw new GarnetException($"Unknown AOF header store type {storeType}"), - }; - } - - static AofStoreType ToAofStoreType(AofEntryType type) - { - return type switch - { - AofEntryType.StoreUpsert or AofEntryType.StoreRMW or AofEntryType.StoreDelete => AofStoreType.MainStoreType, - AofEntryType.ObjectStoreUpsert or AofEntryType.ObjectStoreRMW or AofEntryType.ObjectStoreDelete => AofStoreType.ObjectStoreType, - AofEntryType.TxnStart or AofEntryType.TxnCommit or AofEntryType.TxnAbort or AofEntryType.StoredProcedure => AofStoreType.TxnType, - AofEntryType.CheckpointStartCommit or AofEntryType.ObjectStoreCheckpointStartCommit or AofEntryType.MainStoreStreamingCheckpointStartCommit or AofEntryType.ObjectStoreStreamingCheckpointStartCommit => AofStoreType.CheckpointType, - AofEntryType.CheckpointEndCommit or AofEntryType.ObjectStoreCheckpointEndCommit or AofEntryType.MainStoreStreamingCheckpointEndCommit or AofEntryType.ObjectStoreStreamingCheckpointEndCommit => AofStoreType.CheckpointType, - AofEntryType.FlushAll or AofEntryType.FlushDb => AofStoreType.FlushDbType, - _ => throw new GarnetException($"Conversion to AofStoreType not possible for {type}"), - }; - } + => header.storeVersion > storeWrapper.store.CurrentVersion; } } \ No newline at end of file diff --git a/libs/server/Cluster/CheckpointMetadata.cs b/libs/server/Cluster/CheckpointMetadata.cs index 912e10462c4..798e02106a8 100644 --- a/libs/server/Cluster/CheckpointMetadata.cs +++ b/libs/server/Cluster/CheckpointMetadata.cs @@ -13,12 +13,6 @@ public sealed class CheckpointMetadata public long storeCheckpointCoveredAofAddress; public string storePrimaryReplId; - public long objectStoreVersion; - public Guid objectStoreHlogToken; - public Guid objectStoreIndexToken; - public long objectCheckpointCoveredAofAddress; - public string objectStorePrimaryReplId; - public CheckpointMetadata() { storeVersion = -1; @@ -26,12 +20,6 @@ public CheckpointMetadata() storeIndexToken = default; storeCheckpointCoveredAofAddress = long.MaxValue; storePrimaryReplId = null; - - objectStoreVersion = -1; - objectStoreHlogToken = default; - objectStoreIndexToken = default; - objectCheckpointCoveredAofAddress = long.MaxValue; - objectStorePrimaryReplId = null; } /// @@ -45,12 +33,7 @@ public override string ToString() $"storeHlogToken={storeHlogToken}," + $"storeIndexToken={storeIndexToken}," + $"storeCheckpointCoveredAofAddress={storeCheckpointCoveredAofAddress}," + - $"storePrimaryReplId={storePrimaryReplId ?? "(empty)"}," + - $"objectStoreVersion={objectStoreVersion}," + - $"objectStoreHlogToken={objectStoreHlogToken}," + - $"objectStoreIndexToken={objectStoreIndexToken}," + - $"objectCheckpointCoveredAofAddress={objectCheckpointCoveredAofAddress}," + - $"objectStorePrimaryReplId={objectStorePrimaryReplId ?? "(empty)"}"; + $"storePrimaryReplId={storePrimaryReplId ?? "(empty)"}"; } } } \ No newline at end of file diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs index 8f169b4ae31..ea3472df62b 100644 --- a/libs/server/Cluster/IClusterProvider.cs +++ b/libs/server/Cluster/IClusterProvider.cs @@ -16,8 +16,8 @@ namespace Garnet.server /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Cluster provider diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs index eebe240656b..c8f4755e901 100644 --- a/libs/server/Resp/GarnetDatabaseSession.cs +++ b/libs/server/Resp/GarnetDatabaseSession.cs @@ -7,14 +7,14 @@ namespace Garnet.server /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; using TransactionalGarnetApi = GarnetApi, SpanByteAllocator>>, TransactionalContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Represents a logical database session in Garnet diff --git a/libs/server/Resp/LocalServerSession.cs b/libs/server/Resp/LocalServerSession.cs index 7a0b30c1c4f..2b58da8751b 100644 --- a/libs/server/Resp/LocalServerSession.cs +++ b/libs/server/Resp/LocalServerSession.cs @@ -11,8 +11,8 @@ namespace Garnet.server /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// Local server session diff --git a/libs/server/Resp/Objects/HashCommands.cs b/libs/server/Resp/Objects/HashCommands.cs index 030ed392805..f140a706785 100644 --- a/libs/server/Resp/Objects/HashCommands.cs +++ b/libs/server/Resp/Objects/HashCommands.cs @@ -573,9 +573,6 @@ private unsafe bool HashIncrement(RespCommand command, ref TGarnetAp private unsafe bool HashExpire(RespCommand command, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 4) { return AbortWithWrongNumberOfArguments(command.ToString()); @@ -669,9 +666,6 @@ private unsafe bool HashExpire(RespCommand command, ref TGarnetApi s private unsafe bool HashTimeToLive(RespCommand command, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 3) { return AbortWithWrongNumberOfArguments(command.ToString()); @@ -751,9 +745,6 @@ private unsafe bool HashTimeToLive(RespCommand command, ref TGarnetA private unsafe bool HashPersist(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 3) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.HPERSIST)); diff --git a/libs/server/Resp/Objects/ListCommands.cs b/libs/server/Resp/Objects/ListCommands.cs index 1ca2ffc134b..63bae823c14 100644 --- a/libs/server/Resp/Objects/ListCommands.cs +++ b/libs/server/Resp/Objects/ListCommands.cs @@ -279,9 +279,6 @@ private bool ListBlockingPop(RespCommand command) if (!parseState.TryGetTimeout(parseState.Count - 1, out var timeout, out var error)) return AbortWithErrorMessage(error); - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - var result = storeWrapper.itemBroker.GetCollectionItemAsync(command, keysBytes, this, timeout).Result; if (result.IsForceUnblocked) @@ -370,9 +367,6 @@ private bool ListBlockingMove(PinnedSpanByte srcKey, PinnedSpanByte dstKey, cmdArgs[1] = PinnedSpanByte.FromPinnedPointer(pSrcDir, 1); cmdArgs[2] = PinnedSpanByte.FromPinnedPointer(pDstDir, 1); - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - var result = storeWrapper.itemBroker.MoveCollectionItemAsync(RespCommand.BLMOVE, srcKey.ToArray(), this, timeout, cmdArgs).Result; @@ -907,9 +901,6 @@ private unsafe bool ListBlockingPopMultiple() cmdArgs[1] = PinnedSpanByte.FromPinnedPointer((byte*)&popCount, sizeof(int)); - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - var result = storeWrapper.itemBroker.GetCollectionItemAsync(RespCommand.BLMPOP, keysBytes, this, timeout, cmdArgs).Result; if (result.IsForceUnblocked) diff --git a/libs/server/Resp/Objects/SortedSetCommands.cs b/libs/server/Resp/Objects/SortedSetCommands.cs index 520604d86cd..f6761ab0b8b 100644 --- a/libs/server/Resp/Objects/SortedSetCommands.cs +++ b/libs/server/Resp/Objects/SortedSetCommands.cs @@ -1539,9 +1539,6 @@ private unsafe bool SortedSetUnionStore(ref TGarnetApi storageApi) /// private unsafe bool SortedSetBlockingPop(RespCommand command) { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count < 2) { return AbortWithWrongNumberOfArguments(command.ToString()); @@ -1599,9 +1596,6 @@ private unsafe bool SortedSetBlockingPop(RespCommand command) /// private unsafe bool SortedSetBlockingMPop() { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count < 4) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.BZMPOP)); @@ -1725,9 +1719,6 @@ private unsafe bool SortedSetBlockingMPop() private unsafe bool SortedSetExpire(RespCommand command, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 4) { return AbortWithWrongNumberOfArguments(command.ToString()); @@ -1824,9 +1815,6 @@ private unsafe bool SortedSetExpire(RespCommand command, ref TGarnet private unsafe bool SortedSetTimeToLive(RespCommand command, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 3) { return AbortWithWrongNumberOfArguments(command.ToString()); @@ -1913,9 +1901,6 @@ private unsafe bool SortedSetTimeToLive(RespCommand command, ref TGa private unsafe bool SortedSetPersist(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - if (storeWrapper.objectStore == null) - throw new GarnetException("Object store is disabled"); - if (parseState.Count <= 3) { return AbortWithWrongNumberOfArguments(nameof(RespCommand.ZPERSIST)); diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 2ba11ef9711..8bbd43ee9f3 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -24,14 +24,14 @@ namespace Garnet.server /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; using TransactionalGarnetApi = GarnetApi, SpanByteAllocator>>, TransactionalContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; /// /// RESP server session diff --git a/libs/server/ServerConfig.cs b/libs/server/ServerConfig.cs index 707077d4d9f..ebc1dd7c143 100644 --- a/libs/server/ServerConfig.cs +++ b/libs/server/ServerConfig.cs @@ -131,10 +131,8 @@ private bool NetworkCONFIG_SET() string clusterUsername = null; string clusterPassword = null; string memorySize = null; - string objLogMemory = null; - string objHeapMemory = null; + string heapMemory = null; string index = null; - string objIndex = null; var unknownOption = false; var unknownKey = ""; @@ -146,14 +144,10 @@ private bool NetworkCONFIG_SET() if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.Memory, allowNonAlphabeticChars: false)) memorySize = Encoding.ASCII.GetString(value); - else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.ObjLogMemory, allowNonAlphabeticChars: true)) - objLogMemory = Encoding.ASCII.GetString(value); else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.ObjHeapMemory, allowNonAlphabeticChars: true)) - objHeapMemory = Encoding.ASCII.GetString(value); + heapMemory = Encoding.ASCII.GetString(value); else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.Index, allowNonAlphabeticChars: false)) index = Encoding.ASCII.GetString(value); - else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.ObjIndex, allowNonAlphabeticChars: true)) - objIndex = Encoding.ASCII.GetString(value); else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.CertFileName, allowNonAlphabeticChars: true)) certFileName = Encoding.ASCII.GetString(value); else if (key.EqualsLowerCaseSpanIgnoringCase(CmdStrings.CertPassword, allowNonAlphabeticChars: true)) @@ -210,17 +204,11 @@ private bool NetworkCONFIG_SET() if (memorySize != null) HandleMemorySizeChange(memorySize, sbErrorMsg); - if (objLogMemory != null) - HandleMemorySizeChange(objLogMemory, sbErrorMsg, mainStore: false); - if (index != null) HandleIndexSizeChange(index, sbErrorMsg); - if (objIndex != null) - HandleIndexSizeChange(objIndex, sbErrorMsg, mainStore: false); - - if (objHeapMemory != null) - HandleObjHeapMemorySizeChange(objHeapMemory, sbErrorMsg); + if (heapMemory != null) + HandleHeapMemorySizeChange(heapMemory, sbErrorMsg); } if (sbErrorMsg.Length == 0) @@ -237,20 +225,16 @@ private bool NetworkCONFIG_SET() return true; } - private void HandleMemorySizeChange(string memorySize, StringBuilder sbErrorMsg, bool mainStore = true) + private void HandleMemorySizeChange(string memorySize, StringBuilder sbErrorMsg) { - var option = mainStore ? CmdStrings.Memory : CmdStrings.ObjLogMemory; - if (!ServerOptions.TryParseSize(memorySize, out var newMemorySize)) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIncorrectSizeFormat, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIncorrectSizeFormat, CmdStrings.Memory); return; } // Parse the configured memory size - var confMemorySize = ServerOptions.ParseSize( - mainStore ? storeWrapper.serverOptions.MemorySize - : storeWrapper.serverOptions.ObjectStoreLogMemorySize, out _); + var confMemorySize = ServerOptions.ParseSize(storeWrapper.serverOptions.MemorySize, out _); // If the new memory size is the same as the configured memory size, nothing to do if (newMemorySize == confMemorySize) @@ -262,35 +246,24 @@ private void HandleMemorySizeChange(string memorySize, StringBuilder sbErrorMsg, // If the new memory size is greater than the configured memory size, return an error if (newMemorySize > confMemorySize) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrMemorySizeGreaterThanBuffer, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrMemorySizeGreaterThanBuffer, CmdStrings.Memory); return; } // Parse & adjust the configured page size - var pageSize = ServerOptions.ParseSize( - mainStore ? storeWrapper.serverOptions.PageSize : storeWrapper.serverOptions.ObjectStorePageSize, - out _); + var pageSize = ServerOptions.ParseSize(storeWrapper.serverOptions.PageSize, out _); pageSize = ServerOptions.PreviousPowerOf2(pageSize); // Compute the new minimum empty page count and update the store's log accessor var newMinEmptyPageCount = (int)((confMemorySize - newMemorySize) / pageSize); - if (mainStore) - { - storeWrapper.store.Log.MinEmptyPageCount = newMinEmptyPageCount; - } - else - { - storeWrapper.objectStore.Log.MinEmptyPageCount = newMinEmptyPageCount; - } + storeWrapper.store.Log.MinEmptyPageCount = newMinEmptyPageCount; } - private void HandleIndexSizeChange(string indexSize, StringBuilder sbErrorMsg, bool mainStore = true) + private void HandleIndexSizeChange(string indexSize, StringBuilder sbErrorMsg) { - var option = mainStore ? CmdStrings.Index : CmdStrings.ObjIndex; - if (!ServerOptions.TryParseSize(indexSize, out var newIndexSize)) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIncorrectSizeFormat, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIncorrectSizeFormat, CmdStrings.Index); return; } @@ -298,19 +271,18 @@ private void HandleIndexSizeChange(string indexSize, StringBuilder sbErrorMsg, b var adjNewIndexSize = ServerOptions.PreviousPowerOf2(newIndexSize); if (adjNewIndexSize != newIndexSize) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizePowerOfTwo, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizePowerOfTwo, CmdStrings.Index); return; } // Check if the index auto-grow task is running. If so - return an error. - if ((mainStore && storeWrapper.serverOptions.AdjustedIndexMaxCacheLines > 0) || - (!mainStore && storeWrapper.serverOptions.AdjustedObjectStoreIndexMaxCacheLines > 0)) + if (storeWrapper.serverOptions.AdjustedIndexMaxCacheLines > 0) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeAutoGrow, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeAutoGrow, CmdStrings.Index); return; } - var currIndexSize = mainStore ? storeWrapper.store.IndexSize : storeWrapper.objectStore.IndexSize; + var currIndexSize = storeWrapper.store.IndexSize; // Convert new index size to cache lines adjNewIndexSize /= 64; @@ -322,20 +294,18 @@ private void HandleIndexSizeChange(string indexSize, StringBuilder sbErrorMsg, b // If the new index size is smaller than the current index size, return an error if (currIndexSize > adjNewIndexSize) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeSmallerThanCurrent, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeSmallerThanCurrent, CmdStrings.Index); return; } // Try to grow the index size by doubling it until it reaches the new size while (currIndexSize < adjNewIndexSize) { - var isSuccessful = mainStore - ? storeWrapper.store.GrowIndexAsync().ConfigureAwait(false).GetAwaiter().GetResult() - : storeWrapper.objectStore.GrowIndexAsync().ConfigureAwait(false).GetAwaiter().GetResult(); + var isSuccessful = storeWrapper.store.GrowIndexAsync().ConfigureAwait(false).GetAwaiter().GetResult(); if (!isSuccessful) { - AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeGrowFailed, option); + AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrIndexSizeGrowFailed, CmdStrings.Index); return; } @@ -343,7 +313,7 @@ private void HandleIndexSizeChange(string indexSize, StringBuilder sbErrorMsg, b } } - private void HandleObjHeapMemorySizeChange(string heapMemorySize, StringBuilder sbErrorMsg) + private void HandleHeapMemorySizeChange(string heapMemorySize, StringBuilder sbErrorMsg) { if (!ServerOptions.TryParseSize(heapMemorySize, out var newHeapMemorySize)) { @@ -352,14 +322,14 @@ private void HandleObjHeapMemorySizeChange(string heapMemorySize, StringBuilder } // If the object store size tracker is not running, return an error - if (storeWrapper.objectStoreSizeTracker == null) + if (storeWrapper.sizeTracker == null) { AppendErrorWithTemplate(sbErrorMsg, CmdStrings.GenericErrHeapMemorySizeTrackerNotRunning, CmdStrings.ObjHeapMemory); return; } // Set the new target size for the object store size tracker - storeWrapper.objectStoreSizeTracker.TargetSize = newHeapMemorySize; + storeWrapper.sizeTracker.TargetSize = newHeapMemorySize; } private static void AppendError(StringBuilder sbErrorMsg, string error) diff --git a/libs/server/Storage/Functions/FunctionsState.cs b/libs/server/Storage/Functions/FunctionsState.cs index ec9d82d672a..1cb2877151e 100644 --- a/libs/server/Storage/Functions/FunctionsState.cs +++ b/libs/server/Storage/Functions/FunctionsState.cs @@ -21,7 +21,7 @@ internal sealed class FunctionsState public readonly MemoryPool memoryPool; public readonly CacheSizeTracker objectStoreSizeTracker; public readonly GarnetObjectSerializer garnetObjectSerializer; - public IStoreFunctions objectStoreFunctions; + public IStoreFunctions storeFunctions; public ETagState etagState; public readonly ILogger logger; public byte respProtocolVersion; @@ -39,7 +39,7 @@ public FunctionsState(TsavoriteLog appendOnlyFile, WatchVersionMap watchVersionM this.memoryPool = memoryPool ?? MemoryPool.Shared; this.objectStoreSizeTracker = objectStoreSizeTracker; this.garnetObjectSerializer = storeWrapper.GarnetObjectSerializer; - objectStoreFunctions = storeWrapper.objectStoreFunctions; + storeFunctions = storeWrapper.storeFunctions; this.etagState = new ETagState(); this.logger = logger; diff --git a/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs b/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs index 3a18d7c0086..05bd739968c 100644 --- a/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs @@ -33,7 +33,7 @@ public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. - functionsState.objectStoreFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); logRecord.ClearValueObject(obj => { }); return true; } diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index 6a1b8036505..01a231606d3 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -121,7 +121,7 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. - functionsState.objectStoreFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); logRecord.ClearValueObject(obj => { }); rmwInfo.Action = input.header.type == GarnetObjectType.DelIfExpIm ? RMWAction.ExpireAndStop : RMWAction.ExpireAndResume; return false; @@ -156,7 +156,7 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. - functionsState.objectStoreFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); logRecord.ClearValueObject(obj => { }); rmwInfo.Action = RMWAction.ExpireAndStop; diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 5f60d8b0f42..9912c84c5d1 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -53,12 +53,11 @@ public sealed class StoreWrapper public DateTimeOffset lastSaveTime => databaseManager.LastSaveTime; /// - /// Object store size tracker (of DB 0) + /// Store size tracker (of DB 0) /// - public CacheSizeTracker objectStoreSizeTracker => databaseManager.SizeTracker; + public CacheSizeTracker sizeTracker => databaseManager.SizeTracker; - public IStoreFunctions mainStoreFunctions => store.StoreFunctions; - public IStoreFunctions objectStoreFunctions => objectStore?.StoreFunctions; + public IStoreFunctions storeFunctions => store.StoreFunctions; /// /// Server options diff --git a/test/Garnet.test/CacheSizeTrackerTests.cs b/test/Garnet.test/CacheSizeTrackerTests.cs index df75236c0b2..b19582347f1 100644 --- a/test/Garnet.test/CacheSizeTrackerTests.cs +++ b/test/Garnet.test/CacheSizeTrackerTests.cs @@ -28,7 +28,7 @@ public void Setup() server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "2k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreHeapMemorySize: "3k"); server.Start(); store = server.Provider.StoreWrapper.store; - cacheSizeTracker = server.Provider.StoreWrapper.objectStoreSizeTracker; + cacheSizeTracker = server.Provider.StoreWrapper.sizeTracker; } [TearDown] @@ -95,7 +95,7 @@ public void ReadCacheIncreaseEmptyPageCountTest() server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "1k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreReadCacheHeapMemorySize: "1k", enableObjectStoreReadCache: true); server.Start(); store = server.Provider.StoreWrapper.store; - cacheSizeTracker = server.Provider.StoreWrapper.objectStoreSizeTracker; + cacheSizeTracker = server.Provider.StoreWrapper.sizeTracker; var readCacheEmptyPageCountIncrements = 0; var readCacheEpcEvent = new ManualResetEventSlim(false); diff --git a/test/Garnet.test/ExpiredKeyDeletionTests.cs b/test/Garnet.test/ExpiredKeyDeletionTests.cs index b777c24ca61..ab06a924c12 100644 --- a/test/Garnet.test/ExpiredKeyDeletionTests.cs +++ b/test/Garnet.test/ExpiredKeyDeletionTests.cs @@ -101,11 +101,9 @@ private async Task TestExpiredKeyDeletionScanAsync(Func ex // Merge reviv stats across sessions server.Provider.StoreWrapper.store.DumpRevivificationStats(); - server.Provider.StoreWrapper.objectStore.DumpRevivificationStats(); // Check that revivification happened for expired record ClassicAssert.IsTrue(server.Provider.StoreWrapper.store.RevivificationManager.stats.successfulAdds > 0, "Active expiration did not revivify for main store as expected"); - ClassicAssert.IsTrue(server.Provider.StoreWrapper.objectStore.RevivificationManager.stats.successfulAdds > 0, "Active expiration did not revivify for obj store as expected"); // Post expired key deletion scan, expired records don't exist for sure. This can be fooled by passive expiration too, so check reviv metrics too CheckExistenceConditionOnAllKeys(db, tombstonedRecords, false, "All to be expired should no longer exist post gc"); diff --git a/test/Garnet.test/IndexGrowthTests.cs b/test/Garnet.test/IndexGrowthTests.cs index 7b9b9a9fa96..da76679c5f2 100644 --- a/test/Garnet.test/IndexGrowthTests.cs +++ b/test/Garnet.test/IndexGrowthTests.cs @@ -73,7 +73,7 @@ public void ObjectStoreIndexGrowthTest() server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, objectStoreIndexSize: "64", objectStoreIndexMaxSize: "128", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); server.Start(); - var objectStore = server.Provider.StoreWrapper.objectStore; + var store = server.Provider.StoreWrapper.store; RedisKey[] keys = ["abcdkey", "bcdekey", "cdefkey", "defgkey", "efghkey", "fghikey", "ghijkey", "hijkkey"]; RedisValue[] values = ["abcdval", "bcdeval", "cdefval", "defgval", "efghval", "fghival", "ghijval", "hijkval"]; @@ -81,8 +81,8 @@ public void ObjectStoreIndexGrowthTest() using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) { var db = redis.GetDatabase(0); - ClassicAssert.AreEqual(0, objectStore.OverflowBucketAllocations); - ClassicAssert.AreEqual(1, objectStore.IndexSize); + ClassicAssert.AreEqual(0, store.OverflowBucketAllocations); + ClassicAssert.AreEqual(1, store.IndexSize); for (int i = 0; i < keys.Length; i++) { @@ -90,16 +90,16 @@ public void ObjectStoreIndexGrowthTest() } VerifyObjectStoreSetMembers(db, keys, values); - ClassicAssert.AreEqual(1, objectStore.OverflowBucketAllocations); + ClassicAssert.AreEqual(1, store.OverflowBucketAllocations); // Wait for the resizing to happen for (int waitCycles = 0; waitCycles < indexResizeWaitCycles; waitCycles++) { Thread.Sleep(TimeSpan.FromSeconds(indexResizeTaskDelaySeconds)); - if (objectStore.IndexSize > 1) break; + if (store.IndexSize > 1) break; } - ClassicAssert.AreEqual(2, objectStore.IndexSize); + ClassicAssert.AreEqual(2, store.IndexSize); VerifyObjectStoreSetMembers(db, keys, values); } } @@ -184,7 +184,7 @@ public void ObjectStoreIndexGrowthTestWithDiskReadAndCheckpoint() server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, objectStoreIndexSize: "512", objectStoreIndexMaxSize: "1k", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); server.Start(); - var objectStore = server.Provider.StoreWrapper.objectStore; + var store = server.Provider.StoreWrapper.store; RedisKey[] keys = ["abcdkey", "bcdekey", "cdefkey", "defgkey", "efghkey", "fghikey", "ghijkey", "hijkkey"]; RedisValue[] values = ["abcdval", "bcdeval", "cdefval", "defgval", "efghval", "fghival", "ghijval", "hijkval"]; @@ -193,7 +193,7 @@ public void ObjectStoreIndexGrowthTestWithDiskReadAndCheckpoint() using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) { var db = redis.GetDatabase(0); - ClassicAssert.AreEqual(8, objectStore.IndexSize); + ClassicAssert.AreEqual(8, store.IndexSize); for (int i = 0; i < keys.Length; i++) { @@ -216,10 +216,10 @@ public void ObjectStoreIndexGrowthTestWithDiskReadAndCheckpoint() for (int waitCycles = 0; waitCycles < indexResizeWaitCycles; waitCycles++) { Thread.Sleep(TimeSpan.FromSeconds(indexResizeTaskDelaySeconds)); - if (objectStore.IndexSize > 8) break; + if (store.IndexSize > 8) break; } - ClassicAssert.AreEqual(16, objectStore.IndexSize); + ClassicAssert.AreEqual(16, store.IndexSize); // Check if entry created before resizing is still accessible. VerifyObjectStoreSetMembers(db, keys, values); diff --git a/test/Garnet.test/RespAdminCommandsTests.cs b/test/Garnet.test/RespAdminCommandsTests.cs index 62fbaa70690..87cc59e6e6a 100644 --- a/test/Garnet.test/RespAdminCommandsTests.cs +++ b/test/Garnet.test/RespAdminCommandsTests.cs @@ -361,7 +361,7 @@ public void SeSaveRecoverMultipleObjectsTest(int memorySize, int recoveryMemoryS server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, memorySize: sizeToString(recoveryMemorySize), pageSize: sizeToString(pageSize), objectStoreHeapMemorySize: "64k"); server.Start(); - ClassicAssert.LessOrEqual(server.Provider.StoreWrapper.objectStore.MaxAllocatedPageCount, (recoveryMemorySize / pageSize) + 1); + ClassicAssert.LessOrEqual(server.Provider.StoreWrapper.store.MaxAllocatedPageCount, (recoveryMemorySize / pageSize) + 1); using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) { var db = redis.GetDatabase(0); diff --git a/test/Garnet.test/RespConfigTests.cs b/test/Garnet.test/RespConfigTests.cs index 34547530700..54770eb4fd5 100644 --- a/test/Garnet.test/RespConfigTests.cs +++ b/test/Garnet.test/RespConfigTests.cs @@ -61,27 +61,26 @@ public void TearDown() /// This test verifies that dynamically changing the memory size configuration using CONFIG SET memory / obj-log-memory /// incurs the expected changes in Garnet server metrics, as well as verifies error handling for incorrect inputs. /// - /// Store type (Main / Object) /// Memory size smaller than the initial size /// Memory size larger than the initial size (within buffer bounds) /// Memory size larger than the buffer size /// Malformed memory size string [Test] - [TestCase(StoreType.Main, "16g", "32g", "64g", "g4")] - [TestCase(StoreType.Main, "9gB", "28GB", "33G", "2gBB")] - [TestCase(StoreType.Object, "16m", "32m", "64m", "3bm")] + [TestCase("16g", "32g", "64g", "g4")] + [TestCase("9gB", "28GB", "33G", "2gBB")] + [TestCase("16m", "32m", "64m", "3bm")] [TestCase(StoreType.Object, "5MB", "30M", "128mb", "44d")] - public void ConfigSetMemorySizeTest(StoreType storeType, string smallerSize, string largerSize, string largerThanBufferSize, string malformedSize) + public void ConfigSetMemorySizeTest(string smallerSize, string largerSize, string largerThanBufferSize, string malformedSize) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); - var option = storeType == StoreType.Main ? "memory" : "obj-log-memory"; - var metricType = storeType == StoreType.Main ? InfoMetricsType.STORE : InfoMetricsType.OBJECTSTORE; - var initMemorySize = storeType == StoreType.Main ? memorySize : objectStoreLogMemorySize; + var option = "memory"; + var metricType = InfoMetricsType.STORE; + var initMemorySize = memorySize; var currMemorySize = ServerOptions.ParseSize(initMemorySize, out _); var bufferSize = ServerOptions.NextPowerOf2(currMemorySize); - var pageSize = storeType == StoreType.Main ? 32L * 1024 * 1024 : 4 * 1024; // default page size + var pageSize = 32L * 1024 * 1024; // default page size // Check initial MinEPC before any changes var metrics = server.Metrics.GetInfoMetrics(metricType); @@ -148,23 +147,22 @@ public void ConfigSetMemorySizeTest(StoreType storeType, string smallerSize, str /// This test verifies that dynamically changing the index size configuration using CONFIG SET index / obj-index /// incurs the expected changes in Garnet server metrics, as well as verifies error handling for incorrect inputs. /// - /// Store type (Main / Object) /// Index size smaller than the initial size /// Index size larger than the initial size /// Illegal index size (not a power of 2) /// Malformed index size string [Test] - [TestCase(StoreType.Main, "32m", "128m", "63m", "8d")] - [TestCase(StoreType.Main, "16mB", "256MB", "23m", "g8")] - [TestCase(StoreType.Object, "2m", "32m", "28m", "m9")] - [TestCase(StoreType.Object, "4Mb", "16mB", "129MB", "0.3gb")] - public void ConfigSetIndexSizeTest(StoreType storeType, string smallerSize, string largerSize, string illegalSize, string malformedSize) + [TestCase("32m", "128m", "63m", "8d")] + [TestCase("16mB", "256MB", "23m", "g8")] + [TestCase("2m", "32m", "28m", "m9")] + [TestCase("4Mb", "16mB", "129MB", "0.3gb")] + public void ConfigSetIndexSizeTest(string smallerSize, string largerSize, string illegalSize, string malformedSize) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); - var metricType = storeType == StoreType.Main ? InfoMetricsType.STORE : InfoMetricsType.OBJECTSTORE; - var option = storeType == StoreType.Main ? "index" : "obj-index"; - var initIndexSize = storeType == StoreType.Main ? indexSize : objectStoreIndexSize; + var metricType = InfoMetricsType.STORE; + var option = "index"; + var initIndexSize = indexSize; // Check initial index size before any changes var currIndexSize = ServerOptions.ParseSize(initIndexSize, out _); @@ -605,22 +603,18 @@ public void TearDown() /// This test verifies that dynamically changing the index size configuration using CONFIG SET /// incurs the expected shifts in the overflow buckets of the store, and that no data is lost in the process. /// - /// Store type (Main / Object) /// Larger index size than configured /// Larger index size than previous [Test] - [TestCase(StoreType.Main, "1024", "4096")] - [TestCase(StoreType.Object, "1024", "4096")] - public void ConfigSetIndexSizeUtilizationTest(StoreType storeType, string largerSize1, string largerSize2) + [TestCase("1024", "4096")] + public void ConfigSetIndexSizeUtilizationTest(string largerSize1, string largerSize2) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true)); var db = redis.GetDatabase(0); - var option = storeType == StoreType.Main ? "index" : "obj-index"; + var option = "index"; var parsedIndexSize = ServerOptions.ParseSize(indexSize, out _); - var currIndexSize = storeType == StoreType.Main - ? server.Provider.StoreWrapper.store.IndexSize - : server.Provider.StoreWrapper.objectStore.IndexSize; + var currIndexSize = server.Provider.StoreWrapper.store.IndexSize; // Verify initial index size and overflow bucket allocations are zero ClassicAssert.AreEqual(parsedIndexSize / 64, currIndexSize); @@ -635,10 +629,7 @@ public void ConfigSetIndexSizeUtilizationTest(StoreType storeType, string larger // Insert first batch of data for (var i = 0; i < 250; i++) { - if (storeType == StoreType.Main) - _ = db.StringSet(keys[i], val); - else - _ = db.ListRightPush(keys[i], [val]); + _ = db.StringSet(keys[i], val); } // Verify that overflow bucket allocations are non-zero after initial insertions @@ -657,10 +648,7 @@ public void ConfigSetIndexSizeUtilizationTest(StoreType storeType, string larger // Insert second batch of data for (var i = 250; i < 500; i++) { - if (storeType == StoreType.Main) - _ = db.StringSet(keys[i], val); - else - _ = db.ListRightPush(keys[i], [val]); + _ = db.StringSet(keys[i], val); } prevOverflowBucketAllocations = GetOverflowBucketAllocations(); @@ -680,9 +668,7 @@ public void ConfigSetIndexSizeUtilizationTest(StoreType storeType, string larger } long GetOverflowBucketAllocations() => - storeType == StoreType.Main - ? server.Provider.StoreWrapper.store.OverflowBucketAllocations - : server.Provider.StoreWrapper.objectStore.OverflowBucketAllocations; + server.Provider.StoreWrapper.store.OverflowBucketAllocations; } } @@ -743,8 +729,8 @@ public void ConfigSetHeapSizeUtilizationTest(string largerSize) var option = "obj-heap-memory"; // Verify that initial empty page count is zero - var objectStore = server.Provider.StoreWrapper.objectStore; - ClassicAssert.AreEqual(0, objectStore.Log.EmptyPageCount); + var store = server.Provider.StoreWrapper.store; + ClassicAssert.AreEqual(0, store.Log.EmptyPageCount); // Add objects to store to fill up heap var values = new RedisValue[16]; @@ -762,8 +748,8 @@ public void ConfigSetHeapSizeUtilizationTest(string largerSize) Thread.Sleep(sizeTrackerDelay); // Verify that empty page count has increased - ClassicAssert.Greater(objectStore.Log.EmptyPageCount, 0); - var prevEpc = objectStore.Log.EmptyPageCount; + ClassicAssert.Greater(store.Log.EmptyPageCount, 0); + var prevEpc = store.Log.EmptyPageCount; // Try to set heap size to a larger value than current var result = db.Execute("CONFIG", "SET", option, largerSize); @@ -773,7 +759,7 @@ public void ConfigSetHeapSizeUtilizationTest(string largerSize) Thread.Sleep(sizeTrackerDelay); // Verify that empty page count has decreased - ClassicAssert.Less(objectStore.Log.EmptyPageCount, prevEpc); + ClassicAssert.Less(store.Log.EmptyPageCount, prevEpc); } } } \ No newline at end of file diff --git a/test/Garnet.test/RespInfoTests.cs b/test/Garnet.test/RespInfoTests.cs index 383f9cff5b1..694546436b9 100644 --- a/test/Garnet.test/RespInfoTests.cs +++ b/test/Garnet.test/RespInfoTests.cs @@ -82,11 +82,8 @@ public async Task InfoHlogScanTest() // hydrate var startingHA = server.Provider.StoreWrapper.store.Log.HeadAddress; - var startingHAObj = server.Provider.StoreWrapper.objectStore.Log.HeadAddress; - await Task.WhenAll( - HydrateStore(db, (db, key, value) => db.StringSetAsync(key, value), () => startingHA == server.Provider.StoreWrapper.store.Log.HeadAddress), - HydrateStore(db, (db, key, value) => db.SetAddAsync(key, value), () => startingHAObj == server.Provider.StoreWrapper.objectStore.Log.HeadAddress) - ); + await HydrateStore(db, (db, key, value) => db.StringSetAsync(key, value), + () => startingHA == server.Provider.StoreWrapper.store.Log.HeadAddress); // Wait for the immediate expirations to kick in await Task.Delay(500); diff --git a/test/Garnet.test/RespSortedSetTests.cs b/test/Garnet.test/RespSortedSetTests.cs index e57f138d109..598995aa62d 100644 --- a/test/Garnet.test/RespSortedSetTests.cs +++ b/test/Garnet.test/RespSortedSetTests.cs @@ -23,8 +23,8 @@ namespace Garnet.test /* MainStoreFunctions */ StoreFunctions, SpanByteAllocator>>, BasicContext, - ObjectAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + SpanByteAllocator>>>; [TestFixture] public class RespSortedSetTests From 62e66240207830459a3006a3b69d4f87b4647d98 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Tue, 16 Sep 2025 15:14:56 -0700 Subject: [PATCH 03/28] wip --- .../Server/Migration/MigrateSessionKeys.cs | 7 +- .../ReplicationSyncManager.cs | 2 - .../ReplicaOps/ReplicaDisklessSync.cs | 1 - .../Server/Replication/ReplicationManager.cs | 12 +- libs/cluster/Session/ClusterCommands.cs | 23 +--- .../RespClusterSlotManagementCommands.cs | 6 +- libs/host/Configuration/Options.cs | 84 ++----------- libs/host/defaults.conf | 49 +------- libs/server/Databases/DatabaseManagerBase.cs | 5 +- libs/server/Servers/GarnetServerOptions.cs | 86 +------------ libs/server/StoreWrapper.cs | 11 +- test/Garnet.fuzz/Targets/GarnetEndToEnd.cs | 3 - .../ClusterManagementTests.cs | 1 - .../ClusterMigrateTests.cs | 6 +- .../ClusterNegativeTests.cs | 12 +- .../Garnet.test.cluster/ClusterTestContext.cs | 6 - .../ClusterReplicationBaseTests.cs | 54 ++++---- .../ClusterReplicationDisklessSyncTests.cs | 10 +- test/Garnet.test/CacheSizeTrackerTests.cs | 6 +- test/Garnet.test/GarnetClientTests.cs | 2 +- test/Garnet.test/GarnetServerConfigTests.cs | 8 +- test/Garnet.test/IndexGrowthTests.cs | 6 +- test/Garnet.test/ReadCacheTests.cs | 14 +-- test/Garnet.test/RespAdminCommandsTests.cs | 12 +- test/Garnet.test/RespConfigTests.cs | 117 ++++++------------ test/Garnet.test/RespEtagTests.cs | 10 +- test/Garnet.test/RespHashTests.cs | 4 +- test/Garnet.test/RespModuleTests.cs | 2 - test/Garnet.test/RespSlowLogTests.cs | 2 +- test/Garnet.test/RespSortedSetTests.cs | 4 +- test/Garnet.test/RespTests.cs | 16 +-- test/Garnet.test/TestUtils.cs | 54 ++------ 32 files changed, 154 insertions(+), 481 deletions(-) diff --git a/libs/cluster/Server/Migration/MigrateSessionKeys.cs b/libs/cluster/Server/Migration/MigrateSessionKeys.cs index 687f5dd6367..629f95c8f95 100644 --- a/libs/cluster/Server/Migration/MigrateSessionKeys.cs +++ b/libs/cluster/Server/Migration/MigrateSessionKeys.cs @@ -104,11 +104,8 @@ public bool MigrateKeys() return false; // Migrate object store keys - if (!clusterProvider.serverOptions.DisableObjects) - { - if (!MigrateKeysFromObjectStore()) - return false; - } + if (!MigrateKeysFromObjectStore()) + return false; } catch (Exception ex) { diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 86ebf79378d..ddd4f691ee7 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -155,8 +155,6 @@ public async Task ReplicationSyncDriver(ReplicaSyncSession repli async Task MainStreamingSnapshotDriver() { // Parameters for sync operation - var disableObjects = ClusterProvider.serverOptions.DisableObjects; - try { // Lock to avoid the addition of new replica sync sessions while sync is in progress diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 1aa8c3e63e8..9cbd61a1336 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -56,7 +56,6 @@ public bool TryReplicateDisklessSync( async Task TryBeginReplicaSync(bool downgradeLock) { var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync; - var disableObjects = clusterProvider.serverOptions.DisableObjects; GarnetClientSession gcs = null; try { diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index bbbbe8af27b..978acce60b7 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -90,18 +90,10 @@ public GarnetClusterCheckpointManager GetCkptManager() => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager; public long GetRecoveredSafeAofAddress() - { - var storeAofAddress = clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress; - var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress; - return Math.Min(storeAofAddress, objectStoreAofAddress); - } + => clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress; public long GetCurrentSafeAofAddress() - { - var storeAofAddress = clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress; - var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress; - return Math.Min(storeAofAddress, objectStoreAofAddress); - } + => clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress; public ReplicationManager(ClusterProvider clusterProvider, ILogger logger = null) { diff --git a/libs/cluster/Session/ClusterCommands.cs b/libs/cluster/Session/ClusterCommands.cs index 104e05144b7..c9dd8c4b13c 100644 --- a/libs/cluster/Session/ClusterCommands.cs +++ b/libs/cluster/Session/ClusterCommands.cs @@ -15,7 +15,7 @@ internal sealed unsafe partial class ClusterSession : IClusterSession { ClusterConfig lastSentConfig; - private int CountKeysInSessionStore(int slot) + private int CountKeysInSlot(int slot) { ClusterKeyIterationFunctions.MainStoreCountKeys iterFuncs = new(slot); var cursor = 0L; @@ -23,33 +23,12 @@ private int CountKeysInSessionStore(int slot) return iterFuncs.KeyCount; } - private int CountKeysInObjectStore(int slot) - { - if (!clusterProvider.serverOptions.DisableObjects) - { - ClusterKeyIterationFunctions.ObjectStoreCountKeys iterFuncs = new(slot); - var cursor = 0L; - _ = basicGarnetApi.IterateObjectStore(ref iterFuncs, ref cursor); - return iterFuncs.KeyCount; - } - return 0; - } - - private int CountKeysInSlot(int slot) => CountKeysInSessionStore(slot) + CountKeysInObjectStore(slot); - private List GetKeysInSlot(int slot, int keyCount) { List keys = []; ClusterKeyIterationFunctions.MainStoreGetKeysInSlot mainIterFuncs = new(keys, slot, keyCount); var cursor = 0L; _ = basicGarnetApi.IterateMainStore(ref mainIterFuncs, ref cursor); - - if (!clusterProvider.serverOptions.DisableObjects) - { - ClusterKeyIterationFunctions.ObjectStoreGetKeysInSlot objectIterFuncs = new(keys, slot); - var objectCursor = 0L; - _ = basicGarnetApi.IterateObjectStore(ref objectIterFuncs, ref objectCursor); - } return keys; } diff --git a/libs/cluster/Session/RespClusterSlotManagementCommands.cs b/libs/cluster/Session/RespClusterSlotManagementCommands.cs index 692f37ba8b9..a47b6509833 100644 --- a/libs/cluster/Session/RespClusterSlotManagementCommands.cs +++ b/libs/cluster/Session/RespClusterSlotManagementCommands.cs @@ -288,8 +288,7 @@ private bool NetworkClusterDelKeysInSlot(out bool invalidParameters) var slots = new HashSet { slot }; ClusterManager.DeleteKeysInSlotsFromMainStore(basicGarnetApi, slots); - if (!clusterProvider.serverOptions.DisableObjects) - ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots); + ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); @@ -325,8 +324,7 @@ private bool NetworkClusterDelKeysInSlotRange(out bool invalidParameters) } ClusterManager.DeleteKeysInSlotsFromMainStore(basicGarnetApi, slots); - if (!clusterProvider.serverOptions.DisableObjects) - ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots); + ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs index b7826d8f79c..d1db2b6fba2 100644 --- a/libs/host/Configuration/Options.cs +++ b/libs/host/Configuration/Options.cs @@ -90,48 +90,12 @@ internal sealed class Options : ICloneable public string ReadCachePageSize { get; set; } [MemorySizeValidation(false)] - [Option("obj-heap-memory", Required = false, HelpText = "Object store heap memory size in bytes (Sum of size taken up by all object instances in the heap)")] - public string ObjectStoreHeapMemorySize { get; set; } - - [MemorySizeValidation] - [Option("obj-log-memory", Required = false, HelpText = "Object store log memory used in bytes (Size of only the log with references to heap objects, excludes size of heap memory consumed by the objects themselves referred to from the log)")] - public string ObjectStoreLogMemorySize { get; set; } - - [MemorySizeValidation] - [Option("obj-page", Required = false, HelpText = "Size of each object store page in bytes (rounds down to power of 2)")] - public string ObjectStorePageSize { get; set; } - - [MemorySizeValidation] - [Option("obj-segment", Required = false, HelpText = "Size of each object store log segment in bytes on disk (rounds down to power of 2)")] - public string ObjectStoreSegmentSize { get; set; } - - [MemorySizeValidation] - [Option("obj-index", Required = false, HelpText = "Start size of object store hash index in bytes (rounds down to power of 2)")] - public string ObjectStoreIndexSize { get; set; } - - [MemorySizeValidation(false)] - [Option("obj-index-max-size", Required = false, HelpText = "Max size of object store hash index in bytes (rounds down to power of 2)")] - public string ObjectStoreIndexMaxSize { get; set; } - - [PercentageValidation] - [Option("obj-mutable-percent", Required = false, HelpText = "Percentage of object store log memory that is kept mutable")] - public int ObjectStoreMutablePercent { get; set; } - - [OptionValidation] - [Option("obj-readcache", Required = false, HelpText = "Enables object store read cache for faster access to on-disk records.")] - public bool? EnableObjectStoreReadCache { get; set; } - - [MemorySizeValidation] - [Option("obj-readcache-log-memory", Required = false, HelpText = "Total object store read cache log memory used in bytes (rounds down to power of 2)")] - public string ObjectStoreReadCacheLogMemorySize { get; set; } - - [MemorySizeValidation] - [Option("obj-readcache-page", Required = false, HelpText = "Size of each object store read cache page in bytes (rounds down to power of 2)")] - public string ObjectStoreReadCachePageSize { get; set; } + [Option("heap-memory", Required = false, HelpText = "Heap memory size in bytes (Sum of size taken up by all object instances in the heap)")] + public string HeapMemorySize { get; set; } [MemorySizeValidation(false)] - [Option("obj-readcache-heap-memory", Required = false, HelpText = "Object store read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap)")] - public string ObjectStoreReadCacheHeapMemorySize { get; set; } + [Option("readcache-heap-memory", Required = false, HelpText = "Read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap)")] + public string ReadCacheHeapMemorySize { get; set; } [OptionValidation] [Option("storage-tier", Required = false, HelpText = "Enable tiering of records (hybrid log) to storage, to support a larger-than-memory store. Use --logdir to specify storage directory.")] @@ -141,10 +105,6 @@ internal sealed class Options : ICloneable [Option("copy-reads-to-tail", Required = false, HelpText = "When records are read from the main store's in-memory immutable region or storage device, copy them to the tail of the log.")] public bool? CopyReadsToTail { get; set; } - [OptionValidation] - [Option("obj-copy-reads-to-tail", Required = false, HelpText = "When records are read from the object store's in-memory immutable region or storage device, copy them to the tail of the log.")] - public bool? ObjectStoreCopyReadsToTail { get; set; } - [LogDirValidation(false, false)] [Option('l', "logdir", Required = false, HelpText = "Storage directory for tiered records (hybrid log), if storage tiering (--storage) is enabled. Uses current directory if unspecified.")] public string LogDir { get; set; } @@ -169,10 +129,6 @@ internal sealed class Options : ICloneable [Option("pubsub-pagesize", Required = false, HelpText = "Page size of log used for pub/sub (rounds down to power of 2)")] public string PubSubPageSize { get; set; } - [OptionValidation] - [Option("no-obj", Required = false, HelpText = "Disable support for data structure objects.")] - public bool? DisableObjects { get; set; } - [OptionValidation] [Option("cluster", Required = false, HelpText = "Enable cluster.")] public bool? EnableCluster { get; set; } @@ -273,10 +229,6 @@ internal sealed class Options : ICloneable [Option("compaction-max-segments", Required = false, HelpText = "Number of log segments created on disk before compaction triggers.")] public int CompactionMaxSegments { get; set; } - [IntRangeValidation(0, int.MaxValue)] - [Option("obj-compaction-max-segments", Required = false, HelpText = "Number of object store log segments created on disk before compaction triggers.")] - public int ObjectStoreCompactionMaxSegments { get; set; } - [OptionValidation] [Option("lua", Required = false, HelpText = "Enable Lua scripts on server.")] public bool? EnableLua { get; set; } @@ -518,8 +470,7 @@ internal sealed class Options : ICloneable [DoubleRangeValidation(0, 1)] [Option("reviv-fraction", Required = false, - HelpText = "#: Fraction of mutable in-memory log space, from the highest log address down to the read-only region, that is eligible for revivification." + - " Applies to both main and object store.")] + HelpText = "#: Fraction of mutable in-memory log space, from the highest log address down to the read-only region, that is eligible for revivification.")] public double RevivifiableFraction { get; set; } [OptionValidation] @@ -545,15 +496,9 @@ internal sealed class Options : ICloneable [OptionValidation] [Option("reviv-in-chain-only", Required = false, HelpText = "Revivify tombstoned records in tag chains only (do not use free list)." + - " Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts. Propagates to object store by default.")] + " Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts.")] public bool? RevivInChainOnly { get; set; } - [IntRangeValidation(0, int.MaxValue)] - [Option("reviv-obj-bin-record-count", Required = false, - HelpText = "Number of records in the single free record bin for the object store. The Object store has only a single bin, unlike the main store." + - " Ignored unless the main store is using the free record list.")] - public int RevivObjBinRecordCount { get; set; } - [IntRangeValidation(0, int.MaxValue)] [Option("object-scan-count-limit", Required = false, HelpText = "Limit of items to return in one iteration of *SCAN command")] public int ObjectScanCountLimit { get; set; } @@ -823,27 +768,16 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) EnableReadCache = EnableReadCache.GetValueOrDefault(), ReadCacheMemorySize = ReadCacheMemorySize, ReadCachePageSize = ReadCachePageSize, - ObjectStoreHeapMemorySize = ObjectStoreHeapMemorySize, - ObjectStoreLogMemorySize = ObjectStoreLogMemorySize, - ObjectStorePageSize = ObjectStorePageSize, - ObjectStoreSegmentSize = ObjectStoreSegmentSize, - ObjectStoreIndexSize = ObjectStoreIndexSize, - ObjectStoreIndexMaxSize = ObjectStoreIndexMaxSize, - ObjectStoreMutablePercent = ObjectStoreMutablePercent, - EnableObjectStoreReadCache = EnableObjectStoreReadCache.GetValueOrDefault(), - ObjectStoreReadCachePageSize = ObjectStoreReadCachePageSize, - ObjectStoreReadCacheLogMemorySize = ObjectStoreReadCacheLogMemorySize, - ObjectStoreReadCacheHeapMemorySize = ObjectStoreReadCacheHeapMemorySize, + HeapMemorySize = HeapMemorySize, + ReadCacheHeapMemorySize = ReadCacheHeapMemorySize, EnableStorageTier = enableStorageTier, CopyReadsToTail = CopyReadsToTail.GetValueOrDefault(), - ObjectStoreCopyReadsToTail = ObjectStoreCopyReadsToTail.GetValueOrDefault(), LogDir = logDir, CheckpointDir = checkpointDir, Recover = Recover.GetValueOrDefault(), EnableIncrementalSnapshots = EnableIncrementalSnapshots.GetValueOrDefault(), DisablePubSub = DisablePubSub.GetValueOrDefault(), PubSubPageSize = PubSubPageSize, - DisableObjects = DisableObjects.GetValueOrDefault(), EnableCluster = EnableCluster.GetValueOrDefault(), CleanClusterConfig = CleanClusterConfig.GetValueOrDefault(), ParallelMigrateTaskCount = ParallelMigrateTaskCount, @@ -863,7 +797,6 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) CompactionType = CompactionType, CompactionForceDelete = CompactionForceDelete.GetValueOrDefault(), CompactionMaxSegments = CompactionMaxSegments, - ObjectStoreCompactionMaxSegments = ObjectStoreCompactionMaxSegments, GossipSamplePercent = GossipSamplePercent, GossipDelay = GossipDelay, ClusterTimeout = ClusterTimeout, @@ -920,7 +853,6 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) RevivBinBestFitScanLimit = RevivBinBestFitScanLimit, RevivNumberOfBinsToSearch = RevivNumberOfBinsToSearch, RevivInChainOnly = RevivInChainOnly.GetValueOrDefault(), - RevivObjBinRecordCount = RevivObjBinRecordCount, EnableDebugCommand = EnableDebugCommand, EnableModuleCommand = EnableModuleCommand, ExtensionBinPaths = FileUtils.ConvertToAbsolutePaths(ExtensionBinPaths), diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf index af7be040685..a2ea1cc575d 100644 --- a/libs/host/defaults.conf +++ b/libs/host/defaults.conf @@ -42,48 +42,18 @@ /* Size of each read cache page in bytes (rounds down to power of 2) */ "ReadCachePageSize" : "32m", - /* Object store heap memory size in bytes (Sum of size taken up by all object instances in the heap) */ - "ObjectStoreHeapMemorySize" : "", + /* Heap memory size in bytes (Sum of size taken up by all object instances in the heap) */ + "HeapMemorySize" : "", - /* Object store log memory used in bytes (Size of only the log with references to heap objects, excludes size of heap memory consumed by the objects themselves referred to from the log) */ - "ObjectStoreLogMemorySize" : "32m", - - /* Size of each object store page in bytes (rounds down to power of 2) */ - "ObjectStorePageSize" : "4k", - - /* Size of each object store log segment in bytes on disk (rounds down to power of 2) */ - "ObjectStoreSegmentSize" : "32m", - - /* Start size of object store hash index in bytes (rounds down to power of 2) */ - "ObjectStoreIndexSize" : "16m", - - /* Max size of object store hash index in bytes (rounds down to power of 2) */ - "ObjectStoreIndexMaxSize": "", - - /* Percentage of object store log memory that is kept mutable */ - "ObjectStoreMutablePercent" : 90, - - /* Enables object store read cache for faster access to on-disk records */ - "EnableObjectStoreReadCache" : false, - - /* Total object store read cache log memory used in bytes (rounds down to power of 2) */ - "ObjectStoreReadCacheLogMemorySize" : "32m", - - /* Size of each object store read cache page in bytes (rounds down to power of 2) */ - "ObjectStoreReadCachePageSize" : "1m", - - /* Object store read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap) */ - "ObjectStoreReadCacheHeapMemorySize" : "", + /* Read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap) */ + "ReadCacheHeapMemorySize" : "", /* Enable tiering of records (hybrid log) to storage, to support a larger-than-memory store. Use --logdir to specify storage directory. */ "EnableStorageTier" : false, - /* When records are read from the main store's in-memory immutable region or storage device, copy them to the tail of the log. */ + /* When records are read from the main store''s in-memory immutable region or storage device, copy them to the tail of the log. */ "CopyReadsToTail" : false, - /* When records are read from the object store's in-memory immutable region or storage device, copy them to the tail of the log. */ - "ObjectStoreCopyReadsToTail" : false, - /* Storage directory for tiered records (hybrid log), if storage tiering (--storage) is enabled. Uses current directory if unspecified. */ "LogDir" : null, @@ -102,9 +72,6 @@ /* Page size of log used for pub/sub (rounds down to power of 2) */ "PubSubPageSize" : "4k", - /* Disable support for data structure objects. */ - "DisableObjects" : false, - /* Enable cluster. */ "EnableCluster" : false, @@ -191,9 +158,6 @@ /* Number of log segments created on disk before compaction triggers. */ "CompactionMaxSegments" : 32, - /* Number of object store log segments created on disk before compaction triggers. */ - "ObjectStoreCompactionMaxSegments" : 32, - /* Enable Lua scripts on server. */ "EnableLua" : false, @@ -368,9 +332,6 @@ /* Revivify tombstoned records in tag chains only (do not use free list). Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts. Propagates to object store by default. */ "RevivInChainOnly" : false, - /* Number of records in the single free record bin for the object store. The Object store has only a single bin, unlike the main store. Ignored unless the main store is using the free record list. */ - "RevivObjBinRecordCount" : 256, - /* Limit of items to return in one iteration of *SCAN command */ "ObjectScanCountLimit" : 1000, diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index e50042b2617..0b8b81433ec 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -389,8 +389,7 @@ protected void DoCompaction(GarnetDatabase db, bool isFromCheckpoint = false, IL // If periodic compaction is enabled and this is called from checkpointing, skip compaction if (isFromCheckpoint && StoreWrapper.serverOptions.CompactionFrequencySecs > 0) return; - DoCompaction(db, StoreWrapper.serverOptions.CompactionMaxSegments, - StoreWrapper.serverOptions.ObjectStoreCompactionMaxSegments, 1, + DoCompaction(db, StoreWrapper.serverOptions.CompactionMaxSegments, 1, StoreWrapper.serverOptions.CompactionType, StoreWrapper.serverOptions.CompactionForceDelete); } catch (Exception ex) @@ -435,7 +434,7 @@ protected bool GrowIndexIfNeeded(long indexMaxSize, long overflowCount, Func public class GarnetServerOptions : ServerOptions { - /// - /// Support data structure objects. - /// - public bool DisableObjects = false; - /// /// Heap memory size limit of object store. /// - public string ObjectStoreHeapMemorySize = ""; - - /// - /// Object store log memory used in bytes excluding heap memory. - /// - public string ObjectStoreLogMemorySize = "32m"; - - /// - /// Size of each object store page in bytes (rounds down to power of 2). - /// - public string ObjectStorePageSize = "4k"; - - /// - /// Size of each object store log segment in bytes on disk (rounds down to power of 2). - /// - public string ObjectStoreSegmentSize = "32m"; - - /// - /// Size of object store hash index in bytes (rounds down to power of 2). - /// - public string ObjectStoreIndexSize = "16m"; - - /// - /// Max size of object store hash index in bytes (rounds down to power of 2). - /// If unspecified, index size doesn't grow (default behavior). - /// - public string ObjectStoreIndexMaxSize = string.Empty; - - /// - /// Percentage of object store log memory that is kept mutable. - /// - public int ObjectStoreMutablePercent = 90; + public string HeapMemorySize = ""; /// /// Enable cluster. @@ -176,11 +140,6 @@ public class GarnetServerOptions : ServerOptions /// public int CompactionMaxSegments = 32; - /// - /// Number of object store log segments created on disk before compaction triggers. - /// - public int ObjectStoreCompactionMaxSegments = 32; - /// /// Percent of cluster nodes to gossip with at each gossip iteration. /// @@ -426,11 +385,6 @@ public class GarnetServerOptions : ServerOptions /// public bool RevivInChainOnly; - /// - /// Number of records in the single free record bin for the object store. - /// - public int RevivObjBinRecordCount; - /// Max size of hash index (cache lines) after rounding down size in bytes to power of 2. public int AdjustedIndexMaxCacheLines; @@ -466,13 +420,7 @@ public class GarnetServerOptions : ServerOptions public string ReadCachePageSize = "32m"; - public string ObjectStoreReadCachePageSize = "1m"; - - public string ObjectStoreReadCacheLogMemorySize = "32m"; - - public string ObjectStoreReadCacheHeapMemorySize = ""; - - public bool EnableObjectStoreReadCache = false; + public string ReadCacheHeapMemorySize = ""; public LuaOptions LuaOptions; @@ -643,7 +591,7 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, St if (LatencyMonitor && MetricsSamplingFrequency == 0) throw new Exception("LatencyMonitor requires MetricsSamplingFrequency to be set"); - heapMemorySize = ParseSize(ObjectStoreHeapMemorySize, out _); + heapMemorySize = ParseSize(HeapMemorySize, out _); logger?.LogInformation("[Store] Heap memory size is {heapMemorySize}", heapMemorySize > 0 ? PrettySize(heapMemorySize) : "unlimited"); // Read cache related settings @@ -660,7 +608,7 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, St logger?.LogInformation("[Store] Read cache enabled with page size of {ReadCachePageSize} and memory size of {ReadCacheMemorySize}", PrettySize(kvSettings.ReadCachePageSize), PrettySize(kvSettings.ReadCacheMemorySize)); - readCacheHeapMemorySize = ParseSize(ObjectStoreReadCacheHeapMemorySize, out _); + readCacheHeapMemorySize = ParseSize(ReadCacheHeapMemorySize, out _); logger?.LogInformation("[Store] Read cache heap memory size is {readCacheHeapMemorySize}", readCacheHeapMemorySize > 0 ? PrettySize(readCacheHeapMemorySize) : "unlimited"); } @@ -831,19 +779,6 @@ public int AofSizeLimitSizeBits() return (int)Math.Log(adjustedSize, 2); } - /// - /// Get object store page size - /// - /// - public int ObjectStorePageSizeBits() - { - long size = ParseSize(ObjectStorePageSize, out _); - long adjustedSize = PreviousPowerOf2(size); - if (size != adjustedSize) - logger?.LogInformation("Warning: using lower object store page size than specified (power of 2)"); - return (int)Math.Log(adjustedSize, 2); - } - /// /// Get integer value of ReplicaDisklessSyncFullSyncAofThreshold /// @@ -851,19 +786,6 @@ public int ObjectStorePageSizeBits() public long ReplicaDisklessSyncFullSyncAofThresholdValue() => ParseSize(string.IsNullOrEmpty(ReplicaDisklessSyncFullSyncAofThreshold) ? AofMemorySize : ReplicaDisklessSyncFullSyncAofThreshold, out _); - /// - /// Get object store segment size - /// - /// - public int ObjectStoreSegmentSizeBits() - { - long size = ParseSize(ObjectStoreSegmentSize, out _); - long adjustedSize = PreviousPowerOf2(size); - if (size != adjustedSize) - logger?.LogInformation("Warning: using lower object store disk segment size than specified (power of 2)"); - return (int)Math.Log(adjustedSize, 2); - } - /// /// Get device for AOF /// diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 9912c84c5d1..b173c7b82d7 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -193,8 +193,7 @@ public StoreWrapper( if (serverOptions.SlowLogThreshold > 0) this.slowLogContainer = new SlowLogContainer(serverOptions.SlowLogMaxEntries); - if (!serverOptions.DisableObjects) - this.itemBroker = new CollectionItemBroker(); + this.itemBroker = new CollectionItemBroker(); // Initialize store scripting cache if (serverOptions.EnableLua) @@ -695,12 +694,6 @@ async Task ObjectCollectTask(int objectCollectFrequencySecs, CancellationToken t Debug.Assert(objectCollectFrequencySecs > 0); try { - if (serverOptions.DisableObjects) - { - logger?.LogWarning("ExpiredObjectCollectionFrequencySecs option is configured but Object store is disabled. Stopping the background hash collect task."); - return; - } - while (true) { if (token.IsCancellationRequested) return; @@ -833,7 +826,7 @@ public bool HasKeysInSlots(List slots) } } - if (!hasKeyInSlots && !serverOptions.DisableObjects) + if (!hasKeyInSlots) { var functionsState = databaseManager.CreateFunctionsState(); var objStorefunctions = new ObjectSessionFunctions(functionsState); diff --git a/test/Garnet.fuzz/Targets/GarnetEndToEnd.cs b/test/Garnet.fuzz/Targets/GarnetEndToEnd.cs index 5217e269fe5..031f071e713 100644 --- a/test/Garnet.fuzz/Targets/GarnetEndToEnd.cs +++ b/test/Garnet.fuzz/Targets/GarnetEndToEnd.cs @@ -358,17 +358,14 @@ private static EmbeddedRespServer CreateServer() { ThreadPoolMinThreads = 100, SegmentSize = "1g", - ObjectStoreSegmentSize = "1g", EnableStorageTier = true, LogDir = LogDir.FullName, CheckpointDir = CheckpointDir.FullName, EndPoints = [new IPEndPoint(IPAddress.Loopback, 1234)], DisablePubSub = false, - DisableObjects = false, EnableDebugCommand = ConnectionProtectionOption.Yes, Recover = false, IndexSize = "1m", - ObjectStoreIndexSize = "16k", EnableCluster = true, CleanClusterConfig = true, ClusterTimeout = -1, diff --git a/test/Garnet.test.cluster/ClusterManagementTests.cs b/test/Garnet.test.cluster/ClusterManagementTests.cs index 20b87d1a3cb..4e36e8befcf 100644 --- a/test/Garnet.test.cluster/ClusterManagementTests.cs +++ b/test/Garnet.test.cluster/ClusterManagementTests.cs @@ -396,7 +396,6 @@ public void ClusterRestartNodeDropGossip() context.nodes[restartingNode] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(restartingNode), - disableObjects: true, tryRecover: false, enableAOF: true, timeout: 60, diff --git a/test/Garnet.test.cluster/ClusterMigrateTests.cs b/test/Garnet.test.cluster/ClusterMigrateTests.cs index a79e778e75d..70a412b7ad9 100644 --- a/test/Garnet.test.cluster/ClusterMigrateTests.cs +++ b/test/Garnet.test.cluster/ClusterMigrateTests.cs @@ -1908,7 +1908,7 @@ public void ClusterMigrateWrite() var sourceNodeIndex = 0; var targetNodeIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: true); + context.CreateInstances(nodes_count); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(sourceNodeIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -1977,7 +1977,7 @@ public void ClusterMigrateSetCopyUpdate(CancellationToken cancellationToken) var sourceNodeIndex = 0; var targetNodeIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: true); + context.CreateInstances(nodes_count); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(sourceNodeIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -2050,7 +2050,7 @@ public void ClusterMigrateCustomProcDelRMW(CancellationToken cancellationToken) var sourceNodeIndex = 0; var targetNodeIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: true); + context.CreateInstances(nodes_count); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(sourceNodeIndex, [(0, 16383)], addslot: true, logger: context.logger); diff --git a/test/Garnet.test.cluster/ClusterNegativeTests.cs b/test/Garnet.test.cluster/ClusterNegativeTests.cs index 8807958cac4..446ab643ae8 100644 --- a/test/Garnet.test.cluster/ClusterNegativeTests.cs +++ b/test/Garnet.test.cluster/ClusterNegativeTests.cs @@ -163,7 +163,7 @@ public void ClusterExceptionInjectionAtPrimarySyncSession([Values] bool enableDi var primaryIndex = 0; var replicaIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, enableDisklessSync: enableDisklessSync, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, enableDisklessSync: enableDisklessSync, timeout: timeout); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -193,7 +193,7 @@ public void ClusterFailoverDuringRecovery(CancellationToken cancellationToken) var primaryIndex = 0; var replicaIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, enableDisklessSync: true, timeout: timeout, replicaDisklessSyncDelay: 10); + context.CreateInstances(nodes_count, enableAOF: true, enableDisklessSync: true, timeout: timeout, replicaDisklessSyncDelay: 10); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -260,7 +260,7 @@ public void ClusterCheckpointAcquireTest([Values] bool fastAofTruncate, [Values] var primaryIndex = 0; var replicaIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, timeout: timeout, FastAofTruncate: fastAofTruncate, OnDemandCheckpoint: onDemandCheckpoint, CommitFrequencyMs: fastAofTruncate ? -1 : 0); + context.CreateInstances(nodes_count, enableAOF: true, timeout: timeout, FastAofTruncate: fastAofTruncate, OnDemandCheckpoint: onDemandCheckpoint, CommitFrequencyMs: fastAofTruncate ? -1 : 0); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -297,7 +297,6 @@ public void ClusterReplicaAttachIntenseWrite(CancellationToken cancellationToken var nodes_count = 2; context.CreateInstances( nodes_count, - disableObjects: false, enableAOF: true, timeout: timeout, OnDemandCheckpoint: true, @@ -373,7 +372,7 @@ public void ClusterFailedToAddAofSyncTask() var primaryIndex = 0; var replicaIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, timeout: timeout); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -423,7 +422,7 @@ public void ClusterReplicaSyncTimeoutTest() var primaryIndex = 0; var replicaIndex = 1; var nodes_count = 2; - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, timeout: timeout, replicaSyncTimeout: 1); + context.CreateInstances(nodes_count, enableAOF: true, timeout: timeout, replicaSyncTimeout: 1); context.CreateConnection(); _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); @@ -472,7 +471,6 @@ public async Task ClusterParallelFailoverOnDistinctShards(CancellationToken canc var nodes_count = 4; context.CreateInstances( nodes_count, - disableObjects: false, enableAOF: true, timeout: timeout, OnDemandCheckpoint: true, diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index b15a2506304..da28e01eb55 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -151,7 +151,6 @@ public void RegisterCustomTxn(string name, Func proc /// /// /// - /// /// /// /// @@ -185,7 +184,6 @@ public void CreateInstances( bool enableCluster = true, bool cleanClusterConfig = true, bool tryRecover = false, - bool disableObjects = false, bool lowMemory = false, string memorySize = default, string pageSize = default, @@ -234,7 +232,6 @@ public void CreateInstances( (nodes, nodeOptions) = TestUtils.CreateGarnetCluster( TestFolder, disablePubSub: disablePubSub, - disableObjects: disableObjects, enableCluster: enableCluster, endpoints: endpoints, enableAOF: enableAOF, @@ -294,7 +291,6 @@ public void CreateInstances( /// /// /// - /// /// /// /// @@ -321,7 +317,6 @@ public GarnetServer CreateInstance( bool cleanClusterConfig = true, bool disableEpochCollision = false, bool tryRecover = false, - bool disableObjects = false, bool lowMemory = false, string MemorySize = default, string PageSize = default, @@ -350,7 +345,6 @@ public GarnetServer CreateInstance( endpoint, enableCluster: enableCluster, disablePubSub: true, - disableObjects: disableObjects, enableAOF: enableAOF, timeout: timeout, gossipDelay: gossipDelay, diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs index d85d1305aae..d5326137f46 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs @@ -114,7 +114,7 @@ public void ClusterSRTest([Values] bool disableObjects) var primary_count = 1; var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -151,7 +151,7 @@ public void ClusterSRNoCheckpointRestartSecondary([Values] bool performRMW, [Val var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -195,7 +195,6 @@ public void ClusterSRNoCheckpointRestartSecondary([Values] bool performRMW, [Val // Restart secondary context.nodes[1] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(1), - disableObjects: disableObjects, tryRecover: true, enableAOF: true, timeout: timeout, @@ -217,7 +216,7 @@ public void ClusterSRPrimaryCheckpoint([Values] bool performRMW, [Values] bool d var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -268,7 +267,6 @@ public void ClusterSRPrimaryCheckpoint([Values] bool performRMW, [Values] bool d // Restart secondary context.nodes[1] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(1), - disableObjects: disableObjects, tryRecover: true, enableAOF: true, timeout: timeout, @@ -316,7 +314,7 @@ void ClusterSRPrimaryCheckpointRetrieve(bool performRMW, bool disableObjects, bo var primary_count = 1; var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, lowMemory: lowMemory, segmentSize: manySegments ? "4k" : "1g", DisableStorageTier: disableStorageTier, EnableIncrementalSnapshots: incrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, lowMemory: lowMemory, segmentSize: manySegments ? "4k" : "1g", DisableStorageTier: disableStorageTier, EnableIncrementalSnapshots: incrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -364,7 +362,6 @@ void ClusterSRPrimaryCheckpointRetrieve(bool performRMW, bool disableObjects, bo // Restart secondary context.nodes[replicaIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(replicaIndex), - disableObjects: disableObjects, tryRecover: true, enableAOF: true, timeout: timeout, @@ -392,7 +389,7 @@ public void ClusterSRAddReplicaAfterPrimaryCheckpoint([Values] bool performRMW, var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, tryRecover: true, disableObjects: disableObjects, lowMemory: lowMemory, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, tryRecover: true, lowMemory: lowMemory, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, new List<(int, int)>() { (0, 16383) }, true, context.logger)); @@ -451,7 +448,7 @@ public void ClusterSRPrimaryRestart([Values] bool performRMW, [Values] bool disa var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, tryRecover: true, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, tryRecover: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, new List<(int, int)>() { (0, 16383) }, true, context.logger)); @@ -490,7 +487,6 @@ public void ClusterSRPrimaryRestart([Values] bool performRMW, [Values] bool disa // Restart Primary context.nodes[0] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(0), - disableObjects: disableObjects, tryRecover: true, enableAOF: true, timeout: timeout, @@ -543,7 +539,7 @@ public void ClusterSRRedirectWrites() public void ClusterSRReplicaOfTest([Values] bool performRMW) { var nodes_count = 2; - context.CreateInstances(nodes_count, tryRecover: true, disableObjects: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, tryRecover: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, [(0, 16383)], true, context.logger)); @@ -582,7 +578,7 @@ public void ClusterReplicationSimpleFailover([Values] bool performRMW, [Values] var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -655,7 +651,7 @@ public void ClusterFailoverAttachReplicas([Values] bool performRMW, [Values] boo var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -736,7 +732,7 @@ public void ClusterReplicationCheckpointCleanupTest([Values] bool performRMW, [V var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, tryRecover: true, disableObjects: disableObjects, lowMemory: true, segmentSize: "4k", EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, useNativeDeviceLinux: true); + context.CreateInstances(nodes_count, tryRecover: true, lowMemory: true, segmentSize: "4k", EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, useNativeDeviceLinux: true); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, [(0, 16383)], true, context.logger)); context.clusterTestUtils.BumpEpoch(0, logger: context.logger); @@ -774,7 +770,7 @@ public void ClusterMainMemoryReplicationAttachReplicas() var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, FastAofTruncate: true, OnDemandCheckpoint: true, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, FastAofTruncate: true, OnDemandCheckpoint: true, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, new List<(int, int)>() { (0, 16383) }, true)); @@ -818,7 +814,7 @@ public void ClusterDontKnowReplicaFailTest([Values] bool performRMW, [Values] bo var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, FastAofTruncate: MainMemoryReplication, OnDemandCheckpoint: onDemandCheckpoint, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, FastAofTruncate: MainMemoryReplication, OnDemandCheckpoint: onDemandCheckpoint, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var primaryNodeIndex = 0; @@ -919,7 +915,7 @@ void ClusterDivergentReplicasTest(bool performRMW, bool disableObjects, bool ckp var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, FastAofTruncate: mainMemoryReplication, CommitFrequencyMs: mainMemoryReplication ? -1 : 0, OnDemandCheckpoint: mainMemoryReplication, FastCommit: fastCommit, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, FastAofTruncate: mainMemoryReplication, CommitFrequencyMs: mainMemoryReplication ? -1 : 0, OnDemandCheckpoint: mainMemoryReplication, FastCommit: fastCommit, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1053,7 +1049,7 @@ public void ClusterReplicateFails() ServerCredential userCreds = new(UserName, Password, IsAdmin: true, UsedForClusterAuth: false, IsClearText: true); context.GenerateCredentials([userCreds, clusterCreds]); - context.CreateInstances(2, disableObjects: true, disablePubSub: true, enableAOF: true, clusterCreds: clusterCreds, useAcl: true, FastAofTruncate: true, CommitFrequencyMs: -1, asyncReplay: asyncReplay); + context.CreateInstances(2, disablePubSub: true, enableAOF: true, clusterCreds: clusterCreds, useAcl: true, FastAofTruncate: true, CommitFrequencyMs: -1, asyncReplay: asyncReplay); var primaryEndpoint = (IPEndPoint)context.endpoints.First(); var replicaEndpoint = (IPEndPoint)context.endpoints.Last(); @@ -1083,7 +1079,7 @@ public void ClusterReplicationCheckpointAlignmentTest([Values] bool performRMW) var primaryNodeIndex = 0; var replicaNodeIndex = 1; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, FastAofTruncate: true, CommitFrequencyMs: -1, OnDemandCheckpoint: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, FastAofTruncate: true, CommitFrequencyMs: -1, OnDemandCheckpoint: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1124,7 +1120,6 @@ public void ClusterReplicationCheckpointAlignmentTest([Values] bool performRMW) // Restart primary and do not recover context.nodes[primaryNodeIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(primaryNodeIndex), - disableObjects: true, tryRecover: false, enableAOF: true, FastAofTruncate: true, @@ -1139,7 +1134,6 @@ public void ClusterReplicationCheckpointAlignmentTest([Values] bool performRMW) // Restart secondary and recover context.nodes[replicaNodeIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(replicaNodeIndex), - disableObjects: true, tryRecover: true, enableAOF: true, FastAofTruncate: true, @@ -1194,7 +1188,7 @@ public void ClusterReplicationLua([Values] bool luaTransactionMode) var replicaNodeIndex = 1; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, enableLua: true, luaTransactionMode: luaTransactionMode); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, enableLua: true, luaTransactionMode: luaTransactionMode); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1225,7 +1219,7 @@ public void ClusterReplicationStoredProc([Values] bool enableDisklessSync, [Valu var expectedKeys = new[] { "X", "Y" }; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, enableDisklessSync: enableDisklessSync); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, enableDisklessSync: enableDisklessSync); context.CreateConnection(useTLS: useTLS); var primaryServer = context.clusterTestUtils.GetServer(primaryNodeIndex); @@ -1290,7 +1284,7 @@ public void ClusterReplicationManualCheckpointing() var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1); context.CreateConnection(useTLS: true); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1363,7 +1357,7 @@ public async Task ReplicaSyncTaskFaultsRecoverAsync(ExceptionInjectionType fault var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1, clusterReplicationReestablishmentTimeout: 1); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1, clusterReplicationReestablishmentTimeout: 1); context.CreateConnection(useTLS: true); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1441,7 +1435,7 @@ public async Task ClusterReplicationMultiRestartRecover() ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, cleanClusterConfig: false); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, cleanClusterConfig: false); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1489,7 +1483,6 @@ void RestartRecover(int iteration) context.nodes[replicaNodeIndex].Dispose(false); context.nodes[replicaNodeIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(replicaNodeIndex), - disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, @@ -1510,7 +1503,7 @@ public async Task ReplicasRestartAsReplicasAsync(CancellationToken cancellation) var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1, clusterReplicationReestablishmentTimeout: 1); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: true, tryRecover: false, FastAofTruncate: true, CommitFrequencyMs: -1, clusterReplicationReestablishmentTimeout: 1); context.CreateConnection(useTLS: true); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1570,7 +1563,6 @@ public async Task PrimaryUnavailableRecoveryAsync(ExceptionInjectionType faultTy nodes_count, tryRecover: true, disablePubSub: false, - disableObjects: false, enableAOF: true, AofMemorySize: "128m", CommitFrequencyMs: -1, @@ -1900,7 +1892,7 @@ public void ClusterReplicationDivergentHistoryWithoutCheckpoint() var replicaNodeIndex = 1; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, cleanClusterConfig: false); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, cleanClusterConfig: false); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1926,7 +1918,6 @@ public void ClusterReplicationDivergentHistoryWithoutCheckpoint() // Restart primary and do not recover context.nodes[primaryNodeIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(primaryNodeIndex), - disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, @@ -1943,7 +1934,6 @@ public void ClusterReplicationDivergentHistoryWithoutCheckpoint() // Restart replica with recover to be ahead of primary but without checkpoint context.nodes[replicaNodeIndex] = context.CreateInstance( context.clusterTestUtils.GetEndPoint(replicaNodeIndex), - disableObjects: false, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay, diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 9a6c75dbe15..3cdf1122a51 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -109,7 +109,7 @@ public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Value var nodes_count = 2; var primaryIndex = 0; var replicaIndex = 1; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -151,7 +151,7 @@ public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] var nodes_count = 2; var primaryIndex = 0; var replicaIndex = 1; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout, replicaDisklessSyncFullSyncAofThreshold: forceFullSync ? "1k" : string.Empty); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout, replicaDisklessSyncFullSyncAofThreshold: forceFullSync ? "1k" : string.Empty); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -209,7 +209,7 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, var primaryIndex = 0; var replicaOneIndex = 1; var replicaTwoIndex = 2; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -301,7 +301,7 @@ public void ClusterDisklessSyncParallelAttach([Values] bool disableObjects, [Val var replicaOneIndex = 1; var replicaTwoIndex = 2; var replicaThreeIndex = 3; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -344,7 +344,7 @@ public void ClusterDisklessSyncFailover([Values] bool disableObjects, [Values] b int[] nOffsets = [primary, replicaOne, replicaTwo]; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); + context.CreateInstances(nodes_count, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica diff --git a/test/Garnet.test/CacheSizeTrackerTests.cs b/test/Garnet.test/CacheSizeTrackerTests.cs index b19582347f1..30937ba6ba9 100644 --- a/test/Garnet.test/CacheSizeTrackerTests.cs +++ b/test/Garnet.test/CacheSizeTrackerTests.cs @@ -25,7 +25,7 @@ public class CacheSizeTrackerTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "2k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreHeapMemorySize: "3k"); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "2k", pageSize: "512", lowMemory: true, indexSize: "1k", heapMemorySize: "3k"); server.Start(); store = server.Provider.StoreWrapper.store; cacheSizeTracker = server.Provider.StoreWrapper.sizeTracker; @@ -92,7 +92,7 @@ public void IncreaseEmptyPageCountTest() public void ReadCacheIncreaseEmptyPageCountTest() { server?.Dispose(); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "1k", pageSize: "512", lowMemory: true, objectStoreIndexSize: "1k", objectStoreReadCacheHeapMemorySize: "1k", enableObjectStoreReadCache: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: "1k", pageSize: "512", lowMemory: true, indexSize: "1k", readCacheHeapMemorySize: "1k", enableReadCache: true); server.Start(); store = server.Provider.StoreWrapper.store; cacheSizeTracker = server.Provider.StoreWrapper.sizeTracker; @@ -126,7 +126,7 @@ public void ReadCacheIncreaseEmptyPageCountTest() // K/V lengths fit into a single byte each, so the record size is: RecordInfo, MinLengthMetadataSize, keyLength, valueLength; the total rounded up to record alignment. // ValueLength is 4 for the ObjectId, so this becomes 8 + 3 + (10 or 11) + 4 totalling 25 or 26, both rounding up to 32 which is a even divisor for the page size. // First valid address is 64, and there are 25 total records. - var info = TestUtils.GetStoreAddressInfo(redis.GetServer(TestUtils.EndPoint), includeReadCache: true, isObjectStore: true); + var info = TestUtils.GetStoreAddressInfo(redis.GetServer(TestUtils.EndPoint), includeReadCache: true); ClassicAssert.AreEqual(64 + 32 * NumRecords, info.ReadCacheTailAddress); if (!readCacheEpcEvent.Wait(TimeSpan.FromSeconds(3 * 3 * LogSizeTracker.ResizeTaskDelaySeconds))) diff --git a/test/Garnet.test/GarnetClientTests.cs b/test/Garnet.test/GarnetClientTests.cs index f69389dafa2..0610aa7e9c8 100644 --- a/test/Garnet.test/GarnetClientTests.cs +++ b/test/Garnet.test/GarnetClientTests.cs @@ -372,7 +372,7 @@ public async Task ShouldNotThrowExceptionForEmptyArrayResponseAsync() [Test] public async Task CanUseMGetTests([Values] bool disableObjectStore) { - using var server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: disableObjectStore); + using var server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); using var db = TestUtils.GetGarnetClient(); diff --git a/test/Garnet.test/GarnetServerConfigTests.cs b/test/Garnet.test/GarnetServerConfigTests.cs index ad2f5de785b..953a5d2c4e0 100644 --- a/test/Garnet.test/GarnetServerConfigTests.cs +++ b/test/Garnet.test/GarnetServerConfigTests.cs @@ -125,7 +125,7 @@ public void ImportExportConfigLocal() var binPaths = new[] { GetFullExtensionBinPath("Garnet.test"), GetFullExtensionBinPath("Garnet.test.cluster") }; var modules = new[] { Assembly.GetExecutingAssembly().Location }; - var args = new[] { "--config-export-path", configPath, "-p", "4m", "-m", "128m", "-s", "2g", "--index", "128m", "--recover", "--port", "53", "--reviv-obj-bin-record-count", "2", "--reviv-fraction", "0.5", "--reviv-bin-record-counts", "1,2,3", "--extension-bin-paths", string.Join(',', binPaths), "--loadmodulecs", string.Join(',', modules) }; + var args = new[] { "--config-export-path", configPath, "-p", "4m", "-m", "128m", "-s", "2g", "--index", "128m", "--recover", "--port", "53", "--reviv-fraction", "0.5", "--reviv-bin-record-counts", "1,2,3", "--extension-bin-paths", string.Join(',', binPaths), "--loadmodulecs", string.Join(',', modules) }; parseSuccessful = ServerSettingsManager.TryParseCommandLineArguments(args, out options, out invalidOptions, out optionsJson, out exitGracefully, silentMode: true); ClassicAssert.IsTrue(parseSuccessful); ClassicAssert.AreEqual(invalidOptions.Count, 0); @@ -133,7 +133,6 @@ public void ImportExportConfigLocal() ClassicAssert.AreEqual("128m", options.MemorySize); ClassicAssert.AreEqual("2g", options.SegmentSize); ClassicAssert.AreEqual(53, options.Port); - ClassicAssert.AreEqual(2, options.RevivObjBinRecordCount); ClassicAssert.AreEqual(0.5, options.RevivifiableFraction); CollectionAssert.AreEqual(new[] { 1, 2, 3 }, options.RevivBinRecordCounts); ClassicAssert.IsTrue(options.Recover); @@ -182,7 +181,7 @@ public void ImportExportConfigLocal() // Import from previous export command, include command line args, export to file // Check values from import path override values from default.conf, and values from command line override values from default.conf and import path binPaths = [GetFullExtensionBinPath("Garnet.test")]; - args = ["--config-import-path", configPath, "-p", "12m", "-s", "1g", "--recover", "false", "--index", "256m", "--port", "0", "--no-obj", "--aof", "--reviv-bin-record-counts", "4,5", "--extension-bin-paths", string.Join(',', binPaths)]; + args = ["--config-import-path", configPath, "-p", "12m", "-s", "1g", "--recover", "false", "--index", "256m", "--port", "0", "--aof", "--reviv-bin-record-counts", "4,5", "--extension-bin-paths", string.Join(',', binPaths)]; parseSuccessful = ServerSettingsManager.TryParseCommandLineArguments(args, out options, out invalidOptions, out optionsJson, out exitGracefully, silentMode: true); ClassicAssert.IsTrue(parseSuccessful); ClassicAssert.AreEqual(invalidOptions.Count, 0); @@ -191,7 +190,6 @@ public void ImportExportConfigLocal() ClassicAssert.AreEqual("1g", options.SegmentSize); ClassicAssert.AreEqual(0, options.Port); ClassicAssert.IsFalse(options.Recover); - ClassicAssert.IsTrue(options.DisableObjects); ClassicAssert.IsTrue(options.EnableAOF); CollectionAssert.AreEqual(new[] { 4, 5 }, options.RevivBinRecordCounts); CollectionAssert.AreEqual(binPaths, options.ExtensionBinPaths); @@ -210,8 +208,6 @@ public void ImportExportConfigLocal() ((JsonElement)nonDefaultOptions[nameof(Options.RevivBinRecordCounts)]).EnumerateArray() .Select(i => i.GetInt32())); ClassicAssert.IsFalse(nonDefaultOptions.ContainsKey(nameof(Options.Recover))); - ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.DisableObjects))); - ClassicAssert.IsTrue(((JsonElement)nonDefaultOptions[nameof(Options.DisableObjects)]).GetBoolean()); // No import path, include command line args // Check that all invalid options flagged diff --git a/test/Garnet.test/IndexGrowthTests.cs b/test/Garnet.test/IndexGrowthTests.cs index da76679c5f2..949bdee4e37 100644 --- a/test/Garnet.test/IndexGrowthTests.cs +++ b/test/Garnet.test/IndexGrowthTests.cs @@ -70,7 +70,7 @@ public void IndexGrowthTest() [Test] public void ObjectStoreIndexGrowthTest() { - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, objectStoreIndexSize: "64", objectStoreIndexMaxSize: "128", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, indexSize: "64", indexMaxSize: "128", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); server.Start(); var store = server.Provider.StoreWrapper.store; @@ -181,7 +181,7 @@ public void IndexGrowthTestWithDiskReadAndCheckpoint() [Test] public void ObjectStoreIndexGrowthTestWithDiskReadAndCheckpoint() { - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, objectStoreIndexSize: "512", objectStoreIndexMaxSize: "1k", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, indexSize: "512", indexMaxSize: "1k", indexResizeFrequencySecs: indexResizeTaskDelaySeconds); server.Start(); var store = server.Provider.StoreWrapper.store; @@ -231,7 +231,7 @@ public void ObjectStoreIndexGrowthTestWithDiskReadAndCheckpoint() } server.Dispose(false); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, objectStoreIndexSize: "512", objectStoreIndexMaxSize: "1k"); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, indexSize: "512", indexMaxSize: "1k"); server.Start(); using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) diff --git a/test/Garnet.test/ReadCacheTests.cs b/test/Garnet.test/ReadCacheTests.cs index 4e6015bc543..2199d13a841 100644 --- a/test/Garnet.test/ReadCacheTests.cs +++ b/test/Garnet.test/ReadCacheTests.cs @@ -16,7 +16,7 @@ public class ReadCacheTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, enableObjectStoreReadCache: true, lowMemory: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, lowMemory: true); server.Start(); } @@ -99,7 +99,7 @@ public void ObjectStoreReadCacheTest() using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true)); var db = redis.GetDatabase(0); var server = redis.GetServer(TestUtils.EndPoint); - var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); // Start at tail address of 24 ClassicAssert.AreEqual(24, info.ReadCacheBeginAddress); @@ -113,7 +113,7 @@ public void ObjectStoreReadCacheTest() _ = db.ListRightPush(key, value); } - info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); // Ensure data has spilled to disk ClassicAssert.Greater(info.HeadAddress, info.BeginAddress); @@ -124,13 +124,13 @@ public void ObjectStoreReadCacheTest() var key0 = $"objKey00000"; var value0 = db.ListGetByIndex(key0, 0); ClassicAssert.AreEqual("objVal00000", (string)value0); - info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); ClassicAssert.AreEqual(24 + 24, info.ReadCacheTailAddress); // 24 bytes for one record // Issue read again to ensure read cache is not updated value0 = db.ListGetByIndex(key0, 0); ClassicAssert.AreEqual("objVal00000", (string)value0); - info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); ClassicAssert.AreEqual(24 + 24, info.ReadCacheTailAddress); // Read more keys to update read cache @@ -140,7 +140,7 @@ public void ObjectStoreReadCacheTest() var value = db.ListGetByIndex(key, 0); ClassicAssert.AreEqual($"objVal{j:00000}", (string)value); } - info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); ClassicAssert.AreEqual(24 + 24 * 40 + 8, info.ReadCacheTailAddress); // 24 bytes for 20 records + 8 bytes for page boundary alignment ClassicAssert.AreEqual(24, info.ReadCacheBeginAddress); // Read cache should not have been evicted yet @@ -151,7 +151,7 @@ public void ObjectStoreReadCacheTest() var value = db.ListGetByIndex(key, 0); ClassicAssert.AreEqual($"objVal{j:00000}", (string)value); } - info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); ClassicAssert.Greater(info.ReadCacheBeginAddress, 24); // Read cache entries should have been evicted } } diff --git a/test/Garnet.test/RespAdminCommandsTests.cs b/test/Garnet.test/RespAdminCommandsTests.cs index 87cc59e6e6a..389e63c14b5 100644 --- a/test/Garnet.test/RespAdminCommandsTests.cs +++ b/test/Garnet.test/RespAdminCommandsTests.cs @@ -146,12 +146,12 @@ public void SeSaveTest() } [Test] - public void SeSaveRecoverTest([Values] bool disableObj, [Values] bool useAzure) + public void SeSaveRecoverTest([Values] bool useAzure) { if (useAzure) TestUtils.IgnoreIfNotRunningAzureTests(); server.Dispose(); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: disableObj, useAzureStorage: useAzure); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, useAzureStorage: useAzure); server.Start(); using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) @@ -358,7 +358,7 @@ public void SeSaveRecoverMultipleObjectsTest(int memorySize, int recoveryMemoryS } server.Dispose(false); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, memorySize: sizeToString(recoveryMemorySize), pageSize: sizeToString(pageSize), objectStoreHeapMemorySize: "64k"); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, memorySize: sizeToString(recoveryMemorySize), pageSize: sizeToString(pageSize), heapMemorySize: "64k"); server.Start(); ClassicAssert.LessOrEqual(server.Provider.StoreWrapper.store.MaxAllocatedPageCount, (recoveryMemorySize / pageSize) + 1); @@ -383,10 +383,8 @@ public void SeSaveRecoverMultipleObjectsTest(int memorySize, int recoveryMemoryS [TestCase("5k", "64k")] public void SeSaveRecoverMultipleKeysTest(string memorySize, string recoveryMemorySize) { - bool disableObj = true; - server.Dispose(); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: disableObj, lowMemory: true, memorySize: memorySize, pageSize: "512", enableAOF: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, memorySize: memorySize, pageSize: "512", enableAOF: true); server.Start(); using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) @@ -425,7 +423,7 @@ public void SeSaveRecoverMultipleKeysTest(string memorySize, string recoveryMemo } server.Dispose(false); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: disableObj, tryRecover: true, lowMemory: true, memorySize: recoveryMemorySize, pageSize: "512", enableAOF: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, lowMemory: true, memorySize: recoveryMemorySize, pageSize: "512", enableAOF: true); server.Start(); using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true))) diff --git a/test/Garnet.test/RespConfigTests.cs b/test/Garnet.test/RespConfigTests.cs index 54770eb4fd5..bdaf0edeecc 100644 --- a/test/Garnet.test/RespConfigTests.cs +++ b/test/Garnet.test/RespConfigTests.cs @@ -26,9 +26,7 @@ public class RespConfigTests GarnetServer server; private string memorySize = "17g"; private string indexSize = "64m"; - private string objectStoreLogMemorySize = "17m"; - private string objectStoreHeapMemorySize = "32m"; - private string objectStoreIndexSize = "8m"; + private string heapMemorySize = "32m"; private bool useReviv; public RespConfigTests(bool useReviv) @@ -43,9 +41,7 @@ public void Setup() server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, memorySize: memorySize, indexSize: indexSize, - objectStoreLogMemorySize: objectStoreLogMemorySize, - objectStoreIndexSize: objectStoreIndexSize, - objectStoreHeapMemorySize: objectStoreHeapMemorySize, + heapMemorySize: heapMemorySize, useReviv: useReviv); server.Start(); } @@ -226,7 +222,7 @@ public void ConfigObjHeapSizeTest(string smallerSize, string largerSize, string var db = redis.GetDatabase(0); var option = "obj-heap-memory"; - var currObjHeapSize = ServerOptions.ParseSize(objectStoreHeapMemorySize, out _); + var currObjHeapSize = ServerOptions.ParseSize(heapMemorySize, out _); // Check initial heap size before any changes var metrics = server.Metrics.GetInfoMetrics(InfoMetricsType.MEMORY); @@ -236,7 +232,7 @@ public void ConfigObjHeapSizeTest(string smallerSize, string largerSize, string ClassicAssert.AreEqual(currObjHeapSize, objHeapTargetSize); // Try to set heap size to the same value as current - var result = db.Execute("CONFIG", "SET", option, objectStoreHeapMemorySize); + var result = db.Execute("CONFIG", "SET", option, heapMemorySize); ClassicAssert.AreEqual("OK", result.ToString()); // Heap size should remain unchanged @@ -286,9 +282,7 @@ public class RespConfigUtilizationTests GarnetServer server; private string memorySize = "3m"; private string indexSize = "1m"; - private string objectStoreLogMemorySize = "2500"; - private string objectStoreHeapMemorySize = "1m"; - private string objectStoreIndexSize = "2048"; + private string heapMemorySize = "1m"; private string pageSize = "1024"; private bool useReviv; @@ -305,10 +299,7 @@ public void Setup() memorySize: memorySize, indexSize: indexSize, pageSize: pageSize, - objectStorePageSize: pageSize, - objectStoreLogMemorySize: objectStoreLogMemorySize, - objectStoreIndexSize: objectStoreIndexSize, - objectStoreHeapMemorySize: objectStoreHeapMemorySize, + heapMemorySize: heapMemorySize, useReviv: useReviv); server.Start(); } @@ -324,28 +315,27 @@ public void TearDown() /// This test verifies that dynamically changing the memory size configuration using CONFIG SET /// incurs the expected shifts in the head and tail addresses of the store. /// - /// Store Type (Main / Object) /// Memory size smaller than the initial size /// Memory size larger than the initial size (within buffer bounds) [Test] - [TestCase(StoreType.Main, "1m", "4m")] - [TestCase(StoreType.Main, "1024k", "4000k")] - [TestCase(StoreType.Object, "1024", "4000")] - [TestCase(StoreType.Object, "1024", "4096")] - public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string smallerSize, string largerSize) + [TestCase("1m", "4m")] + [TestCase("1024k", "4000k")] + [TestCase( "1024", "4000")] + [TestCase("1024", "4096")] + public void ConfigSetMemorySizeUtilizationTest(string smallerSize, string largerSize) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig(allowAdmin: true)); var db = redis.GetDatabase(0); - var option = storeType == StoreType.Main ? "memory" : "obj-log-memory"; - var initMemorySize = storeType == StoreType.Main ? memorySize : objectStoreLogMemorySize; + var option = "memory"; + var initMemorySize = memorySize; var currMemorySize = TestUtils.GetEffectiveMemorySize(initMemorySize, pageSize, out var parsedPageSize); var garnetServer = redis.GetServer(TestUtils.EndPoint); - var info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); - ClassicAssert.AreEqual(storeType == StoreType.Main ? 64 : 24, info.TailAddress); + var info = TestUtils.GetStoreAddressInfo(garnetServer); + ClassicAssert.AreEqual(64, info.TailAddress); var i = 0; - var val = new RedisValue(new string('x', storeType == StoreType.Main ? 512 - 32 : 1)); + var val = new RedisValue(new string('x', 512 - 32)); // Insert records until head address moves var prevHead = info.HeadAddress; @@ -353,14 +343,11 @@ public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string small while (info.HeadAddress == prevHead) { var key = $"key{i++:00000}"; - if (storeType == StoreType.Main) - _ = db.StringSet(key, val); - else - _ = db.ListRightPush(key, [val]); + _ = db.StringSet(key, val); prevHead = info.HeadAddress; prevTail = info.TailAddress; - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); } // Verify that records were inserted up to the configured memory size limit @@ -372,7 +359,7 @@ public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string small ClassicAssert.AreEqual("OK", result.ToString()); // Verify that head address moved forward - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); Assert.That(info.HeadAddress, Is.GreaterThan(prevHead)); currMemorySize = TestUtils.GetEffectiveMemorySize(smallerSize, pageSize, out _); @@ -383,14 +370,11 @@ public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string small while (info.HeadAddress == prevHead) { var key = $"key{i++:00000}"; - if (storeType == StoreType.Main) - _ = db.StringSet(key, val); - else - _ = db.ListRightPush(key, [val]); + _ = db.StringSet(key, val); prevHead = info.HeadAddress; prevTail = info.TailAddress; - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); } // Verify that records were inserted up to the configured memory size limit @@ -408,14 +392,11 @@ public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string small while (info.HeadAddress == prevHead) { var key = $"key{i++:00000}"; - if (storeType == StoreType.Main) - _ = db.StringSet(key, val); - else - _ = db.ListRightPush(key, [val]); + _ = db.StringSet(key, val); prevHead = info.HeadAddress; prevTail = info.TailAddress; - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); } // Verify that memory is fully utilized @@ -428,15 +409,14 @@ public void ConfigSetMemorySizeUtilizationTest(StoreType storeType, string small /// The test fills the store to a larger capacity than the initial memory size, then verifies that recovering with the /// smaller initial memory size retains the last inserted keys in the expected initial capacity. /// - /// Store Type (Main / Object) /// Memory size larger than the initial size (within buffer bounds) [Test] - [TestCase(StoreType.Main, "4m")] - [TestCase(StoreType.Object, "4096")] - public void ConfigSetMemorySizeRecoveryTest(StoreType storeType, string largerSize) + [TestCase("4m")] + [TestCase("4096")] + public void ConfigSetMemorySizeRecoveryTest(string largerSize) { - var option = storeType == StoreType.Main ? "memory" : "obj-log-memory"; - var initMemorySize = storeType == StoreType.Main ? memorySize : objectStoreLogMemorySize; + var option = "memory"; + var initMemorySize = memorySize; var currMemorySize = TestUtils.GetEffectiveMemorySize(initMemorySize, pageSize, out var parsedPageSize); @@ -447,11 +427,11 @@ public void ConfigSetMemorySizeRecoveryTest(StoreType storeType, string largerSi { var db = redis.GetDatabase(0); var garnetServer = redis.GetServer(TestUtils.EndPoint); - var info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); - ClassicAssert.AreEqual(storeType == StoreType.Main ? 64 : 24, info.TailAddress); + var info = TestUtils.GetStoreAddressInfo(garnetServer); + ClassicAssert.AreEqual(64, info.TailAddress); var i = 0; - var val = new RedisValue(new string('x', storeType == StoreType.Main ? 512 - 32 : 1)); + var val = new RedisValue(new string('x', 512 - 32)); // Insert records until head address moves var prevHead = info.HeadAddress; @@ -459,14 +439,11 @@ public void ConfigSetMemorySizeRecoveryTest(StoreType storeType, string largerSi while (info.HeadAddress == prevHead) { var key = $"key{i++:00000}"; - if (storeType == StoreType.Main) - _ = db.StringSet(key, val); - else - _ = db.ListRightPush(key, [val]); + _ = db.StringSet(key, val); prevHead = info.HeadAddress; prevTail = info.TailAddress; - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); } var lastIdxFirstRound = i - 1; @@ -498,14 +475,11 @@ public void ConfigSetMemorySizeRecoveryTest(StoreType storeType, string largerSi while (info.HeadAddress == prevHead) { var key = $"key{i++:00000}"; - if (storeType == StoreType.Main) - _ = db.StringSet(key, val); - else - _ = db.ListRightPush(key, [val]); + _ = db.StringSet(key, val); prevHead = info.HeadAddress; prevTail = info.TailAddress; - info = TestUtils.GetStoreAddressInfo(garnetServer, isObjectStore: storeType == StoreType.Object); + info = TestUtils.GetStoreAddressInfo(garnetServer); } lastIdxSecondRound = i - 1; @@ -524,10 +498,7 @@ public void ConfigSetMemorySizeRecoveryTest(StoreType storeType, string largerSi memorySize: memorySize, indexSize: indexSize, pageSize: pageSize, - objectStorePageSize: pageSize, - objectStoreLogMemorySize: objectStoreLogMemorySize, - objectStoreIndexSize: objectStoreIndexSize, - objectStoreHeapMemorySize: objectStoreHeapMemorySize, + heapMemorySize: heapMemorySize, useReviv: useReviv, tryRecover: true); server.Start(); @@ -566,8 +537,7 @@ public class RespConfigIndexUtilizationTests GarnetServer server; private string memorySize = "3m"; private string indexSize = "512"; - private string objectStoreLogMemorySize = "16384"; - private string objectStoreHeapMemorySize = "16384"; + private string heapMemorySize = "16384"; private string pageSize = "1024"; private bool useReviv; @@ -584,10 +554,7 @@ public void Setup() memorySize: memorySize, indexSize: indexSize, pageSize: pageSize, - objectStorePageSize: pageSize, - objectStoreLogMemorySize: objectStoreLogMemorySize, - objectStoreIndexSize: indexSize, - objectStoreHeapMemorySize: objectStoreHeapMemorySize, + heapMemorySize: heapMemorySize, useReviv: useReviv); server.Start(); } @@ -682,8 +649,7 @@ public class RespConfigHeapUtilizationTests GarnetServer server; private string memorySize = "3m"; private string indexSize = "512"; - private string objectStoreLogMemorySize = "8192"; - private string objectStoreHeapMemorySize = "4096"; + private string heapMemorySize = "4096"; private string pageSize = "1024"; private bool useReviv; @@ -700,10 +666,7 @@ public void Setup() memorySize: memorySize, indexSize: indexSize, pageSize: pageSize, - objectStorePageSize: pageSize, - objectStoreLogMemorySize: objectStoreLogMemorySize, - objectStoreIndexSize: indexSize, - objectStoreHeapMemorySize: objectStoreHeapMemorySize, + heapMemorySize: heapMemorySize, useReviv: useReviv); server.Start(); } diff --git a/test/Garnet.test/RespEtagTests.cs b/test/Garnet.test/RespEtagTests.cs index 75276e5db2a..a4b434c146c 100644 --- a/test/Garnet.test/RespEtagTests.cs +++ b/test/Garnet.test/RespEtagTests.cs @@ -1363,7 +1363,7 @@ public void SingleDeleteWithObjectStoreDisabledForEtagSetData() TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); var key = "delKey"; @@ -1389,7 +1389,7 @@ public void SingleDeleteWithObjectStoreDisable_LTMForEtagSetData() TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true); server.Start(); using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); @@ -1430,7 +1430,7 @@ public void MultiKeyDeleteForEtagSetData([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } @@ -1465,7 +1465,7 @@ public void MultiKeyUnlinkForEtagSetData([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } @@ -1499,7 +1499,7 @@ public void SingleExistsForEtagSetData([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); diff --git a/test/Garnet.test/RespHashTests.cs b/test/Garnet.test/RespHashTests.cs index 139e84f9b1e..84821f77589 100644 --- a/test/Garnet.test/RespHashTests.cs +++ b/test/Garnet.test/RespHashTests.cs @@ -24,7 +24,7 @@ public class RespHashTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, enableObjectStoreReadCache: true, enableAOF: true, lowMemory: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, enableAOF: true, lowMemory: true); server.Start(); } @@ -1140,7 +1140,7 @@ public async Task CanDoHashExpireLTM() db.HashSet(key, [new HashEntry("Field1", "StringValue"), new HashEntry("Field2", "1")]); } - var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); // Ensure data has spilled to disk ClassicAssert.Greater(info.HeadAddress, info.BeginAddress); diff --git a/test/Garnet.test/RespModuleTests.cs b/test/Garnet.test/RespModuleTests.cs index 092325547d0..5a7e0b4ae37 100644 --- a/test/Garnet.test/RespModuleTests.cs +++ b/test/Garnet.test/RespModuleTests.cs @@ -384,7 +384,6 @@ public void TearDown() public void TestNoAllowedPathsForModuleLoading() { using var server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, - disableObjects: true, disablePubSub: true, enableModuleCommand: Garnet.server.Auth.Settings.ConnectionProtectionOption.Yes, extensionBinPaths: null, @@ -413,7 +412,6 @@ public void TestNoAllowedPathsForModuleLoading() public void TestModuleCommandNotEnabled() { using var server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, - disableObjects: true, disablePubSub: true, enableModuleCommand: Garnet.server.Auth.Settings.ConnectionProtectionOption.No, extensionBinPaths: [testModuleDir, binPath], diff --git a/test/Garnet.test/RespSlowLogTests.cs b/test/Garnet.test/RespSlowLogTests.cs index 430f1f78b33..59e5c7058ca 100644 --- a/test/Garnet.test/RespSlowLogTests.cs +++ b/test/Garnet.test/RespSlowLogTests.cs @@ -17,7 +17,7 @@ public class RespSlowLogTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disablePubSub: true, latencyMonitor: false, disableObjects: false, slowLogThreshold: slowLogThreshold); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disablePubSub: true, latencyMonitor: false, slowLogThreshold: slowLogThreshold); server.Start(); } diff --git a/test/Garnet.test/RespSortedSetTests.cs b/test/Garnet.test/RespSortedSetTests.cs index 598995aa62d..76195d62b40 100644 --- a/test/Garnet.test/RespSortedSetTests.cs +++ b/test/Garnet.test/RespSortedSetTests.cs @@ -78,7 +78,7 @@ public class RespSortedSetTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, enableObjectStoreReadCache: true, enableAOF: true, lowMemory: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableReadCache: true, enableAOF: true, lowMemory: true); server.Start(); } @@ -2180,7 +2180,7 @@ public async Task CanDoSortedSetExpireLTM() ]); } - var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true, isObjectStore: true); + var info = TestUtils.GetStoreAddressInfo(server, includeReadCache: true); // Ensure data has spilled to disk ClassicAssert.Greater(info.HeadAddress, info.BeginAddress); diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index dbefbfa3380..3bf07526fd4 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -1441,7 +1441,7 @@ public void SingleDeleteWithObjectStoreDisabled() TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); var key = "delKey"; @@ -1466,7 +1466,7 @@ public void SingleDeleteWithObjectStoreDisable_LTM() TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true); server.Start(); using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); @@ -1505,7 +1505,7 @@ public void GarnetObjectStoreDisabledError() { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); @@ -1537,7 +1537,7 @@ public void MultiKeyDelete([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } @@ -1605,7 +1605,7 @@ public void MultiKeyUnlink([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } @@ -1671,7 +1671,7 @@ public void SingleExists([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); @@ -1936,7 +1936,7 @@ public void SingleRenameKeyEdgeCase([Values] bool withoutObjectStore) { TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); } using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); @@ -3942,7 +3942,7 @@ public void AsyncTest1() // Set up low-memory database TearDown(); TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, disableObjects: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true); server.Start(); string firstKey = null, firstValue = null, lastKey = null, lastValue = null; diff --git a/test/Garnet.test/TestUtils.cs b/test/Garnet.test/TestUtils.cs index bfff3da0c0f..9170456fcc6 100644 --- a/test/Garnet.test/TestUtils.cs +++ b/test/Garnet.test/TestUtils.cs @@ -226,11 +226,9 @@ public static GarnetServer CreateGarnetServer( bool tryRecover = false, bool lowMemory = false, string memorySize = default, - string objectStoreLogMemorySize = default, string pageSize = default, bool enableAOF = false, bool enableTLS = false, - bool disableObjects = false, int metricsSamplingFreq = -1, bool latencyMonitor = false, int commitFrequencyMs = 0, @@ -239,11 +237,8 @@ public static GarnetServer CreateGarnetServer( string defaultPassword = null, bool useAcl = false, // NOTE: Temporary until ACL is enforced as default string aclFile = null, - string objectStorePageSize = default, - string objectStoreHeapMemorySize = default, - string objectStoreIndexSize = "16k", - string objectStoreIndexMaxSize = default, - string objectStoreReadCacheHeapMemorySize = default, + string heapMemorySize = default, + string readCacheHeapMemorySize = default, string indexSize = "1m", string indexMaxSize = default, string[] extensionBinPaths = null, @@ -255,7 +250,6 @@ public static GarnetServer CreateGarnetServer( ConnectionProtectionOption enableModuleCommand = ConnectionProtectionOption.No, bool enableLua = false, bool enableReadCache = false, - bool enableObjectStoreReadCache = false, ILogger logger = null, IEnumerable loadModulePaths = null, string pubSubPageSize = null, @@ -323,7 +317,6 @@ public static GarnetServer CreateGarnetServer( DisablePubSub = disablePubSub, Recover = tryRecover, IndexSize = indexSize, - ObjectStoreIndexSize = objectStoreIndexSize, EnableAOF = enableAOF, EnableLua = enableLua, CommitFrequencyMs = commitFrequencyMs, @@ -336,7 +329,6 @@ public static GarnetServer CreateGarnetServer( issuerCertificatePath: null, null, 0, false, null, logger: logger) : null, - DisableObjects = disableObjects, QuietMode = true, MetricsSamplingFrequency = metricsSamplingFreq, LatencyMonitor = latencyMonitor, @@ -354,7 +346,6 @@ public static GarnetServer CreateGarnetServer( EnableDebugCommand = enableDebugCommand, EnableModuleCommand = enableModuleCommand, EnableReadCache = enableReadCache, - EnableObjectStoreReadCache = enableObjectStoreReadCache, ReplicationOffsetMaxLag = asyncReplay ? -1 : 0, LuaOptions = enableLua ? new LuaOptions(luaMemoryMode, luaMemoryLimit, luaTimeout ?? Timeout.InfiniteTimeSpan, luaLoggingMode, luaAllowedFunctions ?? [], logger) : null, UnixSocketPath = unixSocketPath, @@ -366,42 +357,28 @@ public static GarnetServer CreateGarnetServer( if (!string.IsNullOrEmpty(memorySize)) opts.MemorySize = memorySize; - if (!string.IsNullOrEmpty(objectStoreLogMemorySize)) - opts.ObjectStoreLogMemorySize = objectStoreLogMemorySize; - if (!string.IsNullOrEmpty(pageSize)) opts.PageSize = pageSize; if (!string.IsNullOrEmpty(pubSubPageSize)) opts.PubSubPageSize = pubSubPageSize; - if (!string.IsNullOrEmpty(objectStorePageSize)) - opts.ObjectStorePageSize = objectStorePageSize; - - if (!string.IsNullOrEmpty(objectStoreHeapMemorySize)) - opts.ObjectStoreHeapMemorySize = objectStoreHeapMemorySize; + if (!string.IsNullOrEmpty(heapMemorySize)) + opts.HeapMemorySize = heapMemorySize; - if (!string.IsNullOrEmpty(objectStoreReadCacheHeapMemorySize)) - opts.ObjectStoreReadCacheHeapMemorySize = objectStoreReadCacheHeapMemorySize; + if (!string.IsNullOrEmpty(readCacheHeapMemorySize)) + opts.ReadCacheHeapMemorySize = readCacheHeapMemorySize; if (indexMaxSize != default) opts.IndexMaxSize = indexMaxSize; - if (objectStoreIndexMaxSize != default) opts.ObjectStoreIndexMaxSize = objectStoreIndexMaxSize; - if (lowMemory) { - opts.MemorySize = opts.ObjectStoreLogMemorySize = memorySize == default ? "1024" : memorySize; - opts.PageSize = opts.ObjectStorePageSize = pageSize == default ? "512" : pageSize; + opts.MemorySize = memorySize == default ? "1024" : memorySize; + opts.PageSize = pageSize == default ? "512" : pageSize; if (enableReadCache) { opts.ReadCacheMemorySize = opts.MemorySize; opts.ReadCachePageSize = opts.PageSize; } - - if (enableObjectStoreReadCache) - { - opts.ObjectStoreReadCacheLogMemorySize = opts.MemorySize; - opts.ObjectStoreReadCachePageSize = opts.PageSize; - } } ILoggerFactory loggerFactory = null; @@ -432,7 +409,6 @@ public static GarnetServer CreateGarnetServer( opts.RevivInChainOnly = false; opts.RevivBinRecordCounts = []; opts.RevivBinRecordSizes = []; - opts.RevivObjBinRecordCount = 256; } if (useInChainRevivOnly) @@ -468,7 +444,6 @@ public static (GarnetServer[] Nodes, GarnetServerOptions[] Options) CreateGarnet EndPointCollection endpoints, bool enableCluster = true, bool disablePubSub = false, - bool disableObjects = false, bool tryRecover = false, bool enableAOF = false, int timeout = -1, @@ -531,7 +506,6 @@ public static (GarnetServer[] Nodes, GarnetServerOptions[] Options) CreateGarnet endpoint, enableCluster: enableCluster, disablePubSub, - disableObjects, tryRecover, enableAOF, timeout, @@ -603,7 +577,6 @@ public static GarnetServerOptions GetGarnetServerOptions( EndPoint endpoint, bool enableCluster = true, bool disablePubSub = false, - bool disableObjects = false, bool tryRecover = false, bool enableAOF = false, int timeout = -1, @@ -702,18 +675,15 @@ public static GarnetServerOptions GetGarnetServerOptions( { ThreadPoolMinThreads = 100, SegmentSize = segmentSize, - ObjectStoreSegmentSize = segmentSize, EnableStorageTier = useAzureStorage || (!disableStorageTier && logDir != null), LogDir = disableStorageTier ? null : logDir, CheckpointDir = checkpointDir, EndPoints = [endpoint], DisablePubSub = disablePubSub, - DisableObjects = disableObjects, EnableDebugCommand = ConnectionProtectionOption.Yes, EnableModuleCommand = ConnectionProtectionOption.Yes, Recover = tryRecover, IndexSize = "1m", - ObjectStoreIndexSize = "16k", EnableCluster = enableCluster, CleanClusterConfig = cleanClusterConfig, ClusterTimeout = timeout, @@ -779,8 +749,8 @@ public static GarnetServerOptions GetGarnetServerOptions( if (lowMemory) { - opts.MemorySize = opts.ObjectStoreLogMemorySize = memorySize == default ? "1024" : memorySize; - opts.PageSize = opts.ObjectStorePageSize = pageSize == default ? "512" : pageSize; + opts.MemorySize = memorySize == default ? "1024" : memorySize; + opts.PageSize = pageSize == default ? "512" : pageSize; } return opts; @@ -1104,10 +1074,10 @@ 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, bool includeReadCache = false) { StoreAddressInfo result = default; - var info = isObjectStore ? server.Info("OBJECTSTORE") : server.Info("STORE"); + var info = server.Info("STORE"); foreach (var section in info) { foreach (var entry in section) From 2c709f14116a540a5cb7d691ac87ac1606eec50a Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Wed, 17 Sep 2025 15:37:34 -0700 Subject: [PATCH 04/28] Added unified store session --- libs/server/AOF/AofEntryType.cs | 13 ++ libs/server/AOF/AofProcessor.cs | 14 +- libs/server/InputHeader.cs | 121 ++++++++++++++++++ .../Objects/Types/GarnetObjectStoreOutput.cs | 21 ++- .../Functions/UnifiedStore/CallbackMethods.cs | 23 ++++ .../Functions/UnifiedStore/DeleteMethods.cs | 44 +++++++ .../Functions/UnifiedStore/PrivateMethods.cs | 27 ++++ .../Functions/UnifiedStore/RMWMethods.cs | 42 ++++++ .../Functions/UnifiedStore/ReadMethods.cs | 18 +++ .../UnifiedStore/UnifiedStoreFunctions.cs | 29 +++++ .../Functions/UnifiedStore/UpsertMethods.cs | 51 ++++++++ .../UnifiedStore/VarLenInputMethods.cs | 27 ++++ libs/server/Storage/Session/StorageSession.cs | 12 ++ 13 files changed, 434 insertions(+), 8 deletions(-) create mode 100644 libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs create mode 100644 libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs diff --git a/libs/server/AOF/AofEntryType.cs b/libs/server/AOF/AofEntryType.cs index bccbdd6af07..315fc765bfe 100644 --- a/libs/server/AOF/AofEntryType.cs +++ b/libs/server/AOF/AofEntryType.cs @@ -79,6 +79,19 @@ public enum AofEntryType : byte /// FlushDb = 0x61, + /// + /// Unified store upsert + /// + UnifiedStoreUpsert = 0x70, + /// + /// Unified store RMW + /// + UnifiedStoreRMW = 0x71, + /// + /// Unified store delete + /// + UnifiedStoreDelete = 0x72, + #region Deprecated markers /// /// Deprecated with unified checkpointing: Checkpoint for object store start diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index b48e0e853b8..b01d259765c 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -46,6 +46,11 @@ public sealed unsafe partial class AofProcessor /// BasicContext objectStoreBasicContext; + /// + /// Session for unified store + /// + BasicContext unifiedStoreBasicContext; + readonly Dictionary> inflightTxns; readonly byte[] buffer; readonly GCHandle handle; @@ -96,6 +101,7 @@ public void Dispose() { dbSession.StorageSession.basicContext.Session?.Dispose(); dbSession.StorageSession.objectStoreBasicContext.Session?.Dispose(); + dbSession.StorageSession.unifiedStoreBasicContext.Session?.Dispose(); } handle.Free(); @@ -385,11 +391,9 @@ private void SwitchActiveDatabaseContext(GarnetDatabase db, bool initialSetup = // Switch the storage context to match the session, if necessary 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; + basicContext = respServerSession.storageSession.basicContext.Session.BasicContext; + objectStoreBasicContext = respServerSession.storageSession.objectStoreBasicContext.Session.BasicContext; + unifiedStoreBasicContext = respServerSession.storageSession.unifiedStoreBasicContext.Session.BasicContext; this.activeDbId = db.Id; } } diff --git a/libs/server/InputHeader.cs b/libs/server/InputHeader.cs index 81596288cb2..6cc7ae39e1a 100644 --- a/libs/server/InputHeader.cs +++ b/libs/server/InputHeader.cs @@ -427,6 +427,127 @@ public unsafe int DeserializeFrom(byte* src) } } + /// + /// Header for Garnet Main Store inputs + /// + public struct UnifiedStoreInput : IStoreInput + { + /// + /// Common input header for Garnet + /// + public RespInputHeader header; + + /// + /// Argument for generic usage by command implementation + /// + public long arg1; + + /// + /// Session parse state + /// + public SessionParseState parseState; + + /// + /// Create a new instance of UnifiedStoreInput + /// + /// Command + /// Flags + /// General-purpose argument + public UnifiedStoreInput(RespCommand cmd, RespInputFlags flags = 0, long arg1 = 0) + { + this.header = new RespInputHeader(cmd, flags); + this.arg1 = arg1; + } + + /// + /// Create a new instance of UnifiedStoreInput + /// + /// Command + /// Flags + /// General-purpose argument + public UnifiedStoreInput(ushort cmd, byte flags = 0, long arg1 = 0) : + this((RespCommand)cmd, (RespInputFlags)flags, arg1) + + { + } + + /// + /// Create a new instance of UnifiedStoreInput + /// + /// Command + /// Parse state + /// General-purpose argument + /// Flags + public UnifiedStoreInput(RespCommand cmd, ref SessionParseState parseState, long arg1 = 0, RespInputFlags flags = 0) : this(cmd, flags, arg1) + { + this.parseState = parseState; + } + + /// + /// Create a new instance of UnifiedStoreInput + /// + /// Command + /// Parse state + /// First command argument index in parse state + /// General-purpose argument + /// Flags + public UnifiedStoreInput(RespCommand cmd, ref SessionParseState parseState, int startIdx, long arg1 = 0, RespInputFlags flags = 0) : this(cmd, flags, arg1) + { + this.parseState = parseState.Slice(startIdx); + } + + /// + public int SerializedLength => header.SpanByte.TotalSize + + sizeof(long) // arg1 + + parseState.GetSerializedLength(); + + /// + public unsafe int CopyTo(byte* dest, int length) + { + Debug.Assert(length >= this.SerializedLength); + + var curr = dest; + + // Serialize header + header.SpanByte.SerializeTo(curr); + curr += header.SpanByte.TotalSize; + + // Serialize arg1 + *(long*)curr = arg1; + curr += sizeof(long); + + // Serialize parse state + var remainingLength = length - (int)(curr - dest); + var len = parseState.SerializeTo(curr, remainingLength); + curr += len; + + // Serialize length + return (int)(curr - dest); + } + + /// + public unsafe int DeserializeFrom(byte* src) + { + var curr = src; + + // Deserialize header + var header = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + ref var h = ref Unsafe.AsRef(header.ToPointer()); + curr += header.TotalSize; + this.header = h; + + // Deserialize arg1 + arg1 = *(long*)curr; + curr += sizeof(long); + + // Deserialize parse state + var len = parseState.DeserializeFrom(curr); + curr += len; + + return (int)(curr - src); + } + } + /// /// Header for Garnet CustomProcedure inputs /// diff --git a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs index 0547c6042b4..c7da0575ad2 100644 --- a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs +++ b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs @@ -58,16 +58,18 @@ public struct GarnetObjectStoreOutput /// /// True if output flag WrongType is set /// - public readonly bool HasWrongType => (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType; + public readonly bool HasWrongType => + (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType; /// /// True if output flag RemoveKey is set /// - public readonly bool HasRemoveKey => (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey; + public readonly bool HasRemoveKey => + (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey; public GarnetObjectStoreOutput() => SpanByteAndMemory = new(null); - public GarnetObjectStoreOutput(SpanByteAndMemory spam) => SpanByteAndMemory = spam; + public GarnetObjectStoreOutput(SpanByteAndMemory span) => SpanByteAndMemory = span; public static unsafe GarnetObjectStoreOutput FromPinnedPointer(byte* pointer, int length) => new(new SpanByteAndMemory() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length) }); @@ -77,4 +79,17 @@ public void ConvertToHeap() // Does not convert to heap when going pending, because we immediately complete pending operations for object store. } } + + /// + /// Output type used by Garnet unified store. + /// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller). + /// That is in order to justify transferring data in this struct through the Tsavorite storage layer. + /// + public struct GarnetUnifiedStoreOutput + { + /// + /// Some result of operation (e.g., number of items added successfully) + /// + public long result1; + } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs b/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs new file mode 100644 index 00000000000..6873ef68670 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) + { + } + + public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) + { + } + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs new file mode 100644 index 00000000000..b40c2dd6d2f --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + { + logRecord.InfoRef.ClearHasETag(); + functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); + return true; + } + + public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + { + if (functionsState.appendOnlyFile != null) + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); + } + + public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + { + logRecord.ClearOptionals(); + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); + + if (logRecord.Info.ValueIsObject) + { + // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); + logRecord.ClearValueObject(obj => { }); + } + + return true; + } + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs new file mode 100644 index 00000000000..a579790ff38 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + /// + /// Logging Delete from + /// a. InPlaceDeleter + /// b. PostInitialDeleter + /// + void WriteLogDelete(ReadOnlySpan key, long version, int sessionID) + { + if (functionsState.StoredProcMode) + return; + + functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.UnifiedStoreDelete, storeVersion = version, sessionID = sessionID }, key, item2: default, out _); + } + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs new file mode 100644 index 00000000000..8daf2e6dc9d --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public bool NeedInitialUpdate(ReadOnlySpan key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, + ref RMWInfo rmwInfo) => + throw new NotImplementedException(); + + public bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => + throw new NotImplementedException(); + + public void PostInitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => + throw new NotImplementedException(); + + public bool NeedCopyUpdate(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => + throw new NotImplementedException(); + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs new file mode 100644 index 00000000000..e329e78c5c1 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs @@ -0,0 +1,18 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public bool Reader(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs b/libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs new file mode 100644 index 00000000000..d45faf69447 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + readonly FunctionsState functionsState; + + /// + /// Constructor + /// + internal UnifiedSessionFunctions(FunctionsState functionsState) + { + this.functionsState = functionsState; + } + + public void ConvertOutputToHeap(ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + { + // TODO: Inspect input to determine whether we're in a context requiring ConvertToHeap. + //output.ConvertToHeap(); + } + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs new file mode 100644 index 00000000000..616d9f7b394 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, + ref UnifiedStoreInput input, in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, + ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + ReadOnlySpan newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + IHeapObject newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => + throw new NotImplementedException(); + + public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, + in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + } +} diff --git a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs new file mode 100644 index 00000000000..a6782242342 --- /dev/null +++ b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Unified store functions + /// + public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions + { + public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + + public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref UnifiedStoreInput input) => throw new NotImplementedException(); + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref UnifiedStoreInput input) => throw new NotImplementedException(); + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref UnifiedStoreInput input) => throw new NotImplementedException(); + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, + ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord => + throw new NotImplementedException(); + } +} diff --git a/libs/server/Storage/Session/StorageSession.cs b/libs/server/Storage/Session/StorageSession.cs index e3df4bfc904..c13c0d98a02 100644 --- a/libs/server/Storage/Session/StorageSession.cs +++ b/libs/server/Storage/Session/StorageSession.cs @@ -39,6 +39,12 @@ sealed partial class StorageSession : IDisposable public BasicContext objectStoreBasicContext; public TransactionalContext objectStoreTransactionalContext; + /// + /// Session Contexts for unified store + /// + public BasicContext unifiedStoreBasicContext; + public TransactionalContext unifiedStoreTransactionalContext; + public readonly ScratchBufferBuilder scratchBufferBuilder; public readonly FunctionsState functionsState; @@ -80,10 +86,15 @@ public StorageSession(StoreWrapper storeWrapper, var objectStoreFunctions = new ObjectSessionFunctions(functionsState); var objectStoreSession = db.Store.NewSession(objectStoreFunctions); + var unifiedStoreFunctions = new UnifiedSessionFunctions(functionsState); + var unifiedStoreSession = db.Store.NewSession(unifiedStoreFunctions); + basicContext = session.BasicContext; transactionalContext = session.TransactionalContext; objectStoreBasicContext = objectStoreSession.BasicContext; objectStoreTransactionalContext = objectStoreSession.TransactionalContext; + unifiedStoreBasicContext = unifiedStoreSession.BasicContext; + unifiedStoreTransactionalContext = unifiedStoreSession.TransactionalContext; HeadAddress = db.Store.Log.HeadAddress; ObjectScanCountLimit = storeWrapper.serverOptions.ObjectScanCountLimit; @@ -102,6 +113,7 @@ public void Dispose() sectorAlignedMemoryBitmap?.Dispose(); basicContext.Session.Dispose(); objectStoreBasicContext.Session?.Dispose(); + unifiedStoreBasicContext.Session?.Dispose(); sectorAlignedMemoryHll1?.Dispose(); sectorAlignedMemoryHll2?.Dispose(); } From 671e708bb869002a1f45050cf13793502cddaa5a Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 18 Sep 2025 11:24:17 -0700 Subject: [PATCH 05/28] Correcting generic typing --- .../cluster/Server/ClusterManagerSlotState.cs | 8 +++---- libs/cluster/Server/ClusterProvider.cs | 8 +++---- libs/cluster/Session/ClusterSession.cs | 8 +++---- .../Session/RespClusterMigrateCommands.cs | 10 ++++---- libs/host/GarnetServer.cs | 4 ++-- libs/server/AOF/AofProcessor.cs | 4 ++-- libs/server/API/GarnetApi.cs | 4 ++-- libs/server/API/GarnetApiObjectCommands.cs | 4 ++-- libs/server/Cluster/IClusterProvider.cs | 8 +++---- libs/server/Databases/DatabaseManagerBase.cs | 4 ++-- libs/server/Databases/IDatabaseManager.cs | 4 ++-- libs/server/GarnetDatabase.cs | 4 ++-- libs/server/Providers/GarnetProvider.cs | 6 ++--- libs/server/Resp/GarnetDatabaseSession.cs | 16 ++++++------- libs/server/Resp/LocalServerSession.cs | 8 +++---- libs/server/Resp/RespServerSession.cs | 16 ++++++------- .../Storage/Session/MainStore/AdvancedOps.cs | 14 +++++------ .../Storage/Session/MainStore/BitmapOps.cs | 24 +++++++++---------- .../Session/MainStore/CompletePending.cs | 6 ++--- .../Session/MainStore/HyperLogLogOps.cs | 12 +++++----- .../Storage/Session/MainStore/MainStoreOps.cs | 6 ++--- .../Session/ObjectStore/AdvancedOps.cs | 4 ++-- .../Storage/Session/ObjectStore/Common.cs | 4 ++-- .../Session/ObjectStore/CompletePending.cs | 4 ++-- .../Storage/Session/ObjectStore/HashOps.cs | 4 ++-- .../Storage/Session/ObjectStore/ListOps.cs | 4 ++-- .../Storage/Session/ObjectStore/SetOps.cs | 4 ++-- .../Session/ObjectStore/SortedSetGeoOps.cs | 4 ++-- .../Session/ObjectStore/SortedSetOps.cs | 4 ++-- libs/server/Storage/Session/StorageSession.cs | 4 ++-- .../Storage/SizeTracker/CacheSizeTracker.cs | 4 ++-- libs/server/StoreWrapper.cs | 4 ++-- libs/server/Transaction/TransactionManager.cs | 20 ++++++++-------- libs/server/Transaction/TxnKeyEntry.cs | 4 ++-- .../Transaction/TxnKeyEntryComparison.cs | 4 ++-- .../Index/StoreFunctions/StoreFunctions.cs | 12 +++++----- test/Garnet.test/CacheSizeTrackerTests.cs | 4 ++-- test/Garnet.test/GarnetObjectTests.cs | 4 ++-- test/Garnet.test/RespConfigTests.cs | 4 ++-- test/Garnet.test/RespSortedSetTests.cs | 8 +++---- 40 files changed, 141 insertions(+), 141 deletions(-) diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs index 27356859b80..554891de264 100644 --- a/libs/cluster/Server/ClusterManagerSlotState.cs +++ b/libs/cluster/Server/ClusterManagerSlotState.cs @@ -13,11 +13,11 @@ namespace Garnet.cluster { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Cluster manager diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs index 8b36bed74cb..73d7a5b3d57 100644 --- a/libs/cluster/Server/ClusterProvider.cs +++ b/libs/cluster/Server/ClusterProvider.cs @@ -16,11 +16,11 @@ namespace Garnet.cluster { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Cluster provider diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs index 83e78f8e051..cf019380c84 100644 --- a/libs/cluster/Session/ClusterSession.cs +++ b/libs/cluster/Session/ClusterSession.cs @@ -13,11 +13,11 @@ namespace Garnet.cluster { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession { diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index d11898350c4..9280af7a1cd 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -13,11 +13,11 @@ namespace Garnet.cluster { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext, - SpanByteAllocator>>>; + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, + BasicContext, + ObjectAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession { diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index cdbdda8c7f5..380f85dd744 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -20,8 +20,8 @@ namespace Garnet { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Implementation Garnet server diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index b48e0e853b8..595c3eed414 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -13,8 +13,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index 7ba856482a0..daa7f3257af 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; // See TransactionManager.cs for aliases BasicGarnetApi and TransactionalGarnetApi diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs index f577c1a1cd3..e0c68d634a0 100644 --- a/libs/server/API/GarnetApiObjectCommands.cs +++ b/libs/server/API/GarnetApiObjectCommands.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Garnet API implementation diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs index ea3472df62b..b7e1d79b5c6 100644 --- a/libs/server/Cluster/IClusterProvider.cs +++ b/libs/server/Cluster/IClusterProvider.cs @@ -13,11 +13,11 @@ namespace Garnet.server { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Cluster provider diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index 0b8b81433ec..d1f63d21e15 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Base class for logical database management diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs index 5b658113c0e..d5e6c4aa5d4 100644 --- a/libs/server/Databases/IDatabaseManager.cs +++ b/libs/server/Databases/IDatabaseManager.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Interface for logical database management diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs index 502dee4660f..31b5eccd604 100644 --- a/libs/server/GarnetDatabase.cs +++ b/libs/server/GarnetDatabase.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Represents a logical database in Garnet diff --git a/libs/server/Providers/GarnetProvider.cs b/libs/server/Providers/GarnetProvider.cs index e8b53202805..b72d4160f01 100644 --- a/libs/server/Providers/GarnetProvider.cs +++ b/libs/server/Providers/GarnetProvider.cs @@ -8,13 +8,13 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Session provider for Garnet /// - public sealed class GarnetProvider : TsavoriteKVProviderBase + public sealed class GarnetProvider : TsavoriteKVProviderBase { readonly StoreWrapper storeWrapper; diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs index c8f4755e901..04a94e3bc5c 100644 --- a/libs/server/Resp/GarnetDatabaseSession.cs +++ b/libs/server/Resp/GarnetDatabaseSession.cs @@ -4,17 +4,17 @@ namespace Garnet.server { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; using TransactionalGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, TransactionalContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Represents a logical database session in Garnet diff --git a/libs/server/Resp/LocalServerSession.cs b/libs/server/Resp/LocalServerSession.cs index 2b58da8751b..4ce3d3950b9 100644 --- a/libs/server/Resp/LocalServerSession.cs +++ b/libs/server/Resp/LocalServerSession.cs @@ -8,11 +8,11 @@ namespace Garnet.server { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Local server session diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 8bbd43ee9f3..eecc0925586 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -21,17 +21,17 @@ namespace Garnet.server { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; using TransactionalGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, TransactionalContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// RESP server session diff --git a/libs/server/Storage/Session/MainStore/AdvancedOps.cs b/libs/server/Storage/Session/MainStore/AdvancedOps.cs index 8782034934d..78c2c38bb45 100644 --- a/libs/server/Storage/Session/MainStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/MainStore/AdvancedOps.cs @@ -8,13 +8,13 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { public GarnetStatus GET_WithPending(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.Read(key, ref input, ref output, ctx); @@ -39,7 +39,7 @@ public GarnetStatus GET_WithPending(ReadOnlySpan key, ref RawStr } public bool GET_CompletePending((GarnetStatus, SpanByteAndMemory)[] outputArr, bool wait, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { Debug.Assert(outputArr != null); @@ -63,7 +63,7 @@ public bool GET_CompletePending((GarnetStatus, SpanByteAndMemory)[] ou } public bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { latencyMetrics?.Start(LatencyMetricsType.PENDING_LAT); var ret = context.CompletePendingWithOutputs(out completedOutputs, wait); @@ -72,7 +72,7 @@ public bool GET_CompletePending(out CompletedOutputIterator(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.RMW(key, ref input, ref output); @@ -86,7 +86,7 @@ public GarnetStatus RMW_MainStore(ReadOnlySpan key, ref RawStrin } public GarnetStatus Read_MainStore(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = context.Read(key, ref input, ref output); diff --git a/libs/server/Storage/Session/MainStore/BitmapOps.cs b/libs/server/Storage/Session/MainStore/BitmapOps.cs index b2223b4b485..23d73fec7ad 100644 --- a/libs/server/Storage/Session/MainStore/BitmapOps.cs +++ b/libs/server/Storage/Session/MainStore/BitmapOps.cs @@ -10,13 +10,13 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { public unsafe GarnetStatus StringSetBit(PinnedSpanByte key, PinnedSpanByte offset, bool bit, out bool previous, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { previous = false; @@ -38,7 +38,7 @@ public unsafe GarnetStatus StringSetBit(PinnedSpanByte key, PinnedSpan } public unsafe GarnetStatus StringGetBit(PinnedSpanByte key, PinnedSpanByte offset, out bool bValue, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { bValue = false; @@ -200,7 +200,7 @@ public GarnetStatus StringBitOperation(BitmapOperation bitOp, PinnedSpanByte des } public unsafe GarnetStatus StringBitCount(PinnedSpanByte key, long start, long end, bool useBitInterval, out long result, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { result = 0; @@ -265,7 +265,7 @@ public unsafe GarnetStatus StringBitCount(PinnedSpanByte key, long sta } public unsafe GarnetStatus StringBitField(PinnedSpanByte key, List commandArguments, out List result, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.BITFIELD); @@ -386,23 +386,23 @@ public unsafe GarnetStatus StringBitField(PinnedSpanByte key, List(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); public GarnetStatus StringGetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); public unsafe GarnetStatus StringBitCount(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); public unsafe GarnetStatus StringBitPosition(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); public unsafe GarnetStatus StringBitField(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { GarnetStatus status; if (secondaryCommand == RespCommand.GET) @@ -416,7 +416,7 @@ public unsafe GarnetStatus StringBitField(PinnedSpanByte key, ref RawS } public unsafe GarnetStatus StringBitFieldReadOnly(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var status = GarnetStatus.NOTFOUND; diff --git a/libs/server/Storage/Session/MainStore/CompletePending.cs b/libs/server/Storage/Session/MainStore/CompletePending.cs index 59625f713cf..7338b643a11 100644 --- a/libs/server/Storage/Session/MainStore/CompletePending.cs +++ b/libs/server/Storage/Session/MainStore/CompletePending.cs @@ -6,8 +6,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession { @@ -18,7 +18,7 @@ sealed partial class StorageSession /// /// static void CompletePendingForSession(ref Status status, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { context.CompletePendingWithOutputs(out var completedOutputs, wait: true); var more = completedOutputs.Next(); diff --git a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs index 2568d074ede..dedce5df223 100644 --- a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs +++ b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -16,7 +16,7 @@ sealed partial class StorageSession : IDisposable /// Adds all the element arguments to the HyperLogLog data structure stored at the variable name specified as key. /// public unsafe GarnetStatus HyperLogLogAdd(PinnedSpanByte key, string[] elements, out bool updated, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { updated = false; @@ -60,11 +60,11 @@ public unsafe GarnetStatus HyperLogLogAdd(PinnedSpanByte key, string[] /// /// public GarnetStatus HyperLogLogAdd(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext => RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); public unsafe GarnetStatus HyperLogLogLength(Span keys, out long count, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { parseState.Initialize(keys.Length); for (var i = 0; i < keys.Length; i++) @@ -87,7 +87,7 @@ public unsafe GarnetStatus HyperLogLogLength(Span keys /// /// public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, out long count, out bool error, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { error = false; count = default; diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 0b2f14f7515..22dafef763b 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -1162,7 +1162,7 @@ public GarnetStatus MemoryUsageForKey(PinnedSpanByte k var status = GET(key, out PinnedSpanByte keyValue, ref context); if (status == GarnetStatus.NOTFOUND) - { + { status = GET(key, out GarnetObjectStoreOutput objectValue, ref objectContext); if (status != GarnetStatus.NOTFOUND) { diff --git a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs index 1504908d1bb..12eb92e1d26 100644 --- a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs @@ -6,8 +6,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { diff --git a/libs/server/Storage/Session/ObjectStore/Common.cs b/libs/server/Storage/Session/ObjectStore/Common.cs index 9c739070741..da3dbc13379 100644 --- a/libs/server/Storage/Session/ObjectStore/Common.cs +++ b/libs/server/Storage/Session/ObjectStore/Common.cs @@ -12,8 +12,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { diff --git a/libs/server/Storage/Session/ObjectStore/CompletePending.cs b/libs/server/Storage/Session/ObjectStore/CompletePending.cs index a850e3dab0b..ec88b96052e 100644 --- a/libs/server/Storage/Session/ObjectStore/CompletePending.cs +++ b/libs/server/Storage/Session/ObjectStore/CompletePending.cs @@ -6,8 +6,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession { diff --git a/libs/server/Storage/Session/ObjectStore/HashOps.cs b/libs/server/Storage/Session/ObjectStore/HashOps.cs index b9c8c08c154..496d2b17740 100644 --- a/libs/server/Storage/Session/ObjectStore/HashOps.cs +++ b/libs/server/Storage/Session/ObjectStore/HashOps.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Server API methods - HASH diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index 18281d09ee3..fe0aa3de127 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -7,8 +7,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index 72a2f934a7d..eb246b5b939 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Server session for RESP protocol - SET diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index 8860bd84b6a..2c2da0f5797 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index 035e287edc5..42a53093a6b 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -12,8 +12,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { diff --git a/libs/server/Storage/Session/StorageSession.cs b/libs/server/Storage/Session/StorageSession.cs index e3df4bfc904..cda559fef06 100644 --- a/libs/server/Storage/Session/StorageSession.cs +++ b/libs/server/Storage/Session/StorageSession.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Storage Session - the internal layer that Garnet uses to perform storage operations diff --git a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs index a7afa5cb2d3..2e3e91e34a8 100644 --- a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs +++ b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Tracks the size of the main log and read cache. diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index b173c7b82d7..cd824e16b87 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -19,8 +19,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index dec6afd16f2..f12ff791407 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -11,19 +11,19 @@ namespace Garnet.server { using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; using TransactionalGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, TransactionalContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; /// /// Transaction manager diff --git a/libs/server/Transaction/TxnKeyEntry.cs b/libs/server/Transaction/TxnKeyEntry.cs index 3b5f0d80cdd..ca384a8b97a 100644 --- a/libs/server/Transaction/TxnKeyEntry.cs +++ b/libs/server/Transaction/TxnKeyEntry.cs @@ -8,8 +8,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Entry for a key to lock and unlock in transactions diff --git a/libs/server/Transaction/TxnKeyEntryComparison.cs b/libs/server/Transaction/TxnKeyEntryComparison.cs index 6e87e610460..cd83979d773 100644 --- a/libs/server/Transaction/TxnKeyEntryComparison.cs +++ b/libs/server/Transaction/TxnKeyEntryComparison.cs @@ -6,8 +6,8 @@ namespace Garnet.server { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; internal sealed class TxnKeyComparison { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs index f32202e7b74..9c3a58c5e21 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs @@ -103,9 +103,9 @@ public static StoreFunctions Create /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. /// - public static StoreFunctions Create(TKeyComparer keyComparer, Func> valueSerializerCreator) + public static StoreFunctions Create(TKeyComparer keyComparer, Func> valueSerializerCreator) where TKeyComparer : IKeyComparer - => new(keyComparer, valueSerializerCreator, new SpanByteRecordDisposer()); + => new(keyComparer, valueSerializerCreator, new DefaultRecordDisposer()); /// /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. @@ -118,14 +118,14 @@ public static StoreFunctions Create /// Store functions that take only the /// - public static StoreFunctions Create(TKeyComparer keyComparer) + public static StoreFunctions Create(TKeyComparer keyComparer) where TKeyComparer : IKeyComparer - => new(keyComparer, valueSerializerCreator: null, SpanByteRecordDisposer.Instance); + => new(keyComparer, valueSerializerCreator: null, DefaultRecordDisposer.Instance); /// /// Store functions for Key and Value /// - public static StoreFunctions Create() - => new(SpanByteComparer.Instance, valueSerializerCreator: null, SpanByteRecordDisposer.Instance); + public static StoreFunctions Create() + => new(SpanByteComparer.Instance, valueSerializerCreator: null, DefaultRecordDisposer.Instance); } } \ No newline at end of file diff --git a/test/Garnet.test/CacheSizeTrackerTests.cs b/test/Garnet.test/CacheSizeTrackerTests.cs index 30937ba6ba9..5ce9b3a7849 100644 --- a/test/Garnet.test/CacheSizeTrackerTests.cs +++ b/test/Garnet.test/CacheSizeTrackerTests.cs @@ -11,8 +11,8 @@ namespace Garnet.test { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; [TestFixture] public class CacheSizeTrackerTests diff --git a/test/Garnet.test/GarnetObjectTests.cs b/test/Garnet.test/GarnetObjectTests.cs index 6254c8ebfcd..8e365f1b0a0 100644 --- a/test/Garnet.test/GarnetObjectTests.cs +++ b/test/Garnet.test/GarnetObjectTests.cs @@ -10,8 +10,8 @@ namespace Garnet.test { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; [TestFixture] public class GarnetObjectTests diff --git a/test/Garnet.test/RespConfigTests.cs b/test/Garnet.test/RespConfigTests.cs index bdaf0edeecc..2a2e61adc91 100644 --- a/test/Garnet.test/RespConfigTests.cs +++ b/test/Garnet.test/RespConfigTests.cs @@ -13,8 +13,8 @@ namespace Garnet.test { - using StoreAllocator = SpanByteAllocator>; - using StoreFunctions = StoreFunctions; + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; /// /// Test dynamically changing server configuration using CONFIG SET command. diff --git a/test/Garnet.test/RespSortedSetTests.cs b/test/Garnet.test/RespSortedSetTests.cs index 76195d62b40..c4cbec2702d 100644 --- a/test/Garnet.test/RespSortedSetTests.cs +++ b/test/Garnet.test/RespSortedSetTests.cs @@ -20,11 +20,11 @@ namespace Garnet.test { using TestBasicGarnetApi = GarnetApi, - SpanByteAllocator>>, + /* MainStoreFunctions */ StoreFunctions, + ObjectAllocator>>, BasicContext, - SpanByteAllocator>>>; + /* ObjectStoreFunctions */ StoreFunctions, + ObjectAllocator>>>; [TestFixture] public class RespSortedSetTests From 38035abf472f6112c0d71349c3e1b834f55b566c Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 18 Sep 2025 13:31:17 -0700 Subject: [PATCH 06/28] Added MEMORY USAGE + TYPE to unified ops --- .../cluster/Server/ClusterManagerSlotState.cs | 3 + libs/cluster/Server/ClusterProvider.cs | 3 + libs/cluster/Session/ClusterSession.cs | 3 + .../Session/RespClusterMigrateCommands.cs | 3 + libs/common/RespMemoryWriter.cs | 11 +++ libs/server/API/GarnetApi.cs | 23 ++---- libs/server/API/GarnetApiObjectCommands.cs | 3 +- libs/server/API/GarnetApiUnifiedCommands.cs | 35 +++++++++ libs/server/API/IGarnetApi.cs | 11 +-- libs/server/Cluster/IClusterProvider.cs | 3 + libs/server/Custom/CustomObjectBase.cs | 2 +- libs/server/GarnetUnifiedStoreOutput.cs | 54 ++++++++++++++ libs/server/InputHeader.cs | 18 ----- libs/server/Objects/Hash/HashObject.cs | 4 +- libs/server/Objects/List/ListObject.cs | 4 +- libs/server/Objects/Set/SetObject.cs | 4 +- .../Objects/SortedSet/SortedSetObject.cs | 4 +- .../Objects/Types/GarnetObjectStoreOutput.cs | 42 +---------- libs/server/OutputHeader.cs | 66 +++++++++++++++++ libs/server/Resp/ArrayCommands.cs | 14 +++- libs/server/Resp/BasicCommands.cs | 10 ++- libs/server/Resp/CmdStrings.cs | 1 + libs/server/Resp/GarnetDatabaseSession.cs | 6 ++ libs/server/Resp/LocalServerSession.cs | 5 +- libs/server/Resp/RespServerSession.cs | 13 +++- .../Functions/ObjectStore/RMWMethods.cs | 4 +- .../Functions/ObjectStore/ReadMethods.cs | 2 +- .../Functions/UnifiedStore/ReadMethods.cs | 74 ++++++++++++++++++- .../Storage/Session/MainStore/MainStoreOps.cs | 29 +------- .../Session/UnifiedStore/AdvancedOps.cs | 28 +++++++ .../Session/UnifiedStore/CompletePending.cs | 32 ++++++++ .../Session/UnifiedStore/UnifiedStoreOps.cs | 11 +++ libs/server/Transaction/TransactionManager.cs | 6 ++ test/Garnet.test/RespSortedSetTests.cs | 9 ++- test/Garnet.test/RespTests.cs | 2 +- 35 files changed, 407 insertions(+), 135 deletions(-) create mode 100644 libs/server/API/GarnetApiUnifiedCommands.cs create mode 100644 libs/server/GarnetUnifiedStoreOutput.cs create mode 100644 libs/server/OutputHeader.cs create mode 100644 libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs create mode 100644 libs/server/Storage/Session/UnifiedStore/CompletePending.cs create mode 100644 libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs index 554891de264..60031d89275 100644 --- a/libs/cluster/Server/ClusterManagerSlotState.cs +++ b/libs/cluster/Server/ClusterManagerSlotState.cs @@ -17,6 +17,9 @@ namespace Garnet.cluster ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; /// diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs index 73d7a5b3d57..90bd30183b5 100644 --- a/libs/cluster/Server/ClusterProvider.cs +++ b/libs/cluster/Server/ClusterProvider.cs @@ -20,6 +20,9 @@ namespace Garnet.cluster ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; /// diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs index cf019380c84..ee67c83175b 100644 --- a/libs/cluster/Session/ClusterSession.cs +++ b/libs/cluster/Session/ClusterSession.cs @@ -17,6 +17,9 @@ namespace Garnet.cluster ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 9280af7a1cd..1980f807e06 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -17,6 +17,9 @@ namespace Garnet.cluster ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession diff --git a/libs/common/RespMemoryWriter.cs b/libs/common/RespMemoryWriter.cs index 8d6ee383374..ebb864855c1 100644 --- a/libs/common/RespMemoryWriter.cs +++ b/libs/common/RespMemoryWriter.cs @@ -378,6 +378,17 @@ public void WriteSimpleString(ReadOnlySpan simpleString) ReallocateOutput(simpleString.Length); } + /// + /// Write simple string to memory. + /// + /// An ASCII encoded simple string. The string mustn't contain a CR (\r) or LF (\n) bytes. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteSimpleString(ReadOnlySpan simpleString) + { + while (!RespWriteUtils.TryWriteSimpleString(simpleString, ref curr, end)) + ReallocateOutput(simpleString.Length); + } + /// /// Write RESP3 true /// diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index daa7f3257af..eeb7b1f470b 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -16,19 +16,22 @@ namespace Garnet.server /// /// Garnet API implementation /// - public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi + public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi where TContext : ITsavoriteContext where TObjectContext : ITsavoriteContext + where TUnifiedContext : ITsavoriteContext { readonly StorageSession storageSession; TContext context; TObjectContext objectContext; + TUnifiedContext unifiedContext; - internal GarnetApi(StorageSession storageSession, TContext context, TObjectContext objectContext) + internal GarnetApi(StorageSession storageSession, TContext context, TObjectContext objectContext, TUnifiedContext unifiedContext) { this.storageSession = storageSession; this.context = context; this.objectContext = objectContext; + this.unifiedContext = unifiedContext; } #region WATCH @@ -299,22 +302,6 @@ public GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output) => storageSession.GETDEL(key, ref output, ref context); #endregion - #region TYPE - - /// - public GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName) - => storageSession.GetKeyType(key, out typeName, ref context, ref objectContext); - - #endregion - - #region MEMORY - - /// - public GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, int samples = 0) - => storageSession.MemoryUsageForKey(key, out memoryUsage, ref context, ref objectContext, samples); - - #endregion - #region Advanced ops /// public GarnetStatus RMW_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs index e0c68d634a0..cdea775016d 100644 --- a/libs/server/API/GarnetApiObjectCommands.cs +++ b/libs/server/API/GarnetApiObjectCommands.cs @@ -13,9 +13,10 @@ namespace Garnet.server /// /// Garnet API implementation /// - public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi + public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi where TContext : ITsavoriteContext where TObjectContext : ITsavoriteContext + where TUnifiedContext : ITsavoriteContext { #region SortedSet Methods diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs new file mode 100644 index 00000000000..bc0202135ae --- /dev/null +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; + + /// + /// Garnet API implementation + /// + public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext + where TUnifiedContext : ITsavoriteContext + { + #region MEMORY + + /// + public GarnetStatus MEMORYUSAGE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + #endregion + + #region TYPE + + /// + public GarnetStatus TYPE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + #endregion + } +} diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 5d27b7d6591..e915ce25a2d 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -302,9 +302,10 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// string, list, set, zset, and hash. /// /// - /// + /// + /// /// - GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName); + GarnetStatus TYPE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); #endregion @@ -314,10 +315,10 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// Gets the number of bytes that a key and its value require to be stored in RAM. /// /// Name of the key or object to get the memory usage - /// The value in bytes the key or object is using - /// Number of sampled nested values + /// + /// /// GarnetStatus - GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, int samples = 0); + GarnetStatus MEMORYUSAGE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); #endregion diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs index b7e1d79b5c6..2a11d483ff3 100644 --- a/libs/server/Cluster/IClusterProvider.cs +++ b/libs/server/Cluster/IClusterProvider.cs @@ -17,6 +17,9 @@ namespace Garnet.server ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; /// diff --git a/libs/server/Custom/CustomObjectBase.cs b/libs/server/Custom/CustomObjectBase.cs index 81e354f914c..c80c82201fb 100644 --- a/libs/server/Custom/CustomObjectBase.cs +++ b/libs/server/Custom/CustomObjectBase.cs @@ -82,7 +82,7 @@ public sealed override bool Operate(ref ObjectInput input, ref GarnetObjectStore if ((byte)input.header.type != this.type) { // Indicates an incorrect type of key - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; output.SpanByteAndMemory.Length = 0; return true; } diff --git a/libs/server/GarnetUnifiedStoreOutput.cs b/libs/server/GarnetUnifiedStoreOutput.cs new file mode 100644 index 00000000000..5dc04c40499 --- /dev/null +++ b/libs/server/GarnetUnifiedStoreOutput.cs @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Output type used by Garnet unified store. + /// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller). + /// That is in order to justify transferring data in this struct through the Tsavorite storage layer. + /// + public struct GarnetUnifiedStoreOutput + { + /// + /// Span byte and memory + /// + public SpanByteAndMemory SpanByteAndMemory; + + /// + /// Output header + /// + public UnifiedOutputHeader Header; + + /// + /// Output flags + /// + public OutputFlags OutputFlags; + + /// + /// True if output flag WrongType is set + /// + public readonly bool HasWrongType => + (OutputFlags & OutputFlags.WrongType) == OutputFlags.WrongType; + + /// + /// True if output flag RemoveKey is set + /// + public readonly bool HasRemoveKey => + (OutputFlags & OutputFlags.RemoveKey) == OutputFlags.RemoveKey; + + public GarnetUnifiedStoreOutput() => SpanByteAndMemory = new(null); + + public GarnetUnifiedStoreOutput(SpanByteAndMemory span) => SpanByteAndMemory = span; + + public static unsafe GarnetUnifiedStoreOutput FromPinnedPointer(byte* pointer, int length) + => new(new SpanByteAndMemory() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length) }); + + public void ConvertToHeap() + { + // Does not convert to heap when going pending, because we immediately complete pending operations for unified store. + } + } +} \ No newline at end of file diff --git a/libs/server/InputHeader.cs b/libs/server/InputHeader.cs index 6cc7ae39e1a..824623ed3a1 100644 --- a/libs/server/InputHeader.cs +++ b/libs/server/InputHeader.cs @@ -615,22 +615,4 @@ public unsafe int DeserializeFrom(byte* src) return len; } } - - /// - /// Object output header (sometimes used as footer) - /// - [StructLayout(LayoutKind.Explicit, Size = Size)] - public struct ObjectOutputHeader - { - /// - /// Expected size of this object - /// - public const int Size = 4; - - /// - /// Some result of operation (e.g., number of items added successfully) - /// - [FieldOffset(0)] - public int result1; - } } \ No newline at end of file diff --git a/libs/server/Objects/Hash/HashObject.cs b/libs/server/Objects/Hash/HashObject.cs index f5e7022dcfc..759e2a7fafd 100644 --- a/libs/server/Objects/Hash/HashObject.cs +++ b/libs/server/Objects/Hash/HashObject.cs @@ -189,7 +189,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput if (input.header.type != GarnetObjectType.Hash) { //Indicates when there is an incorrect type - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; output.SpanByteAndMemory.Length = 0; return true; } @@ -264,7 +264,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput memorySizeChange = this.MemorySize - previousMemorySize; if (hash.Count == 0) - output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; + output.OutputFlags |= OutputFlags.RemoveKey; return true; } diff --git a/libs/server/Objects/List/ListObject.cs b/libs/server/Objects/List/ListObject.cs index fd8148c1016..87d101563b7 100644 --- a/libs/server/Objects/List/ListObject.cs +++ b/libs/server/Objects/List/ListObject.cs @@ -136,7 +136,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput if (input.header.type != GarnetObjectType.List) { // Indicates an incorrect type of key - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; output.SpanByteAndMemory.Length = 0; return true; } @@ -190,7 +190,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput memorySizeChange = this.MemorySize - previousMemorySize; if (list.Count == 0) - output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; + output.OutputFlags |= OutputFlags.RemoveKey; return true; } diff --git a/libs/server/Objects/Set/SetObject.cs b/libs/server/Objects/Set/SetObject.cs index 87252a3c8df..f0c9530b02e 100644 --- a/libs/server/Objects/Set/SetObject.cs +++ b/libs/server/Objects/Set/SetObject.cs @@ -130,7 +130,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput if (input.header.type != GarnetObjectType.Set) { // Indicates an incorrect type of key - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; output.SpanByteAndMemory.Length = 0; return true; } @@ -172,7 +172,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput memorySizeChange = this.MemorySize - prevMemorySize; if (Set.Count == 0) - output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; + output.OutputFlags |= OutputFlags.RemoveKey; return true; } diff --git a/libs/server/Objects/SortedSet/SortedSetObject.cs b/libs/server/Objects/SortedSet/SortedSetObject.cs index 487cbb446b7..d52b2f54691 100644 --- a/libs/server/Objects/SortedSet/SortedSetObject.cs +++ b/libs/server/Objects/SortedSet/SortedSetObject.cs @@ -327,7 +327,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput if (header.type != GarnetObjectType.SortedSet) { // Indicates an incorrect type of key - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; output.SpanByteAndMemory.Length = 0; return true; } @@ -421,7 +421,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput memorySizeChange = this.MemorySize - prevMemorySize; if (sortedSetDict.Count == 0) - output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; + output.OutputFlags |= OutputFlags.RemoveKey; return true; } diff --git a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs index c7da0575ad2..bb593656c45 100644 --- a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs +++ b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs @@ -1,33 +1,10 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; using Tsavorite.core; namespace Garnet.server { - /// - /// Flags for object store outputs. - /// - [Flags] - public enum ObjectStoreOutputFlags : byte - { - /// - /// No flags set - /// - None = 0, - - /// - /// Remove key - /// - RemoveKey = 1, - - /// - /// Wrong type of object - /// - WrongType = 1 << 1, - } - /// /// Output type used by Garnet object store. /// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller). @@ -53,19 +30,19 @@ public struct GarnetObjectStoreOutput /// /// Output flags /// - public ObjectStoreOutputFlags OutputFlags; + public OutputFlags OutputFlags; /// /// True if output flag WrongType is set /// public readonly bool HasWrongType => - (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType; + (OutputFlags & OutputFlags.WrongType) == OutputFlags.WrongType; /// /// True if output flag RemoveKey is set /// public readonly bool HasRemoveKey => - (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey; + (OutputFlags & OutputFlags.RemoveKey) == OutputFlags.RemoveKey; public GarnetObjectStoreOutput() => SpanByteAndMemory = new(null); @@ -79,17 +56,4 @@ public void ConvertToHeap() // Does not convert to heap when going pending, because we immediately complete pending operations for object store. } } - - /// - /// Output type used by Garnet unified store. - /// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller). - /// That is in order to justify transferring data in this struct through the Tsavorite storage layer. - /// - public struct GarnetUnifiedStoreOutput - { - /// - /// Some result of operation (e.g., number of items added successfully) - /// - public long result1; - } } \ No newline at end of file diff --git a/libs/server/OutputHeader.cs b/libs/server/OutputHeader.cs new file mode 100644 index 00000000000..1648af928b7 --- /dev/null +++ b/libs/server/OutputHeader.cs @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.InteropServices; + +namespace Garnet.server +{ + /// + /// Flags for store outputs. + /// + [Flags] + public enum OutputFlags : byte + { + /// + /// No flags set + /// + None = 0, + + /// + /// Remove key + /// + RemoveKey = 1, + + /// + /// Wrong type of value + /// + WrongType = 1 << 1, + } + + /// + /// Object output header (sometimes used as footer) + /// + [StructLayout(LayoutKind.Explicit, Size = Size)] + public struct ObjectOutputHeader + { + /// + /// Expected size of this struct + /// + public const int Size = 4; + + /// + /// Some result of operation (e.g., number of items added successfully) + /// + [FieldOffset(0)] + public int result1; + } + + /// + /// Unified output header (sometimes used as footer) + /// + [StructLayout(LayoutKind.Explicit, Size = Size)] + public struct UnifiedOutputHeader + { + /// + /// Expected size of this struct + /// + public const int Size = 8; + + /// + /// Some result of operation (e.g., number of items added successfully) + /// + [FieldOffset(0)] + public long result1; + } +} diff --git a/libs/server/Resp/ArrayCommands.cs b/libs/server/Resp/ArrayCommands.cs index 8d6dda4dc75..0abbb7cb46c 100644 --- a/libs/server/Resp/ArrayCommands.cs +++ b/libs/server/Resp/ArrayCommands.cs @@ -438,16 +438,22 @@ private bool NetworkTYPE(ref TGarnetApi storageApi) // TYPE key var keySlice = parseState.GetArgSliceByRef(0); - var status = storageApi.GetKeyType(keySlice, out var typeName); + // Prepare input + var input = new UnifiedStoreInput(RespCommand.TYPE); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + + var status = storageApi.TYPE(keySlice, ref input, ref output); if (status == GarnetStatus.OK) { - while (!RespWriteUtils.TryWriteSimpleString(typeName, ref dcurr, dend)) - SendAndReset(); + ProcessOutput(output.SpanByteAndMemory); } else { - while (!RespWriteUtils.TryWriteSimpleString("none"u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteSimpleString(CmdStrings.none, ref dcurr, dend)) SendAndReset(); } diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs index 5accfc9f569..ed9b3e60fdd 100644 --- a/libs/server/Resp/BasicCommands.cs +++ b/libs/server/Resp/BasicCommands.cs @@ -1433,11 +1433,17 @@ private bool NetworkMemoryUsage(ref TGarnetApi storageApi) } } - var status = storageApi.MemoryUsageForKey(key, out var memoryUsage); + // Prepare input + var input = new UnifiedStoreInput(RespCommand.MEMORY_USAGE); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + var status = storageApi.MEMORYUSAGE(key, ref input, ref output); if (status == GarnetStatus.OK) { - while (!RespWriteUtils.TryWriteInt32((int)memoryUsage, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteInt32((int)output.Header.result1, ref dcurr, dend)) SendAndReset(); } else diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index e0a4a29eb77..426f5677c65 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -357,6 +357,7 @@ static partial class CmdStrings public static ReadOnlySpan hash => "hash"u8; public static ReadOnlySpan STRING => "STRING"u8; public static ReadOnlySpan stringt => "string"u8; + public static ReadOnlySpan none => "none"u8; /// /// Register object types diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs index 04a94e3bc5c..bb56abebf7b 100644 --- a/libs/server/Resp/GarnetDatabaseSession.cs +++ b/libs/server/Resp/GarnetDatabaseSession.cs @@ -8,12 +8,18 @@ namespace Garnet.server ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; using TransactionalGarnetApi = GarnetApi, ObjectAllocator>>, TransactionalContext, + ObjectAllocator>>, + TransactionalContext, ObjectAllocator>>>; /// diff --git a/libs/server/Resp/LocalServerSession.cs b/libs/server/Resp/LocalServerSession.cs index 4ce3d3950b9..2b990828131 100644 --- a/libs/server/Resp/LocalServerSession.cs +++ b/libs/server/Resp/LocalServerSession.cs @@ -12,6 +12,9 @@ namespace Garnet.server ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; /// @@ -50,7 +53,7 @@ public LocalServerSession(StoreWrapper storeWrapper) // Create storage session and API this.storageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId: 0, logger); - this.BasicGarnetApi = new BasicGarnetApi(storageSession, storageSession.basicContext, storageSession.objectStoreBasicContext); + this.BasicGarnetApi = new BasicGarnetApi(storageSession, storageSession.basicContext, storageSession.objectStoreBasicContext, storageSession.unifiedStoreBasicContext); } /// diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index eecc0925586..043c1860e98 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -25,12 +25,18 @@ namespace Garnet.server ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; using TransactionalGarnetApi = GarnetApi, ObjectAllocator>>, TransactionalContext, + ObjectAllocator>>, + TransactionalContext, ObjectAllocator>>>; /// @@ -1492,8 +1498,11 @@ private GarnetDatabaseSession TryGetOrSetDatabaseSession(int dbId, out bool succ private GarnetDatabaseSession CreateDatabaseSession(int dbId) { var dbStorageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId, logger, respProtocolVersion); - var dbGarnetApi = new BasicGarnetApi(dbStorageSession, dbStorageSession.basicContext, dbStorageSession.objectStoreBasicContext); - var dbLockableGarnetApi = new TransactionalGarnetApi(dbStorageSession, dbStorageSession.transactionalContext, dbStorageSession.objectStoreTransactionalContext); + var dbGarnetApi = new BasicGarnetApi(dbStorageSession, dbStorageSession.basicContext, + dbStorageSession.objectStoreBasicContext, dbStorageSession.unifiedStoreBasicContext); + var dbLockableGarnetApi = new TransactionalGarnetApi(dbStorageSession, + dbStorageSession.transactionalContext, dbStorageSession.objectStoreTransactionalContext, + dbStorageSession.unifiedStoreTransactionalContext); var transactionManager = new TransactionManager(storeWrapper, this, dbGarnetApi, dbLockableGarnetApi, dbStorageSession, scratchBufferAllocator, storeWrapper.serverOptions.EnableCluster, logger, dbId); diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index 01a231606d3..9eba297345b 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -171,7 +171,7 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r var garnetValueObject = Unsafe.As(logRecord.ValueObject); if (IncorrectObjectType(ref input, garnetValueObject, ref output.SpanByteAndMemory)) { - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; return true; } @@ -279,7 +279,7 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, // using Clone. Currently, expire and persist commands are performed on the new copy of the object. if (IncorrectObjectType(ref input, value, ref output.SpanByteAndMemory)) { - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; return true; } diff --git a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs index fe53bfe4eda..4ed1fca6c85 100644 --- a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs @@ -61,7 +61,7 @@ public bool Reader(in TSourceLogRecord srcLogRecord, ref Objec if (IncorrectObjectType(ref input, (IGarnetObject)srcLogRecord.ValueObject, ref output.SpanByteAndMemory)) { - output.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= OutputFlags.WrongType; return true; } diff --git a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs index e329e78c5c1..0eb7293dae8 100644 --- a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs @@ -2,17 +2,87 @@ // Licensed under the MIT license. using System; +using Garnet.common; using Tsavorite.core; namespace Garnet.server { +#pragma warning disable IDE0005 // Using directive is unnecessary. + using static LogRecordUtils; + /// /// Unified store functions /// public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions { public bool Reader(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, - ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord + { + if (CheckExpiry(in srcLogRecord)) + { + readInfo.Action = ReadAction.Expire; + return false; + } + + var cmd = input.header.cmd; + return cmd switch + { + RespCommand.MEMORY_USAGE => HandleMemoryUsage(in srcLogRecord, ref input, ref output, ref readInfo), + RespCommand.TYPE => HandleType(in srcLogRecord, ref input, ref output, ref readInfo), + _ => throw new NotImplementedException(), + }; + } + + private bool HandleMemoryUsage(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord + { + long memoryUsage; + if (srcLogRecord.Info.ValueIsObject) + { + memoryUsage = RecordInfo.GetLength() + (2 * IntPtr.Size) + // Log record length + Utility.RoundUp(srcLogRecord.Key.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + // Key allocation in heap with overhead + srcLogRecord.ValueObject.MemorySize; // Value allocation in heap + } + else + { + memoryUsage = RecordInfo.GetLength() + + Utility.RoundUp(srcLogRecord.Key.TotalSize(), RecordInfo.GetLength()) + + Utility.RoundUp(srcLogRecord.ValueSpan.TotalSize(), RecordInfo.GetLength()); + } + + output.Header.result1 = memoryUsage; + return true; + } + + private bool HandleType(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord + { + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + + if (srcLogRecord.Info.ValueIsObject) + { + switch (srcLogRecord.ValueObject) + { + case SortedSetObject: + writer.WriteSimpleString(CmdStrings.zset); + break; + case ListObject: + writer.WriteSimpleString(CmdStrings.list); + break; + case SetObject: + writer.WriteSimpleString(CmdStrings.set); + break; + case HashObject: + writer.WriteSimpleString(CmdStrings.hash); + break; + } + } + else + { + writer.WriteSimpleString(CmdStrings.stringt); + } + + return true; + } } } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 22dafef763b..521c1475b80 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -1112,7 +1112,7 @@ public unsafe GarnetStatus Increment(PinnedSpanByte key, out long outp public unsafe GarnetStatus SCAN(long cursor, PinnedSpanByte match, long count, ref TContext context) => GarnetStatus.OK; - public GarnetStatus GetKeyType(PinnedSpanByte key, out string keyType, ref TContext context, ref TObjectContext objectContext) + public GarnetStatus TYPE(PinnedSpanByte key, out string keyType, ref TContext context, ref TObjectContext objectContext) where TContext : ITsavoriteContext where TObjectContext : ITsavoriteContext { @@ -1152,33 +1152,6 @@ public GarnetStatus GetKeyType(PinnedSpanByte key, out return status; } - public GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, ref TContext context, ref TObjectContext objectContext, int samples = 0) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - memoryUsage = -1; - - // Check if key exists in Main store - var status = GET(key, out PinnedSpanByte keyValue, ref context); - - if (status == GarnetStatus.NOTFOUND) - { - status = GET(key, out GarnetObjectStoreOutput objectValue, ref objectContext); - if (status != GarnetStatus.NOTFOUND) - { - memoryUsage = RecordInfo.GetLength() + (2 * IntPtr.Size) + // Log record length - Utility.RoundUp(key.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + // Key allocation in heap with overhead - objectValue.GarnetObject.MemorySize; // Value allocation in heap - } - } - else - { - memoryUsage = RecordInfo.GetLength() + Utility.RoundUp(key.TotalSize, RecordInfo.GetLength()) + Utility.RoundUp(keyValue.TotalSize, RecordInfo.GetLength()); - } - - return status; - } - /// /// Computes the Longest Common Subsequence (LCS) of two keys. /// diff --git a/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs new file mode 100644 index 00000000000..5d8dc6840ea --- /dev/null +++ b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Garnet.server +{ + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; + + sealed partial class StorageSession : IDisposable + { + public GarnetStatus Read_UnifiedStore(ReadOnlySpan key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref TUnifiedContext unifiedContext) + where TUnifiedContext : ITsavoriteContext + { + var status = unifiedContext.Read(key, ref input, ref output); + + if (status.IsPending) + CompletePendingForUnifiedStoreSession(ref status, ref output, ref unifiedContext); + + if (status.Found) + return GarnetStatus.OK; + else + return GarnetStatus.NOTFOUND; + } + } +} diff --git a/libs/server/Storage/Session/UnifiedStore/CompletePending.cs b/libs/server/Storage/Session/UnifiedStore/CompletePending.cs new file mode 100644 index 00000000000..5810fdca203 --- /dev/null +++ b/libs/server/Storage/Session/UnifiedStore/CompletePending.cs @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; +using Tsavorite.core; + +namespace Garnet.server +{ + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; + + sealed partial class StorageSession + { + /// + /// Handles the complete pending for Unified Store session + /// + /// + /// + /// + static void CompletePendingForUnifiedStoreSession(ref Status status, ref GarnetUnifiedStoreOutput output, ref TUnifiedContext unified) + where TUnifiedContext : ITsavoriteContext + { + unified.CompletePendingWithOutputs(out var completedOutputs, wait: true); + var more = completedOutputs.Next(); + Debug.Assert(more); + status = completedOutputs.Current.Status; + output = completedOutputs.Current.Output; + Debug.Assert(!completedOutputs.Next()); + completedOutputs.Dispose(); + } + } +} diff --git a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs new file mode 100644 index 00000000000..8747c0fe6c0 --- /dev/null +++ b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs @@ -0,0 +1,11 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Garnet.server +{ + sealed partial class StorageSession : IDisposable + { + } +} diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index f12ff791407..40e3180b434 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -15,6 +15,9 @@ namespace Garnet.server ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; using StoreAllocator = ObjectAllocator>; using StoreFunctions = StoreFunctions; @@ -23,6 +26,9 @@ namespace Garnet.server ObjectAllocator>>, TransactionalContext, + ObjectAllocator>>, + TransactionalContext, ObjectAllocator>>>; /// diff --git a/test/Garnet.test/RespSortedSetTests.cs b/test/Garnet.test/RespSortedSetTests.cs index c4cbec2702d..ed02b04e7f7 100644 --- a/test/Garnet.test/RespSortedSetTests.cs +++ b/test/Garnet.test/RespSortedSetTests.cs @@ -24,6 +24,9 @@ namespace Garnet.test ObjectAllocator>>, BasicContext, + ObjectAllocator>>, + BasicContext, ObjectAllocator>>>; [TestFixture] @@ -100,7 +103,8 @@ public unsafe void SortedSetPopTest() db.SortedSetAdd("key1", "b", 2); var session = new RespServerSession(0, new EmbeddedNetworkSender(), server.Provider.StoreWrapper, null, null, false); - var api = new TestBasicGarnetApi(session.storageSession, session.storageSession.basicContext, session.storageSession.objectStoreBasicContext); + var api = new TestBasicGarnetApi(session.storageSession, session.storageSession.basicContext, + session.storageSession.objectStoreBasicContext, session.storageSession.unifiedStoreBasicContext); var key = Encoding.ASCII.GetBytes("key1"); fixed (byte* keyPtr = key) { @@ -132,7 +136,8 @@ public unsafe void SortedSetPopWithExpire() Thread.Sleep(200); var session = new RespServerSession(0, new EmbeddedNetworkSender(), server.Provider.StoreWrapper, null, null, false); - var api = new TestBasicGarnetApi(session.storageSession, session.storageSession.basicContext, session.storageSession.objectStoreBasicContext); + var api = new TestBasicGarnetApi(session.storageSession, session.storageSession.basicContext, + session.storageSession.objectStoreBasicContext, session.storageSession.unifiedStoreBasicContext); var key = Encoding.ASCII.GetBytes("key1"); fixed (byte* keyPtr = key) { diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index 3bf07526fd4..8d2392c940f 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -1830,7 +1830,7 @@ public void PExpiretimeWithUnknownKey() var db = redis.GetDatabase(0); var expireTime = (long)db.Execute("PEXPIRETIME", "keyZ"); - + ClassicAssert.AreEqual(-2, expireTime); } From 222963e898c2c4227b35eb50eacc2405dceeda8b Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 18 Sep 2025 15:27:23 -0700 Subject: [PATCH 07/28] Added TTL, EXPIRETIME and EXISTS to unified ops --- .../SlotVerification/ClusterSlotVerify.cs | 2 +- libs/server/API/GarnetApi.cs | 30 ------ libs/server/API/GarnetApiUnifiedCommands.cs | 28 ++++++ libs/server/API/GarnetWatchApi.cs | 26 ++---- libs/server/API/IGarnetApi.cs | 40 ++++---- libs/server/GarnetUnifiedStoreOutput.cs | 11 --- libs/server/OutputHeader.cs | 18 ---- libs/server/Resp/BasicCommands.cs | 3 +- libs/server/Resp/KeyAdminCommands.cs | 44 +++++---- .../Functions/UnifiedStore/ReadMethods.cs | 37 +++++++- .../Storage/Session/MainStore/MainStoreOps.cs | 93 ------------------- .../Session/UnifiedStore/UnifiedStoreOps.cs | 24 +++++ 12 files changed, 138 insertions(+), 218 deletions(-) diff --git a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs index bcb34cd55f1..4d5afeb0c18 100644 --- a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs +++ b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs @@ -14,7 +14,7 @@ internal sealed unsafe partial class ClusterSession : IClusterSession { [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool Exists(PinnedSpanByte keySlice) - => basicGarnetApi.EXISTS(keySlice, StoreType.All) == GarnetStatus.OK; + => basicGarnetApi.EXISTS(keySlice) == GarnetStatus.OK; private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref PinnedSpanByte keySlice, bool readOnly, byte SessionAsking, int slot = -1) { diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index eeb7b1f470b..686f411fe28 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -87,30 +87,6 @@ public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref S => storageSession.GETRANGE(key, ref input, ref output, ref context); #endregion - #region TTL - - /// - public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.TTL(key, storeType, ref output, ref context, ref objectContext); - - /// - public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.TTL(key, storeType, ref output, ref context, ref objectContext, milliseconds: true); - - #endregion - - #region EXPIRETIME - - /// - public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.EXPIRETIME(key, storeType, ref output, ref context, ref objectContext); - - /// - public GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.EXPIRETIME(key, storeType, ref output, ref context, ref objectContext, milliseconds: true); - - #endregion - #region SET /// public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value) @@ -193,12 +169,6 @@ public GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out i => storageSession.RENAMENX(oldKey, newKey, storeType, out result, withEtag); #endregion - #region EXISTS - /// - public GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All) - => storageSession.EXISTS(key, storeType, ref context, ref objectContext); - #endregion - #region EXPIRE /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All) diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index bc0202135ae..3462766648a 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -31,5 +31,33 @@ public GarnetStatus TYPE(PinnedSpanByte key, ref UnifiedStoreInput input, ref Ga => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); #endregion + + #region TTL + + /// + public GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + #endregion + + #region EXPIRETIME + + /// + public GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + #endregion + + #region EXISTS + + /// + public GarnetStatus EXISTS(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + /// + public GarnetStatus EXISTS(PinnedSpanByte key) + => storageSession.EXISTS(key, ref unifiedContext); + + #endregion } } diff --git a/libs/server/API/GarnetWatchApi.cs b/libs/server/API/GarnetWatchApi.cs index 2452019132c..5be3170b96f 100644 --- a/libs/server/API/GarnetWatchApi.cs +++ b/libs/server/API/GarnetWatchApi.cs @@ -70,17 +70,10 @@ public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref S #region TTL /// - public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) { - garnetApi.WATCH(key, storeType); - return garnetApi.TTL(key, storeType, ref output); - } - - /// - public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - { - garnetApi.WATCH(key, storeType); - return garnetApi.PTTL(key, storeType, ref output); + garnetApi.WATCH(key, StoreType.All); + return garnetApi.TTL(key, ref input, ref output); } #endregion @@ -88,17 +81,10 @@ public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAn #region EXPIRETIME /// - public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) { - garnetApi.WATCH(key, storeType); - return garnetApi.EXPIRETIME(key, storeType, ref output); - } - - /// - public GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) - { - garnetApi.WATCH(key, storeType); - return garnetApi.PEXPIRETIME(key, storeType, ref output); + garnetApi.WATCH(key, StoreType.All); + return garnetApi.EXPIRETIME(key, ref input, ref output); } #endregion diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index e915ce25a2d..4dd4bcef6e3 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -149,13 +149,23 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s #endregion #region EXISTS + /// /// EXISTS /// /// - /// + /// + /// + /// + GarnetStatus EXISTS(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); + + /// + /// EXISTS + /// + /// Key /// - GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All); + GarnetStatus EXISTS(PinnedSpanByte key); + #endregion #region EXPIRE @@ -1242,19 +1252,10 @@ public interface IGarnetReadApi /// Returns the remaining time to live in seconds of a key that has a timeout. /// /// The key to return the remaining time to live in the store - /// The store type to operate on. - /// The span to allocate the output of the operation. - /// - GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); - - /// - /// Returns the remaining time to live in milliseconds of a key that has a timeout. - /// - /// The key to return the remaining time to live in the store. - /// The store type to operate on. + /// /// The span to allocate the output of the operation. /// - GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); + GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); #endregion @@ -1264,19 +1265,10 @@ public interface IGarnetReadApi /// Returns the absolute Unix timestamp (since January 1, 1970) in seconds at which the given key will expire. /// /// The key to get the expiration time for. - /// The type of store to retrieve the key from. - /// The output containing the expiration time. - /// The status of the operation. - GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); - - /// - /// Returns the absolute Unix timestamp (since January 1, 1970) in milliseconds at which the given key will expire. - /// - /// The key to get the expiration time for. - /// The type of store to retrieve the key from. + /// /// The output containing the expiration time. /// The status of the operation. - GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); + GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); #endregion diff --git a/libs/server/GarnetUnifiedStoreOutput.cs b/libs/server/GarnetUnifiedStoreOutput.cs index 5dc04c40499..4d14a411a48 100644 --- a/libs/server/GarnetUnifiedStoreOutput.cs +++ b/libs/server/GarnetUnifiedStoreOutput.cs @@ -17,22 +17,11 @@ public struct GarnetUnifiedStoreOutput /// public SpanByteAndMemory SpanByteAndMemory; - /// - /// Output header - /// - public UnifiedOutputHeader Header; - /// /// Output flags /// public OutputFlags OutputFlags; - /// - /// True if output flag WrongType is set - /// - public readonly bool HasWrongType => - (OutputFlags & OutputFlags.WrongType) == OutputFlags.WrongType; - /// /// True if output flag RemoveKey is set /// diff --git a/libs/server/OutputHeader.cs b/libs/server/OutputHeader.cs index 1648af928b7..a5bbcf73202 100644 --- a/libs/server/OutputHeader.cs +++ b/libs/server/OutputHeader.cs @@ -45,22 +45,4 @@ public struct ObjectOutputHeader [FieldOffset(0)] public int result1; } - - /// - /// Unified output header (sometimes used as footer) - /// - [StructLayout(LayoutKind.Explicit, Size = Size)] - public struct UnifiedOutputHeader - { - /// - /// Expected size of this struct - /// - public const int Size = 8; - - /// - /// Some result of operation (e.g., number of items added successfully) - /// - [FieldOffset(0)] - public long result1; - } } diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs index ed9b3e60fdd..7bd04cb8e7b 100644 --- a/libs/server/Resp/BasicCommands.cs +++ b/libs/server/Resp/BasicCommands.cs @@ -1443,8 +1443,7 @@ private bool NetworkMemoryUsage(ref TGarnetApi storageApi) if (status == GarnetStatus.OK) { - while (!RespWriteUtils.TryWriteInt32((int)output.Header.result1, ref dcurr, dend)) - SendAndReset(); + ProcessOutput(output.SpanByteAndMemory); } else { diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index 930147f663e..e5f48da5cb6 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -358,10 +358,16 @@ private bool NetworkEXISTS(ref TGarnetApi storageApi) var exists = 0; + // Prepare input + var input = new UnifiedStoreInput(RespCommand.EXISTS); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + for (var i = 0; i < parseState.Count; i++) { var key = parseState.GetArgSliceByRef(i); - var status = storageApi.EXISTS(key); + var status = storageApi.EXISTS(key, ref input, ref output); if (status == GarnetStatus.OK) exists++; } @@ -513,21 +519,22 @@ private bool NetworkTTL(RespCommand command, ref TGarnetApi storageA { if (parseState.Count != 1) { - return AbortWithWrongNumberOfArguments(nameof(RespCommand.PERSIST)); + return AbortWithWrongNumberOfArguments(command.ToString()); } var key = parseState.GetArgSliceByRef(0); - var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = command == RespCommand.TTL ? - storageApi.TTL(key, StoreType.All, ref o) : - storageApi.PTTL(key, StoreType.All, ref o); + + // Prepare input + var input = new UnifiedStoreInput(command); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + var status = storageApi.TTL(key, ref input, ref output); if (status == GarnetStatus.OK) { - if (!o.IsSpanByte) - SendAndReset(o.Memory, o.Length); - else - dcurr += o.Length; + ProcessOutput(output.SpanByteAndMemory); } else { @@ -553,17 +560,18 @@ private bool NetworkEXPIRETIME(RespCommand command, ref TGarnetApi s } var key = parseState.GetArgSliceByRef(0); - var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = command == RespCommand.EXPIRETIME ? - storageApi.EXPIRETIME(key, StoreType.All, ref o) : - storageApi.PEXPIRETIME(key, StoreType.All, ref o); + + // Prepare input + var input = new UnifiedStoreInput(command); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + var status = storageApi.EXPIRETIME(key, ref input, ref output); if (status == GarnetStatus.OK) { - if (!o.IsSpanByte) - SendAndReset(o.Memory, o.Length); - else - dcurr += o.Length; + ProcessOutput(output.SpanByteAndMemory); } else { diff --git a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs index 0eb7293dae8..abc50db04e9 100644 --- a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs @@ -27,8 +27,13 @@ public bool Reader(in TSourceLogRecord srcLogRecord, ref Unifi var cmd = input.header.cmd; return cmd switch { + RespCommand.EXISTS => true, RespCommand.MEMORY_USAGE => HandleMemoryUsage(in srcLogRecord, ref input, ref output, ref readInfo), RespCommand.TYPE => HandleType(in srcLogRecord, ref input, ref output, ref readInfo), + RespCommand.TTL or + RespCommand.PTTL => HandleTtl(in srcLogRecord, ref input, ref output, ref readInfo, cmd == RespCommand.PTTL), + RespCommand.EXPIRETIME or + RespCommand.PEXPIRETIME => HandleExpireTime(in srcLogRecord, ref input, ref output, ref readInfo, cmd == RespCommand.PEXPIRETIME), _ => throw new NotImplementedException(), }; } @@ -50,7 +55,9 @@ private bool HandleMemoryUsage(in TSourceLogRecord srcLogRecor Utility.RoundUp(srcLogRecord.ValueSpan.TotalSize(), RecordInfo.GetLength()); } - output.Header.result1 = memoryUsage; + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + writer.WriteInt64(memoryUsage); + return true; } @@ -84,5 +91,33 @@ private bool HandleType(in TSourceLogRecord srcLogRecord, ref return true; } + + private bool HandleTtl(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo, bool milliseconds) where TSourceLogRecord : ISourceLogRecord + { + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + + var expiration = srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1; + var ttlValue = milliseconds + ? ConvertUtils.MillisecondsFromDiffUtcNowTicks(expiration) + : ConvertUtils.SecondsFromDiffUtcNowTicks(expiration); + + writer.WriteInt64(ttlValue); + return true; + } + + private bool HandleExpireTime(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, + ref GarnetUnifiedStoreOutput output, ref ReadInfo readInfo, bool milliseconds) where TSourceLogRecord : ISourceLogRecord + { + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + + var expiration = srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1; + var expireTime = milliseconds + ? ConvertUtils.UnixTimeInMillisecondsFromTicks(expiration) + : ConvertUtils.UnixTimeInSecondsFromTicks(expiration); + + writer.WriteInt64(expireTime); + return true; + } } } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 521c1475b80..bc082b7b66d 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -259,59 +259,6 @@ public unsafe GarnetStatus TTL(PinnedSpanByte key, Sto return GarnetStatus.NOTFOUND; } - /// - /// Get the absolute Unix timestamp at which the given key will expire. - /// - /// - /// - /// The key to get the Unix timestamp. - /// The store to operate on - /// Span to allocate the output of the operation - /// Basic Context of the store - /// Object Context of the store - /// when true the command to execute is PEXPIRETIME. - /// Returns the absolute Unix timestamp (since January 1, 1970) in seconds or milliseconds at which the given key will expire. - public unsafe GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output, ref TContext context, ref TObjectContext objectContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var cmd = milliseconds ? RespCommand.PEXPIRETIME : RespCommand.EXPIRETIME; - var input = new RawStringInput(cmd); - var status = context.Read(key.ReadOnlySpan, ref input, ref output); - - if (status.IsPending) - { - StartPendingMetrics(); - CompletePendingForSession(ref status, ref output, ref context); - StopPendingMetrics(); - } - - if (status.Found) return GarnetStatus.OK; - } - - if ((storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) - { - var type = milliseconds ? GarnetObjectType.PExpireTime : GarnetObjectType.ExpireTime; - var header = new RespInputHeader(type); - var input = new ObjectInput(header); - - var objO = new GarnetObjectStoreOutput(output); - var status = objectContext.Read(key.ReadOnlySpan, ref input, ref objO); - - if (status.IsPending) - CompletePendingForObjectStoreSession(ref status, ref objO, ref objectContext); - - if (status.Found) - { - output = objO.SpanByteAndMemory; - return GarnetStatus.OK; - } - } - return GarnetStatus.NOTFOUND; - } - public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value, ref TContext context) where TContext : ITsavoriteContext { @@ -1112,46 +1059,6 @@ public unsafe GarnetStatus Increment(PinnedSpanByte key, out long outp public unsafe GarnetStatus SCAN(long cursor, PinnedSpanByte match, long count, ref TContext context) => GarnetStatus.OK; - public GarnetStatus TYPE(PinnedSpanByte key, out string keyType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - keyType = "string"; - // Check if key exists in Main store - var status = EXISTS(key, StoreType.Main, ref context, ref objectContext); - - // If key was not found in the main store then it is an object - if (status != GarnetStatus.OK && !objectStoreBasicContext.IsNull) - { - status = GET(key, out GarnetObjectStoreOutput output, ref objectContext); - if (status == GarnetStatus.OK) - { - if ((output.GarnetObject as SortedSetObject) != null) - { - keyType = "zset"; - } - else if ((output.GarnetObject as ListObject) != null) - { - keyType = "list"; - } - else if ((output.GarnetObject as SetObject) != null) - { - keyType = "set"; - } - else if ((output.GarnetObject as HashObject) != null) - { - keyType = "hash"; - } - } - else - { - keyType = "none"; - status = GarnetStatus.NOTFOUND; - } - } - return status; - } - /// /// Computes the Longest Common Subsequence (LCS) of two keys. /// diff --git a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs index 8747c0fe6c0..8bba1266842 100644 --- a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs @@ -2,10 +2,34 @@ // Licensed under the MIT license. using System; +using Tsavorite.core; namespace Garnet.server { + using StoreAllocator = ObjectAllocator>; + using StoreFunctions = StoreFunctions; + sealed partial class StorageSession : IDisposable { + /// + /// Returns if key is an existing one in the store. + /// + /// + /// The name of the key to use in the operation + /// Basic unifiedContext for the unified store. + /// + public GarnetStatus EXISTS(PinnedSpanByte key, ref TUnifiedContext unifiedContext) + where TUnifiedContext : ITsavoriteContext + { + // Prepare input + var input = new UnifiedStoreInput(RespCommand.EXISTS); + + // Prepare GarnetUnifiedStoreOutput output + var output = new GarnetUnifiedStoreOutput(); + + var status = Read_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + return status; + } } } From d677e6a4b6f8005ee6c244111eb587df84fb30e1 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 18 Sep 2025 20:43:38 -0700 Subject: [PATCH 08/28] implemented DEL in unified ops --- .../cluster/Server/ClusterManagerSlotState.cs | 4 +-- libs/server/API/GarnetApi.cs | 6 ---- libs/server/API/GarnetApiUnifiedCommands.cs | 8 +++++ libs/server/API/IGarnetApi.cs | 7 ++-- libs/server/Resp/ArrayCommands.cs | 3 +- libs/server/Resp/KeyAdminCommands.cs | 3 +- .../Storage/Session/MainStore/MainStoreOps.cs | 34 +++++++------------ .../Storage/Session/ObjectStore/Common.cs | 14 ++++++++ .../Session/UnifiedStore/UnifiedStoreOps.cs | 17 +++++++++- .../Extensions/SampleDeleteTxn.cs | 2 +- test/Garnet.test/DeleteTxn.cs | 2 +- 11 files changed, 62 insertions(+), 38 deletions(-) diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs index 60031d89275..3f35e1c029f 100644 --- a/libs/cluster/Server/ClusterManagerSlotState.cs +++ b/libs/cluster/Server/ClusterManagerSlotState.cs @@ -485,7 +485,7 @@ public static unsafe void DeleteKeysInSlotsFromMainStore(BasicGarnetApi BasicGar var key = iter.Key; var s = HashSlotUtils.HashSlot(key); if (slots.Contains(s)) - _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Main); + _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key)); } } @@ -502,7 +502,7 @@ public static unsafe void DeleteKeysInSlotsFromObjectStore(BasicGarnetApi BasicG var key = iterObject.Key; var s = HashSlotUtils.HashSlot(key); if (slots.Contains(s)) - _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Object); + _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key)); } } } diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index 686f411fe28..a921edf67cf 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -260,12 +260,6 @@ public GarnetStatus IncrementByFloat(PinnedSpanByte key, out double output, doub } #endregion - #region DELETE - /// - public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All) - => storageSession.DELETE(key, storeType, ref context, ref objectContext); - #endregion - #region GETDEL /// public GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output) diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index 3462766648a..74e7d0f8bd6 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -59,5 +59,13 @@ public GarnetStatus EXISTS(PinnedSpanByte key) => storageSession.EXISTS(key, ref unifiedContext); #endregion + + #region DELETE + + /// + public GarnetStatus DELETE(PinnedSpanByte key) + => storageSession.DELETE(key, ref unifiedContext); + + #endregion } } diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 4dd4bcef6e3..98827bdbd97 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -286,13 +286,14 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s #endregion #region DELETE + /// - /// DELETE + /// Deletes a key from the unified store /// /// - /// /// - GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All); + GarnetStatus DELETE(PinnedSpanByte key); + #endregion #region GETDEL diff --git a/libs/server/Resp/ArrayCommands.cs b/libs/server/Resp/ArrayCommands.cs index 0abbb7cb46c..4dab5fca5a0 100644 --- a/libs/server/Resp/ArrayCommands.cs +++ b/libs/server/Resp/ArrayCommands.cs @@ -194,10 +194,11 @@ private bool NetworkDEL(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { int keysDeleted = 0; + for (int c = 0; c < parseState.Count; c++) { var key = parseState.GetArgSliceByRef(c); - var status = storageApi.DELETE(key, StoreType.All); + var status = storageApi.DELETE(key); // This is only an approximate count because the deletion of a key on disk is performed as a blind tombstone append if (status == GarnetStatus.OK) diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index e5f48da5cb6..8c3a998958f 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -361,8 +361,7 @@ private bool NetworkEXISTS(ref TGarnetApi storageApi) // Prepare input var input = new UnifiedStoreInput(RespCommand.EXISTS); - // Prepare GarnetUnifiedStoreOutput output - var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var output = new GarnetUnifiedStoreOutput(); for (var i = 0; i < parseState.Count; i++) { diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index bc082b7b66d..9ab10906e5a 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -494,26 +494,18 @@ public unsafe GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInp return GarnetStatus.OK; } - public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) + /// + /// Deletes a key from the main store context. + /// + /// The name of the key to use in the operation + /// Basic context for the main store. + /// + public GarnetStatus DELETE_MainStore(PinnedSpanByte key, ref TContext context) where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext { - var found = false; - - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var status = context.Delete(key.ReadOnlySpan); - Debug.Assert(!status.IsPending); - if (status.Found) found = true; - } - - if (!objectStoreBasicContext.IsNull && (storeType == StoreType.Object || storeType == StoreType.All)) - { - var status = objectContext.Delete(key.ReadOnlySpan); - Debug.Assert(!status.IsPending); - if (status.Found) found = true; - } - return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; + var status = context.Delete(key.ReadOnlySpan); + Debug.Assert(!status.IsPending); + return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } public unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte newKeySlice, StoreType storeType, bool withEtag) @@ -647,12 +639,12 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne // Delete the old key only when SET NX succeeded if (isNX && result == 1) { - DELETE(oldKey, StoreType.Main, ref context, ref objectContext); + DELETE_MainStore(oldKey, ref context); } else if (!isNX) { // Delete the old key - DELETE(oldKey, StoreType.Main, ref context, ref objectContext); + DELETE_MainStore(oldKey, ref context); returnStatus = GarnetStatus.OK; } } @@ -699,7 +691,7 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne SET(newKeySlice, valObj, ref objectContext); // Delete the old key - DELETE(oldKeySlice, StoreType.Object, ref context, ref objectContext); + DELETE_ObjectStore(oldKeySlice, ref objectContext); result = 1; } diff --git a/libs/server/Storage/Session/ObjectStore/Common.cs b/libs/server/Storage/Session/ObjectStore/Common.cs index da3dbc13379..7cc939fcaf1 100644 --- a/libs/server/Storage/Session/ObjectStore/Common.cs +++ b/libs/server/Storage/Session/ObjectStore/Common.cs @@ -775,6 +775,20 @@ unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan return GarnetStatus.NOTFOUND; } + /// + /// Deletes a key from the object store context. + /// + /// The name of the key to use in the operation + /// Basic context for the object store. + /// + public GarnetStatus DELETE_ObjectStore(PinnedSpanByte key, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + { + var status = objectContext.Delete(key.ReadOnlySpan); + Debug.Assert(!status.IsPending); + return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; + } + /// /// Iterates members of a collection object using a cursor, /// a match pattern and count parameters diff --git a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs index 8bba1266842..d1307d7de77 100644 --- a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Diagnostics; using Tsavorite.core; namespace Garnet.server @@ -12,7 +13,7 @@ namespace Garnet.server sealed partial class StorageSession : IDisposable { /// - /// Returns if key is an existing one in the store. + /// Checks if a key exists in the unified store context. /// /// /// The name of the key to use in the operation @@ -31,5 +32,19 @@ public GarnetStatus EXISTS(PinnedSpanByte key, ref TUnifiedCont return status; } + + /// + /// Deletes a key from the unified store context. + /// + /// The name of the key to use in the operation + /// Basic unifiedContext for the unified store. + /// + public GarnetStatus DELETE(PinnedSpanByte key, ref TUnifiedContext unifiedContext) + where TUnifiedContext : ITsavoriteContext + { + var status = unifiedContext.Delete(key.ReadOnlySpan); + Debug.Assert(!status.IsPending); + return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; + } } } diff --git a/main/GarnetServer/Extensions/SampleDeleteTxn.cs b/main/GarnetServer/Extensions/SampleDeleteTxn.cs index 01866fdd820..5e1c47998d3 100644 --- a/main/GarnetServer/Extensions/SampleDeleteTxn.cs +++ b/main/GarnetServer/Extensions/SampleDeleteTxn.cs @@ -57,7 +57,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p var mainStoreKey = GetNextArg(ref procInput, ref offset); - api.DELETE(mainStoreKey, StoreType.Main); + api.DELETE(mainStoreKey); var sortedSet1Key = GetNextArg(ref procInput, ref offset); var sortedSet1Entry = GetNextArg(ref procInput, ref offset); diff --git a/test/Garnet.test/DeleteTxn.cs b/test/Garnet.test/DeleteTxn.cs index 3c31b965fef..cc522e471e3 100644 --- a/test/Garnet.test/DeleteTxn.cs +++ b/test/Garnet.test/DeleteTxn.cs @@ -27,7 +27,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p { var offset = 0; var key = GetNextArg(ref procInput.parseState, ref offset); - api.DELETE(key, StoreType.Main); + api.DELETE(key); WriteSimpleString(ref output, "SUCCESS"); } } From a446bac55a8bdf3cc584813e8c3934e5dab766a4 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 18 Sep 2025 22:10:44 -0700 Subject: [PATCH 09/28] wip - expire & persist (broken) --- .../Functions/UnifiedStore/PrivateMethods.cs | 111 +++++++++++ .../Functions/UnifiedStore/RMWMethods.cs | 173 ++++++++++++++++-- 2 files changed, 270 insertions(+), 14 deletions(-) diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs index a579790ff38..e07ccd37a23 100644 --- a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -2,6 +2,9 @@ // Licensed under the MIT license. using System; +using System.Diagnostics; +using Garnet.common; +using Microsoft.Extensions.Logging; using Tsavorite.core; namespace Garnet.server @@ -23,5 +26,113 @@ void WriteLogDelete(ReadOnlySpan key, long version, int sessionID) functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.UnifiedStoreDelete, storeVersion = version, sessionID = sessionID }, key, item2: default, out _); } + + /// + /// Logging RMW from + /// a. PostInitialUpdater + /// b. InPlaceUpdater + /// c. PostCopyUpdater + /// + void WriteLogRMW(ReadOnlySpan key, ref UnifiedStoreInput input, long version, int sessionId) + { + if (functionsState.StoredProcMode) return; + input.header.flags |= RespInputFlags.Deterministic; + + functionsState.appendOnlyFile.Enqueue( + new AofHeader { opType = AofEntryType.UnifiedStoreRMW, storeVersion = version, sessionID = sessionId }, + key, ref input, out _); + } + + bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ExpireOption optionType, long newExpiry, ReadOnlySpan newValue, ref GarnetUnifiedStoreOutput output) + { + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + + // TODO ETag? + if (!logRecord.TrySetValueSpan(newValue, in sizeInfo)) + { + functionsState.logger?.LogError("Failed to set value in {methodName}", "EvaluateExpireCopyUpdate"); + writer.WriteZero(); + return false; + } + + return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, writer); + } + + bool EvaluateObjectExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetUnifiedStoreOutput output) + { + Debug.Assert(output.SpanByteAndMemory.IsSpanByte, "This code assumes it is called in-place and did not go pending"); + + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, writer); + } + + bool TrySetRecordExpiration(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, RespMemoryWriter writer) + { + var expiryExists = logRecord.Info.HasExpiration; + + if (expiryExists) + { + // Expiration already exists so there is no need to check for space (i.e. failure of TrySetExpiration) + switch (optionType) + { + case ExpireOption.NX: + writer.WriteZero(); + return true; + case ExpireOption.XX: + case ExpireOption.None: + _ = logRecord.TrySetExpiration(newExpiry); + writer.WriteOne(); + return true; + case ExpireOption.GT: + case ExpireOption.XXGT: + if (newExpiry > logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); + writer.WriteOne(); + return true; + } + writer.WriteZero(); + return true; + case ExpireOption.LT: + case ExpireOption.XXLT: + if (newExpiry < logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); + writer.WriteOne(); + return true; + } + writer.WriteZero(); + return true; + default: + throw new GarnetException($"EvaluateExpireCopyUpdate exception when expiryExists is false: optionType{optionType}"); + } + } + else + { + // No expiration yet. Because this is CopyUpdate we should already have verified the space, but check anyway + switch (optionType) + { + case ExpireOption.NX: + case ExpireOption.None: + case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite + if (!logRecord.TrySetExpiration(newExpiry)) + { + functionsState.logger?.LogError("Failed to add expiration in {methodName}.{caseName}", "EvaluateExpireCopyUpdate", "LT"); + writer.WriteZero(); + return false; + } + writer.WriteOne(); + return true; + case ExpireOption.XX: + case ExpireOption.GT: + case ExpireOption.XXGT: + case ExpireOption.XXLT: + writer.WriteZero(); + return true; + default: + throw new GarnetException($"EvaluateExpireCopyUpdate exception when expiryExists is true: optionType{optionType}"); + } + } + } } } diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs index 8daf2e6dc9d..3a8cf8a334c 100644 --- a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -2,6 +2,8 @@ // Licensed under the MIT license. using System; +using System.IO; +using Garnet.common; using Tsavorite.core; namespace Garnet.server @@ -12,31 +14,174 @@ namespace Garnet.server public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions { public bool NeedInitialUpdate(ReadOnlySpan key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, - ref RMWInfo rmwInfo) => - throw new NotImplementedException(); + ref RMWInfo rmwInfo) + { + return input.header.cmd switch + { + RespCommand.PERSIST or + RespCommand.EXPIRE or + RespCommand.EXPIREAT or + RespCommand.PEXPIRE or + RespCommand.PEXPIREAT => false, + _ => true + }; + } public bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => - throw new NotImplementedException(); + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) + { + return input.header.cmd switch + { + RespCommand.PERSIST or + RespCommand.EXPIRE or + RespCommand.EXPIREAT or + RespCommand.PEXPIRE or + RespCommand.PEXPIREAT => throw new Exception(), + _ => true + }; + } public void PostInitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => - throw new NotImplementedException(); + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) + { + functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + { + input.header.SetExpiredFlag(); + WriteLogRMW(logRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); + } + + if (logRecord.Info.ValueIsObject) + { + functionsState.objectStoreSizeTracker?.AddTrackedSize(logRecord.ValueObject.MemorySize); + } + } public bool NeedCopyUpdate(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input, - ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => true; public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, - in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, + ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + { + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) + { + rmwInfo.Action = RMWAction.ExpireAndResume; + return false; + } + + if (srcLogRecord.Info.ValueIsObject) return true; + + var cmd = input.header.cmd; + + var result = cmd switch + { + RespCommand.EXPIRE => HandleExpire(srcLogRecord, ref dstLogRecord, in sizeInfo, ref input, ref output, ref rmwInfo), + RespCommand.PERSIST => HandlePersist(srcLogRecord, ref dstLogRecord, in sizeInfo, ref input, ref output, ref rmwInfo), + _ => throw new NotImplementedException() + }; + + if (!result) + return false; + + sizeInfo.AssertOptionals(dstLogRecord.Info); + return true; + } public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, - in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, + ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + { + functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); + + if (srcLogRecord.Info.ValueIsObject) + { + // We're performing the object update here (and not in CopyUpdater) so that we are guaranteed that + // the record was CASed into the hash chain before it gets modified + var oldValueSize = srcLogRecord.ValueObject.MemorySize; + var value = ((IGarnetObject)srcLogRecord.ValueObject).CopyUpdate(srcLogRecord.Info.IsInNewVersion, + ref rmwInfo); + + // First copy the new Value and optionals to the new record. This will also ensure space for expiration if it's present. + // Do not set actually set dstLogRecord.Expiration until we know it is a command for which we allocated length in the LogRecord for it. + if (!dstLogRecord.TrySetValueObject(value, in sizeInfo)) + return false; + + var cmd = input.header.cmd; + switch (cmd) + { + case RespCommand.EXPIRE: + var expirationWithOption = new ExpirationWithOption(input.arg1); + + // Expire will have allocated space for the expiration, so copy it over and do the "in-place" logic to replace it in the new record + if (srcLogRecord.Info.HasExpiration) + dstLogRecord.TrySetExpiration(srcLogRecord.Expiration); + if (!EvaluateObjectExpireInPlace(ref dstLogRecord, expirationWithOption.ExpireOption, + expirationWithOption.ExpirationTimeInTicks, ref output)) + return false; + break; + + case RespCommand.PERSIST: + if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) + return false; + if (srcLogRecord.Info.HasExpiration) + { + dstLogRecord.RemoveExpiration(); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); + } + else + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); + + break; + } + + sizeInfo.AssertOptionals(dstLogRecord.Info); + + // If oldValue has been set to null, subtract its size from the tracked heap size + var sizeAdjustment = rmwInfo.ClearSourceValueObject ? value.MemorySize - oldValueSize : value.MemorySize; + functionsState.objectStoreSizeTracker?.AddTrackedSize(sizeAdjustment); + } + + if (functionsState.appendOnlyFile != null) + WriteLogRMW(dstLogRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); + + return true; + } public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) => - throw new NotImplementedException(); + ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) + { + + } + + private bool HandleExpire(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, + ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + { + var expirationWithOption = new ExpirationWithOption(input.arg1); + + // First copy the old Value and non-Expiration optionals to the new record. This will also ensure space for expiration. + if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) + return false; + + return EvaluateExpireCopyUpdate(ref dstLogRecord, in sizeInfo, expirationWithOption.ExpireOption, + expirationWithOption.ExpirationTimeInTicks, dstLogRecord.ValueSpan, ref output); + } + + private bool HandlePersist(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, + in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, + ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + { + if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) + return false; + if (srcLogRecord.Info.HasExpiration) + { + dstLogRecord.RemoveExpiration(); + using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); + writer.WriteOne(); + } + + return true; + } } } From 92cab870489fd4b0c5f4c683ef5f525eeb3262d8 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Mon, 22 Sep 2025 13:38:35 -0700 Subject: [PATCH 10/28] wip - adding expire to unified ops --- libs/server/API/GarnetApi.cs | 3 - libs/server/API/GarnetApiUnifiedCommands.cs | 8 + libs/server/API/IGarnetApi.cs | 5 +- libs/server/Resp/KeyAdminCommands.cs | 13 +- .../Functions/UnifiedStore/DeleteMethods.cs | 1 + .../Functions/UnifiedStore/PrivateMethods.cs | 39 +++- .../Functions/UnifiedStore/RMWMethods.cs | 146 ++++++++++++- ...unctions.cs => UnifiedSessionFunctions.cs} | 0 .../Functions/UnifiedStore/UpsertMethods.cs | 201 ++++++++++++++++-- .../UnifiedStore/VarLenInputMethods.cs | 125 ++++++++++- .../Session/UnifiedStore/AdvancedOps.cs | 18 +- 11 files changed, 507 insertions(+), 52 deletions(-) rename libs/server/Storage/Functions/UnifiedStore/{UnifiedStoreFunctions.cs => UnifiedSessionFunctions.cs} (100%) diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index a921edf67cf..739eb85171e 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -170,9 +170,6 @@ public GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out i #endregion #region EXPIRE - /// - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All) - => storageSession.EXPIRE(key, ref input, out timeoutSet, storeType, ref context, ref objectContext); /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index 74e7d0f8bd6..64a0373064f 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -67,5 +67,13 @@ public GarnetStatus DELETE(PinnedSpanByte key) => storageSession.DELETE(key, ref unifiedContext); #endregion + + #region EXPIRE + + /// + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext); + + #endregion } } diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 98827bdbd97..3dd190a0eb0 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -185,10 +185,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// /// Key /// - /// Whether timeout was set by the call - /// Store type: main, object, or both + /// /// - GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All); + GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); /// /// Set a timeout on key using a timeSpan in seconds diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index 8c3a998958f..78f661300af 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -459,13 +459,16 @@ private bool NetworkEXPIRE(RespCommand command, ref TGarnetApi stora // Encode expiration time and expiration option and pass them into the input object var expirationWithOption = new ExpirationWithOption(expirationTimeInTicks, expireOption); - var input = new RawStringInput(RespCommand.EXPIRE, arg1: expirationWithOption.Word); - var status = storageApi.EXPIRE(key, ref input, out var timeoutSet); + var input = new UnifiedStoreInput(RespCommand.EXPIRE, arg1: expirationWithOption.Word); - if (status == GarnetStatus.OK && timeoutSet) + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + var status = storageApi.EXPIRE(key, ref input, ref output); + + if (status == GarnetStatus.OK) { - while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_RETURN_VAL_1, ref dcurr, dend)) - SendAndReset(); + ProcessOutput(output.SpanByteAndMemory); } else { diff --git a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs index b40c2dd6d2f..fd62eba8460 100644 --- a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs @@ -28,6 +28,7 @@ public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) logRecord.ClearOptionals(); if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); + if (functionsState.appendOnlyFile != null) WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs index e07ccd37a23..13bb8201dd0 100644 --- a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -14,6 +14,43 @@ namespace Garnet.server /// public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions { + /// + /// Logging upsert from + /// a. InPlaceWriter + /// b. PostInitialWriter + /// + void WriteLogUpsert(ReadOnlySpan key, ref UnifiedStoreInput input, ReadOnlySpan value, long version, int sessionID) + { + if (functionsState.StoredProcMode) + return; + + input.header.flags |= RespInputFlags.Deterministic; + + functionsState.appendOnlyFile.Enqueue( + new AofHeader { opType = AofEntryType.UnifiedStoreUpsert, storeVersion = version, sessionID = sessionID }, + key, value, out _); + } + + /// + /// Logging upsert from + /// a. InPlaceWriter + /// b. PostInitialWriter + /// + void WriteLogUpsert(ReadOnlySpan key, ref UnifiedStoreInput input, IGarnetObject value, long version, int sessionID) + { + if (functionsState.StoredProcMode) + return; + input.header.flags |= RespInputFlags.Deterministic; + + GarnetObjectSerializer.Serialize(value, out var valueBytes); + fixed (byte* valPtr = valueBytes) + { + functionsState.appendOnlyFile.Enqueue( + new AofHeader { opType = AofEntryType.UnifiedStoreUpsert, storeVersion = version, sessionID = sessionID }, + key, new ReadOnlySpan(valPtr, valueBytes.Length), out _); + } + } + /// /// Logging Delete from /// a. InPlaceDeleter @@ -58,7 +95,7 @@ bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, in RecordSizeInfo sizeInf return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, writer); } - bool EvaluateObjectExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetUnifiedStoreOutput output) + bool EvaluateExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetUnifiedStoreOutput output) { Debug.Assert(output.SpanByteAndMemory.IsSpanByte, "This code assumes it is called in-place and did not go pending"); diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs index 3a8cf8a334c..a43eda8caa6 100644 --- a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -2,7 +2,7 @@ // Licensed under the MIT license. using System; -using System.IO; +using System.Diagnostics; using Garnet.common; using Tsavorite.core; @@ -30,6 +30,9 @@ RespCommand.PEXPIRE or public bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) { + Debug.Assert(logRecord.Info.ValueIsObject || (!logRecord.Info.HasETag && !logRecord.Info.HasExpiration), + "Should not have Expiration or ETag on InitialUpdater log records"); + return input.header.cmd switch { RespCommand.PERSIST or @@ -64,26 +67,53 @@ public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord { + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) { + if (!srcLogRecord.Info.ValueIsObject) + { + _ = dstLogRecord.RemoveETag(); + // reset etag state that may have been initialized earlier + ETagState.ResetState(ref functionsState.etagState); + } + rmwInfo.Action = RMWAction.ExpireAndResume; return false; } if (srcLogRecord.Info.ValueIsObject) return true; + var recordHadEtagPreMutation = srcLogRecord.Info.HasETag; + var shouldUpdateEtag = recordHadEtagPreMutation; + if (shouldUpdateEtag) + { + // during checkpointing we might skip the inplace calls and go directly to copy update so we need to initialize here if needed + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, in srcLogRecord); + } + var cmd = input.header.cmd; var result = cmd switch { - RespCommand.EXPIRE => HandleExpire(srcLogRecord, ref dstLogRecord, in sizeInfo, ref input, ref output, ref rmwInfo), - RespCommand.PERSIST => HandlePersist(srcLogRecord, ref dstLogRecord, in sizeInfo, ref input, ref output, ref rmwInfo), + RespCommand.EXPIRE => HandleExpire(srcLogRecord, ref dstLogRecord, in sizeInfo, ref shouldUpdateEtag, ref input, ref output), + RespCommand.PERSIST => HandlePersist(srcLogRecord, ref dstLogRecord, in sizeInfo, ref shouldUpdateEtag, ref output), _ => throw new NotImplementedException() }; if (!result) return false; + if (shouldUpdateEtag) + { + dstLogRecord.TrySetETag(functionsState.etagState.ETag + 1); + ETagState.ResetState(ref functionsState.etagState); + } + else if (recordHadEtagPreMutation) + { + // reset etag state that may have been initialized earlier + ETagState.ResetState(ref functionsState.etagState); + } + sizeInfo.AssertOptionals(dstLogRecord.Info); return true; } @@ -116,7 +146,7 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, // Expire will have allocated space for the expiration, so copy it over and do the "in-place" logic to replace it in the new record if (srcLogRecord.Info.HasExpiration) dstLogRecord.TrySetExpiration(srcLogRecord.Expiration); - if (!EvaluateObjectExpireInPlace(ref dstLogRecord, expirationWithOption.ExpireOption, + if (!EvaluateExpireInPlace(ref dstLogRecord, expirationWithOption.ExpireOption, expirationWithOption.ExpirationTimeInTicks, ref output)) return false; break; @@ -151,13 +181,113 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo) { + if (InPlaceUpdaterWorker(ref logRecord, in sizeInfo, ref input, ref output, ref rmwInfo, out var sizeChange)) + { + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogRMW(logRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); + if (logRecord.Info.ValueIsObject) + functionsState.objectStoreSizeTracker?.AddTrackedSize(sizeChange); + return true; + } + return false; + } + + bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref RMWInfo rmwInfo, out long sizeChange) + { + sizeChange = 0; + + // Expired data + if (logRecord.Info.HasExpiration && input.header.CheckExpiry(logRecord.Expiration)) + { + if (logRecord.Info.ValueIsObject) + { + functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); + + // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); + logRecord.ClearValueObject(_ => { }); + } + else + { + logRecord.RemoveETag(); + } + + rmwInfo.Action = RMWAction.ExpireAndResume; + + return false; + } + + var hadETagPreMutation = logRecord.Info.HasETag; + var shouldUpdateEtag = hadETagPreMutation; + if (shouldUpdateEtag) + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, in logRecord); + var shouldCheckExpiration = true; + + var cmd = input.header.cmd; + switch (cmd) + { + case RespCommand.EXPIRE: + var expirationWithOption = new ExpirationWithOption(input.arg1); + + if (!logRecord.Info.ValueIsObject) + { + // reset etag state that may have been initialized earlier, but don't update etag because only the expiration was updated + ETagState.ResetState(ref functionsState.etagState); + } + + return EvaluateExpireInPlace(ref logRecord, expirationWithOption.ExpireOption, + expirationWithOption.ExpirationTimeInTicks, ref output); + case RespCommand.PERSIST: + if (logRecord.Info.HasExpiration) + { + logRecord.RemoveExpiration(); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); + } + else + { + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory);} + + if (!logRecord.Info.ValueIsObject) + { + // reset etag state that may have been initialized earlier, but don't update etag because only the metadata was updated + ETagState.ResetState(ref functionsState.etagState); + shouldUpdateEtag = false; + } + else + { + return true; + } + break; + default: + throw new NotImplementedException(); + } + + if (!logRecord.Info.ValueIsObject) + { + // increment the Etag transparently if in place update happened + if (shouldUpdateEtag) + { + logRecord.TrySetETag(this.functionsState.etagState.ETag + 1); + ETagState.ResetState(ref functionsState.etagState); + } + else if (hadETagPreMutation) + { + // reset etag state that may have been initialized earlier + ETagState.ResetState(ref functionsState.etagState); + } + } + + sizeInfo.AssertOptionals(logRecord.Info, checkExpiration: shouldCheckExpiration); + return true; } private bool HandleExpire(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, - in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, - ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + in RecordSizeInfo sizeInfo, ref bool shouldUpdateEtag, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) where TSourceLogRecord : ISourceLogRecord { + shouldUpdateEtag = false; var expirationWithOption = new ExpirationWithOption(input.arg1); // First copy the old Value and non-Expiration optionals to the new record. This will also ensure space for expiration. @@ -169,9 +299,9 @@ private bool HandleExpire(in TSourceLogRecord srcLogRecord, re } private bool HandlePersist(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, - in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, - ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord + in RecordSizeInfo sizeInfo, ref bool shouldUpdateEtag, ref GarnetUnifiedStoreOutput output) where TSourceLogRecord : ISourceLogRecord { + shouldUpdateEtag = false; if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) return false; if (srcLogRecord.Info.HasExpiration) diff --git a/libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs b/libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs similarity index 100% rename from libs/server/Storage/Functions/UnifiedStore/UnifiedStoreFunctions.cs rename to libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs diff --git a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs index 616d9f7b394..6a9b0b8a569 100644 --- a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using Garnet.common; using Tsavorite.core; namespace Garnet.server @@ -12,40 +13,200 @@ namespace Garnet.server public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions { public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + { + if (!logRecord.TrySetValueSpan(srcValue, in sizeInfo)) + return false; + if (input.arg1 != 0 && !logRecord.TrySetExpiration(input.arg1)) + return false; + sizeInfo.AssertOptionals(logRecord.Info); + return true; + } public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + { + if (!logRecord.TrySetValueObject(srcValue, in sizeInfo)) + return false; + // TODO ETag + if (input.arg1 != 0 && !logRecord.TrySetExpiration(input.arg1)) + return false; + sizeInfo.AssertOptionals(logRecord.Info); + return true; + } - public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, + ref UnifiedStoreInput input, + in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + if (!logRecord.Info.ValueIsObject) + throw new GarnetException("Unified store should not be called with IHeapObject"); + + return logRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); + } public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + ReadOnlySpan srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + { + if (logRecord.Info.ValueIsObject) + { + // TODO: This is called by readcache directly, but is the only ISessionFunctions call for that; the rest is internal. Clean this up, maybe as a new PostReadCacheInsert method. + if (upsertInfo.Address == LogAddress.kInvalidAddress) + { + functionsState.objectStoreSizeTracker?.AddReadCacheTrackedSize( + MemoryUtils.CalculateHeapMemorySize(in logRecord)); + return; + } + } + + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, srcValue, upsertInfo.Version, upsertInfo.SessionID); + + if (logRecord.Info.ValueIsObject) + { + // TODO: Need to track original length as well, if it was overflow, and add overflow here as well as object size + // TODO: Need to track lengths written to readcache, which is now internal in Tsavorite + functionsState.objectStoreSizeTracker?.AddTrackedSize( + MemoryUtils.CalculateHeapMemorySize(in logRecord)); + } + } public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + IHeapObject srcValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + { + var garnetObject = (IGarnetObject)srcValue; + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, garnetObject, upsertInfo.Version, upsertInfo.SessionID); + + // TODO: Need to track original length as well, if it was overflow, and add overflow here as well as object size + functionsState.objectStoreSizeTracker?.AddTrackedSize(MemoryUtils.CalculateHeapMemorySize(in logRecord)); + } public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, - ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord + { + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + { + WriteLogUpsert(logRecord.Key, ref input, inputLogRecord.ValueSpan, upsertInfo.Version, + upsertInfo.SessionID); + } + + if (logRecord.Info.ValueIsObject) + { + // TODO: Need to track original length as well, if it was overflow, and add overflow here as well as object size + functionsState.objectStoreSizeTracker?.AddTrackedSize(MemoryUtils.CalculateHeapMemorySize(in logRecord)); + } + } public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - ReadOnlySpan newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + ReadOnlySpan newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + + { + if (logRecord.Info.ValueIsObject) + { + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TrySetValueSpan(newValue, in sizeInfo); + if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) + return false; + sizeInfo.AssertOptionals(logRecord.Info); + + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, newValue, upsertInfo.Version, upsertInfo.SessionID); + + // TODO: Need to track original length as well, if it was overflow, and add overflow here as well as object size + if (logRecord.Info.ValueIsOverflow) + functionsState.objectStoreSizeTracker?.AddTrackedSize(newValue.Length - oldSize); + return true; + } + + if (!logRecord.TrySetValueSpan(newValue, in sizeInfo)) + return false; + var ok = input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1); + if (ok) + { + if (input.header.CheckWithETagFlag()) + { + var newETag = functionsState.etagState.ETag + 1; + ok = logRecord.TrySetETag(newETag); + if (ok) + functionsState.CopyRespNumber(newETag, ref output.SpanByteAndMemory); + } + else + ok = logRecord.RemoveETag(); + } + if (ok) + { + sizeInfo.AssertOptionals(logRecord.Info); + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, newValue, upsertInfo.Version, upsertInfo.SessionID); + return true; + } + return false; + } public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - IHeapObject newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) => - throw new NotImplementedException(); + IHeapObject newValue, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + { + var garnetObject = (IGarnetObject)newValue; + + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TrySetValueObject(newValue, in sizeInfo); + if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) + return false; + sizeInfo.AssertOptionals(logRecord.Info); + + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, garnetObject, upsertInfo.Version, upsertInfo.SessionID); + + functionsState.objectStoreSizeTracker?.AddTrackedSize(newValue.MemorySize - oldSize); + return true; + } + + public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, + ref UnifiedStoreInput input, in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); + if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) + return false; + sizeInfo.AssertOptionals(logRecord.Info); + + if (!logRecord.Info.Modified) + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + { + if (!inputLogRecord.Info.ValueIsObject) + WriteLogUpsert(logRecord.Key, ref input, logRecord.ValueSpan, upsertInfo.Version, upsertInfo.SessionID); + else + WriteLogUpsert(logRecord.Key, ref input, (IGarnetObject)logRecord.ValueObject, upsertInfo.Version, upsertInfo.SessionID); + } - public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref UnifiedStoreInput input, - in TSourceLogRecord inputLogRecord, ref GarnetUnifiedStoreOutput output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + var newSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(newSize - oldSize); + return true; + } } } diff --git a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs index a6782242342..fb75a428b76 100644 --- a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs @@ -11,17 +11,126 @@ namespace Garnet.server /// public readonly unsafe partial struct UnifiedSessionFunctions : ISessionFunctions { - public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, + ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord + { + var fieldInfo = new RecordFieldInfo + { + KeyDataSize = srcLogRecord.Key.Length, + ValueDataSize = srcLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : 0, + ValueIsObject = srcLogRecord.Info.ValueIsObject, + HasETag = !srcLogRecord.Info.ValueIsObject && (input.header.CheckWithETagFlag() || srcLogRecord.Info.HasETag), + HasExpiration = srcLogRecord.Info.HasExpiration + }; - public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref UnifiedStoreInput input) => throw new NotImplementedException(); + if (input.header.cmd != RespCommand.NONE) + { + var cmd = input.header.cmd; - public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref UnifiedStoreInput input) => throw new NotImplementedException(); + switch (cmd) + { + case RespCommand.EXPIRE: + case RespCommand.PEXPIRE: + case RespCommand.EXPIREAT: + case RespCommand.PEXPIREAT: + { + // Set HasExpiration to match with EvaluateExpireInPlace. + if (srcLogRecord.Info.HasExpiration) + { + // case ExpireOption.NX: // HasExpiration is true so we will retain it + // case ExpireOption.XX: + // case ExpireOption.None: + // case ExpireOption.GT: + // case ExpireOption.XXGT: + // case ExpireOption.LT: + // case ExpireOption.XXLT: + fieldInfo.HasExpiration = true; // Will update or retain + } + else + { + var expirationWithOption = new ExpirationWithOption(input.arg1); + switch (expirationWithOption.ExpireOption) + { + case ExpireOption.NX: + case ExpireOption.None: + case ExpireOption.LT + : // If expiry doesn't exist, LT should treat the current expiration as infinite, so the new value must be less + fieldInfo.HasExpiration = true; // Will update or retain + break; + default: + // case ExpireOption.XX: + // case ExpireOption.GT: // If expiry doesn't exist, GT should treat the current expiration as infinite, so the new value cannot be greater + // case ExpireOption.XXGT: + // case ExpireOption.XXLT: + fieldInfo.HasExpiration = + false; // Will not add one and there is not one there now + break; + } + } + } - public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref UnifiedStoreInput input) => throw new NotImplementedException(); + if (!srcLogRecord.Info.ValueIsObject) + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + return fieldInfo; + case RespCommand.PERSIST: + fieldInfo.HasExpiration = false; + if (!srcLogRecord.Info.ValueIsObject) + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + return fieldInfo; + default: + throw new NotImplementedException(); + } + } - public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, - ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord => - throw new NotImplementedException(); + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.HasExpiration = input.arg1 != 0; + return fieldInfo; + } + + public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref UnifiedStoreInput input) + { + return new RecordFieldInfo + { + KeyDataSize = key.Length, + ValueDataSize = 0, + HasETag = input.header.CheckWithETagFlag() + }; + } + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, + ref UnifiedStoreInput input) + { + return new RecordFieldInfo + { + KeyDataSize = key.Length, + ValueDataSize = value.Length, + ValueIsObject = false, + HasETag = input.header.CheckWithETagFlag() + }; + } + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref UnifiedStoreInput input) + { + return new RecordFieldInfo + { + KeyDataSize = key.Length, + ValueDataSize = ObjectIdMap.ObjectIdSize, + ValueIsObject = true, + HasETag = false + }; + } + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, + in TSourceLogRecord inputLogRecord, + ref UnifiedStoreInput input) where TSourceLogRecord : ISourceLogRecord + { + return new RecordFieldInfo + { + KeyDataSize = key.Length, + ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, + ValueIsObject = inputLogRecord.Info.ValueIsObject, + HasETag = !inputLogRecord.Info.ValueIsObject && input.header.CheckWithETagFlag() + }; + } } } diff --git a/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs index 5d8dc6840ea..672608985d3 100644 --- a/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs @@ -19,10 +19,20 @@ public GarnetStatus Read_UnifiedStore(ReadOnlySpan key, r if (status.IsPending) CompletePendingForUnifiedStoreSession(ref status, ref output, ref unifiedContext); - if (status.Found) - return GarnetStatus.OK; - else - return GarnetStatus.NOTFOUND; + return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; + } + + public GarnetStatus RMW_UnifiedStore(ReadOnlySpan key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output, ref TUnifiedContext context) + where TUnifiedContext : ITsavoriteContext + { + var status = context.RMW(key, ref input, ref output); + + if (status.IsPending) + CompletePendingForUnifiedStoreSession(ref status, ref output, ref context); + + return status.Found || status.Record.Created || status.Record.InPlaceUpdated + ? GarnetStatus.OK + : GarnetStatus.NOTFOUND; } } } From 6dbeecc4f5561e0cc11096c7cb159c8e69411970 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Mon, 22 Sep 2025 16:00:57 -0700 Subject: [PATCH 11/28] wip - expire --- libs/server/API/GarnetApiObjectCommands.cs | 36 +++++----- libs/server/API/GarnetWatchApi.cs | 14 ++-- libs/server/API/IGarnetApi.cs | 40 +++++------ libs/server/GarnetUnifiedStoreOutput.cs | 5 ++ .../Objects/Types/GarnetObjectStoreOutput.cs | 4 +- libs/server/OutputHeader.cs | 2 +- libs/server/Resp/KeyAdminCommands.cs | 5 +- .../Functions/MainStore/PrivateMethods.cs | 4 +- .../Storage/Functions/MainStore/RMWMethods.cs | 8 +-- .../Functions/ObjectStore/PrivateMethods.cs | 2 +- .../Functions/UnifiedStore/PrivateMethods.cs | 67 ++++++++----------- .../Functions/UnifiedStore/RMWMethods.cs | 11 +-- .../Functions/UnifiedStore/UpsertMethods.cs | 2 + .../UnifiedStore/VarLenInputMethods.cs | 3 - .../Storage/Session/MainStore/MainStoreOps.cs | 8 +-- .../Storage/Session/ObjectStore/Common.cs | 8 +-- .../Storage/Session/ObjectStore/HashOps.cs | 12 ++-- .../Storage/Session/ObjectStore/ListOps.cs | 8 +-- .../Storage/Session/ObjectStore/SetOps.cs | 6 +- .../Session/ObjectStore/SortedSetOps.cs | 8 +-- test/Garnet.test/RespTests.cs | 4 +- 21 files changed, 127 insertions(+), 130 deletions(-) diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs index cdea775016d..2bff58addc1 100644 --- a/libs/server/API/GarnetApiObjectCommands.cs +++ b/libs/server/API/GarnetApiObjectCommands.cs @@ -45,7 +45,7 @@ public GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members => storageSession.SortedSetRemove(key, members, out zaddCount, ref objectContext); /// - public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SortedSetRemove(key, ref input, out output, ref objectContext); /// @@ -53,7 +53,7 @@ public GarnetStatus SortedSetLength(PinnedSpanByte key, out int len) => storageSession.SortedSetLength(key, out len, ref objectContext); /// - public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SortedSetLength(key, ref input, out output, ref objectContext); /// @@ -89,11 +89,11 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, re => storageSession.SortedSetCount(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SortedSetLengthByValue(key, ref input, out output, ref objectContext); /// - public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SortedSetRemoveRangeByLex(key, ref input, out output, ref objectContext); /// @@ -239,7 +239,7 @@ public GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte[] elements, => storageSession.ListPush(key, elements, whenExists ? ListOperation.RPUSHX : ListOperation.RPUSH, out itemsCount, ref objectContext); /// - public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.ListPush(key, ref input, out output, ref objectContext); /// @@ -251,7 +251,7 @@ public GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte element, out => storageSession.ListPush(key, element, onlyWhenExists ? ListOperation.LPUSHX : ListOperation.LPUSH, out count, ref objectContext); /// - public GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.ListPush(key, ref input, out output, ref objectContext); /// @@ -297,7 +297,7 @@ public GarnetStatus ListLength(PinnedSpanByte key, out int count) => storageSession.ListLength(key, ref objectContext, out count); /// - public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.ListLength(key, ref input, out output, ref objectContext); /// @@ -317,7 +317,7 @@ public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref Gar => storageSession.ListRange(key, ref input, ref output, ref objectContext); /// - public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.ListInsert(key, ref input, out output, ref objectContext); /// @@ -325,7 +325,7 @@ public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref Gar => storageSession.ListIndex(key, ref input, ref output, ref objectContext); /// - public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.ListRemove(key, ref input, out output, ref objectContext); /// @@ -345,7 +345,7 @@ public GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int => storageSession.SetAdd(key, members, out saddCount, ref objectContext); /// - public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SetAdd(key, ref input, out output, ref objectContext); /// @@ -357,7 +357,7 @@ public GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out => storageSession.SetRemove(key, members, out sremCount, ref objectContext); /// - public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SetRemove(key, ref input, out output, ref objectContext); /// @@ -365,7 +365,7 @@ public GarnetStatus SetLength(PinnedSpanByte key, out int count) => storageSession.SetLength(key, out count, ref objectContext); /// - public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.SetLength(key, ref input, out output, ref objectContext); /// @@ -452,7 +452,7 @@ public GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpa => storageSession.HashSet(key, elements, out count, ref objectContext); /// - public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.HashSet(key, ref input, out output, ref objectContext); /// @@ -492,11 +492,11 @@ public GarnetStatus HashLength(PinnedSpanByte key, out int count) => storageSession.HashLength(key, out count, ref objectContext); /// - public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.HashLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.HashStrLength(key, ref input, out output, ref objectContext); /// @@ -504,7 +504,7 @@ public GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out boo => storageSession.HashExists(key, field, out exists, ref objectContext); /// - public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.HashExists(key, ref input, out output, ref objectContext); /// @@ -520,7 +520,7 @@ public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, r => storageSession.HashRandomField(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) => storageSession.HashDelete(key, ref input, out output, ref objectContext); /// @@ -532,7 +532,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref Garn => storageSession.HashVals(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output) + public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out OutputHeader output) => storageSession.HashIncrement(key, input, out output, ref objectContext); /// diff --git a/libs/server/API/GarnetWatchApi.cs b/libs/server/API/GarnetWatchApi.cs index 5be3170b96f..617ea5849ba 100644 --- a/libs/server/API/GarnetWatchApi.cs +++ b/libs/server/API/GarnetWatchApi.cs @@ -99,7 +99,7 @@ public GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount) } /// - public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetLength(key, ref input, out output); @@ -120,7 +120,7 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, re } /// - public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetLengthByValue(key, ref input, out output); @@ -263,7 +263,7 @@ public GarnetStatus ListLength(PinnedSpanByte key, out int count) } /// - public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.ListLength(key, ref input, out output); @@ -295,7 +295,7 @@ public GarnetStatus SetLength(PinnedSpanByte key, out int scardCount) } /// - public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetLength(key, ref input, out output); @@ -454,14 +454,14 @@ public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, r } /// - public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashStrLength(key, ref input, out output); } /// - public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashExists(key, ref input, out output); @@ -482,7 +482,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref Garn } /// - public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashLength(key, ref input, out output); diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 3dd190a0eb0..deae876cf21 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -397,7 +397,7 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// /// /// - GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Removes all elements in the sorted set between the @@ -407,7 +407,7 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// /// /// - GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Removes and returns the first element from the sorted set stored at key, @@ -648,7 +648,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Removes the specified member from the set. @@ -681,7 +681,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Removes and returns one random member from the set at key. @@ -780,13 +780,13 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// - /// ListLeftPush ArgSlice version with ObjectOutputHeader output + /// ListLeftPush ArgSlice version with OutputHeader output /// /// /// /// /// - GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// ListLeftPush ArgSlice version, one element @@ -809,13 +809,13 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int count, bool whenExists = false); /// - /// ListRightPush ArgSlice version with ObjectOutputHeader output + /// ListRightPush ArgSlice version with OutputHeader output /// /// /// /// /// - public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// ListRightPush ArgSlice version, one element @@ -952,7 +952,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Removes the first count occurrences of elements equal to element from the list. @@ -961,7 +961,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Sets the list element at index to element. @@ -1006,7 +1006,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Set only if field does not yet exist. If key does not exist, a new key holding a hash is created. @@ -1045,7 +1045,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Increments the number stored at field in the hash key by increment parameter. @@ -1054,7 +1054,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r /// /// /// - GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output); + GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out OutputHeader output); /// /// Increments the number stored at field representing a floating point value @@ -1289,7 +1289,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Returns the specified range of elements in the sorted set stored at key. @@ -1350,7 +1350,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// ZRANK: Returns the rank of member in the sorted set, the scores in the sorted set are ordered from low to high @@ -1514,7 +1514,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Gets the specified elements of the list stored at key. @@ -1553,7 +1553,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// SMEMBERS key @@ -1709,7 +1709,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Returns the number of fields contained in the hash Key. @@ -1718,7 +1718,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Returns if field is an existing field in the hash stored at key. @@ -1736,7 +1736,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output); /// /// Returns count random fields from the hash value. diff --git a/libs/server/GarnetUnifiedStoreOutput.cs b/libs/server/GarnetUnifiedStoreOutput.cs index 4d14a411a48..18e12464659 100644 --- a/libs/server/GarnetUnifiedStoreOutput.cs +++ b/libs/server/GarnetUnifiedStoreOutput.cs @@ -17,6 +17,11 @@ public struct GarnetUnifiedStoreOutput /// public SpanByteAndMemory SpanByteAndMemory; + /// + /// Output header + /// + public OutputHeader Header; + /// /// Output flags /// diff --git a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs index bb593656c45..856fa5225b5 100644 --- a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs +++ b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs @@ -23,9 +23,9 @@ public struct GarnetObjectStoreOutput public IGarnetObject GarnetObject; /// - /// Object header + /// Output header /// - public ObjectOutputHeader Header; + public OutputHeader Header; /// /// Output flags diff --git a/libs/server/OutputHeader.cs b/libs/server/OutputHeader.cs index a5bbcf73202..05c1d4ee95d 100644 --- a/libs/server/OutputHeader.cs +++ b/libs/server/OutputHeader.cs @@ -32,7 +32,7 @@ public enum OutputFlags : byte /// Object output header (sometimes used as footer) /// [StructLayout(LayoutKind.Explicit, Size = Size)] - public struct ObjectOutputHeader + public struct OutputHeader { /// /// Expected size of this struct diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index 78f661300af..4212e71401d 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -466,9 +466,10 @@ private bool NetworkEXPIRE(RespCommand command, ref TGarnetApi stora var status = storageApi.EXPIRE(key, ref input, ref output); - if (status == GarnetStatus.OK) + if (status == GarnetStatus.OK && ((OutputHeader*)output.SpanByteAndMemory.SpanByte.ToPointer())->result1 == 1) { - ProcessOutput(output.SpanByteAndMemory); + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_RETURN_VAL_1, ref dcurr, dend)) + SendAndReset(); } else { diff --git a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs index f5bfe64b4b1..53098a397de 100644 --- a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs @@ -303,7 +303,7 @@ void CopyRespToWithInput(in TSourceLogRecord srcLogRecord, ref bool EvaluateExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref SpanByteAndMemory output) { - var o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); + var o = (OutputHeader*)output.SpanByte.ToPointer(); o->result1 = 0; if (logRecord.Info.HasExpiration) { @@ -360,7 +360,7 @@ bool EvaluateExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, lon bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ExpireOption optionType, long newExpiry, ReadOnlySpan newValue, ref SpanByteAndMemory output) { var expiryExists = logRecord.Info.HasExpiration; - var o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); + var o = (OutputHeader*)output.SpanByte.ToPointer(); o->result1 = 0; // TODO ETag? diff --git a/libs/server/Storage/Functions/MainStore/RMWMethods.cs b/libs/server/Storage/Functions/MainStore/RMWMethods.cs index 4a3baa051b3..732a27793c8 100644 --- a/libs/server/Storage/Functions/MainStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/MainStore/RMWMethods.cs @@ -745,8 +745,8 @@ private readonly bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSiz // If both EX and PERSIST were specified, EX wins if (input.arg1 > 0) { - var pbOutput = stackalloc byte[ObjectOutputHeader.Size]; - var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); + var pbOutput = stackalloc byte[OutputHeader.Size]; + var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, OutputHeader.Size)); var newExpiry = input.arg1; if (!EvaluateExpireInPlace(ref logRecord, ExpireOption.None, newExpiry, ref _output)) @@ -1396,8 +1396,8 @@ public readonly bool CopyUpdater(in TSourceLogRecord srcLogRec Debug.Assert(newValue.Length == oldValue.Length); if (input.arg1 > 0) { - var pbOutput = stackalloc byte[ObjectOutputHeader.Size]; - var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); + var pbOutput = stackalloc byte[OutputHeader.Size]; + var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, OutputHeader.Size)); var newExpiry = input.arg1; if (!EvaluateExpireCopyUpdate(ref dstLogRecord, in sizeInfo, ExpireOption.None, newExpiry, newValue, ref _output)) return false; diff --git a/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs b/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs index 5e8c937ec39..debde0ea049 100644 --- a/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs @@ -82,7 +82,7 @@ void WriteLogDelete(ReadOnlySpan key, long version, int sessionID) static bool EvaluateObjectExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetObjectStoreOutput output) { Debug.Assert(output.SpanByteAndMemory.IsSpanByte, "This code assumes it is called in-place and did not go pending"); - var o = (ObjectOutputHeader*)output.SpanByteAndMemory.SpanByte.ToPointer(); + var o = (OutputHeader*)output.SpanByteAndMemory.SpanByte.ToPointer(); o->result1 = 0; if (logRecord.Info.HasExpiration) { diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs index 13bb8201dd0..2e4cab431e1 100644 --- a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -82,29 +82,27 @@ void WriteLogRMW(ReadOnlySpan key, ref UnifiedStoreInput input, long versi bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ExpireOption optionType, long newExpiry, ReadOnlySpan newValue, ref GarnetUnifiedStoreOutput output) { - using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); - // TODO ETag? if (!logRecord.TrySetValueSpan(newValue, in sizeInfo)) { functionsState.logger?.LogError("Failed to set value in {methodName}", "EvaluateExpireCopyUpdate"); - writer.WriteZero(); return false; } - return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, writer); + return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, ref output); } bool EvaluateExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetUnifiedStoreOutput output) { Debug.Assert(output.SpanByteAndMemory.IsSpanByte, "This code assumes it is called in-place and did not go pending"); - using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); - return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, writer); + return TrySetRecordExpiration(ref logRecord, optionType, newExpiry, ref output); } - bool TrySetRecordExpiration(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, RespMemoryWriter writer) + bool TrySetRecordExpiration(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref GarnetUnifiedStoreOutput output) { + var o = (OutputHeader*)output.SpanByteAndMemory.SpanByte.ToPointer(); + o->result1 = 0; var expiryExists = logRecord.Info.HasExpiration; if (expiryExists) @@ -113,62 +111,53 @@ bool TrySetRecordExpiration(ref LogRecord logRecord, ExpireOption optionType, lo switch (optionType) { case ExpireOption.NX: - writer.WriteZero(); return true; case ExpireOption.XX: case ExpireOption.None: _ = logRecord.TrySetExpiration(newExpiry); - writer.WriteOne(); + o->result1 = 1; return true; case ExpireOption.GT: case ExpireOption.XXGT: if (newExpiry > logRecord.Expiration) { _ = logRecord.TrySetExpiration(newExpiry); - writer.WriteOne(); - return true; + o->result1 = 1; } - writer.WriteZero(); return true; case ExpireOption.LT: case ExpireOption.XXLT: if (newExpiry < logRecord.Expiration) { _ = logRecord.TrySetExpiration(newExpiry); - writer.WriteOne(); - return true; + o->result1 = 1; } - writer.WriteZero(); return true; default: throw new GarnetException($"EvaluateExpireCopyUpdate exception when expiryExists is false: optionType{optionType}"); } } - else + + // No expiration yet. Because this is CopyUpdate we should already have verified the space, but check anyway + switch (optionType) { - // No expiration yet. Because this is CopyUpdate we should already have verified the space, but check anyway - switch (optionType) - { - case ExpireOption.NX: - case ExpireOption.None: - case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite - if (!logRecord.TrySetExpiration(newExpiry)) - { - functionsState.logger?.LogError("Failed to add expiration in {methodName}.{caseName}", "EvaluateExpireCopyUpdate", "LT"); - writer.WriteZero(); - return false; - } - writer.WriteOne(); - return true; - case ExpireOption.XX: - case ExpireOption.GT: - case ExpireOption.XXGT: - case ExpireOption.XXLT: - writer.WriteZero(); - return true; - default: - throw new GarnetException($"EvaluateExpireCopyUpdate exception when expiryExists is true: optionType{optionType}"); - } + case ExpireOption.NX: + case ExpireOption.None: + case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite + if (!logRecord.TrySetExpiration(newExpiry)) + { + functionsState.logger?.LogError("Failed to add expiration in {methodName}.{caseName}", "EvaluateExpireCopyUpdate", "LT"); + return false; + } + o->result1 = 1; + return true; + case ExpireOption.XX: + case ExpireOption.GT: + case ExpireOption.XXGT: + case ExpireOption.XXLT: + return true; + default: + throw new GarnetException($"EvaluateExpireCopyUpdate exception when expiryExists is true: optionType{optionType}"); } } } diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs index a43eda8caa6..bfec4d53294 100644 --- a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -3,7 +3,6 @@ using System; using System.Diagnostics; -using Garnet.common; using Tsavorite.core; namespace Garnet.server @@ -154,6 +153,7 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, case RespCommand.PERSIST: if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) return false; + if (srcLogRecord.Info.HasExpiration) { dstLogRecord.RemoveExpiration(); @@ -247,8 +247,7 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); } else - { - functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory);} + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); if (!logRecord.Info.ValueIsObject) { @@ -304,12 +303,14 @@ private bool HandlePersist(in TSourceLogRecord srcLogRecord, r shouldUpdateEtag = false; if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) return false; + if (srcLogRecord.Info.HasExpiration) { dstLogRecord.RemoveExpiration(); - using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); - writer.WriteOne(); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); } + else + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); return true; } diff --git a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs index 6a9b0b8a569..934760c5bac 100644 --- a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs @@ -139,7 +139,9 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r var newETag = functionsState.etagState.ETag + 1; ok = logRecord.TrySetETag(newETag); if (ok) + { functionsState.CopyRespNumber(newETag, ref output.SpanByteAndMemory); + } } else ok = logRecord.RemoveETag(); diff --git a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs index fb75a428b76..79b60cb5976 100644 --- a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs @@ -30,9 +30,6 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe switch (cmd) { case RespCommand.EXPIRE: - case RespCommand.PEXPIRE: - case RespCommand.EXPIREAT: - case RespCommand.PEXPIREAT: { // Set HasExpiration to match with EvaluateExpireInPlace. if (srcLogRecord.Info.HasExpiration) diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 9ab10906e5a..c35bb0bcbfd 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -782,7 +782,7 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, where TContext : ITsavoriteContext where TObjectContext : ITsavoriteContext { - Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; + Span rmwOutput = stackalloc byte[OutputHeader.Size]; var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); timeoutSet = false; @@ -819,7 +819,7 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, } Debug.Assert(output.IsSpanByte); - if (found) timeoutSet = ((ObjectOutputHeader*)output.SpanByte.ToPointer())->result1 == 1; + if (found) timeoutSet = ((OutputHeader*)output.SpanByte.ToPointer())->result1 == 1; return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } @@ -887,7 +887,7 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, where TContext : ITsavoriteContext where TObjectContext : ITsavoriteContext { - Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; + Span rmwOutput = stackalloc byte[OutputHeader.Size]; var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); timeoutSet = false; var found = false; @@ -932,7 +932,7 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, } Debug.Assert(output.IsSpanByte); - if (found) timeoutSet = ((ObjectOutputHeader*)output.SpanByte.ToPointer())->result1 == 1; + if (found) timeoutSet = ((OutputHeader*)output.SpanByte.ToPointer())->result1 == 1; return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } diff --git a/libs/server/Storage/Session/ObjectStore/Common.cs b/libs/server/Storage/Session/ObjectStore/Common.cs index 7cc939fcaf1..7fdcaedf59e 100644 --- a/libs/server/Storage/Session/ObjectStore/Common.cs +++ b/libs/server/Storage/Session/ObjectStore/Common.cs @@ -19,7 +19,7 @@ sealed partial class StorageSession : IDisposable { #region Common ObjectStore Methods - unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) @@ -36,7 +36,7 @@ unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan k } unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, - out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) @@ -714,7 +714,7 @@ unsafe bool TryProcessRespSimple64IntOutput(GarnetObjectStoreOutput output, out /// /// /// - unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) @@ -750,7 +750,7 @@ unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan /// /// /// - unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) diff --git a/libs/server/Storage/Session/ObjectStore/HashOps.cs b/libs/server/Storage/Session/ObjectStore/HashOps.cs index 496d2b17740..5fba189b75d 100644 --- a/libs/server/Storage/Session/ObjectStore/HashOps.cs +++ b/libs/server/Storage/Session/ObjectStore/HashOps.cs @@ -376,7 +376,7 @@ public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, i /// /// /// - public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -444,7 +444,7 @@ public GarnetStatus HashRandomField(PinnedSpanByte key, ref Obje /// /// /// - public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -457,7 +457,7 @@ public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInp /// /// /// - public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -470,7 +470,7 @@ public GarnetStatus HashStrLength(PinnedSpanByte key, ref Object /// /// /// - public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -483,7 +483,7 @@ public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInp /// /// /// - public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -522,7 +522,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput /// /// /// - public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectContext) + public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out OutputHeader output, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, input, out output, ref objectContext); diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index fe0aa3de127..76b247c1921 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -355,7 +355,7 @@ public unsafe bool ListTrim(PinnedSpanByte key, int start, int s /// /// /// - public GarnetStatus ListPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus ListPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -413,7 +413,7 @@ public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInpu /// /// /// - public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext { var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -444,7 +444,7 @@ public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInpu /// /// /// - public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -472,7 +472,7 @@ public unsafe GarnetStatus ListPop(PinnedSpanByte key, ref Objec /// /// /// - public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index eb246b5b939..ae06623ad39 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -651,7 +651,7 @@ private GarnetStatus SetUnion(PinnedSpanByte[] keys, ref TObject /// /// /// - public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); @@ -666,7 +666,7 @@ public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput i /// /// /// - public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); @@ -679,7 +679,7 @@ public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInpu /// /// /// - public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index 42a53093a6b..5558d730bf6 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -799,7 +799,7 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds /// /// /// - public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -812,7 +812,7 @@ public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref Obje /// /// /// - public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); @@ -936,7 +936,7 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref Objec /// /// /// - public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); @@ -951,7 +951,7 @@ public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key /// /// /// - public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output, ref TObjectContext objectStoreContext) where TObjectContext : ITsavoriteContext => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index 8d2392c940f..9c91a771fe6 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -2553,7 +2553,9 @@ public void KeyExpireStringTest(string command) ClassicAssert.AreEqual(key, (string)value); if (command.Equals("EXPIRE")) - db.KeyExpire(key, TimeSpan.FromSeconds(1)); + { + var res = db.KeyExpire(key, TimeSpan.FromSeconds(1)); + } else db.Execute(command, [key, 1000]); From 36bf62fe3d7ac237a7ea915c814f57c69b0bbf2a Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 22 Sep 2025 17:20:14 -0700 Subject: [PATCH 12/28] add cref to server-side replication inter-node commands --- .../GarnetClientSessionClusterExtensions.cs | 69 ------------------- ...arnetClientSessionReplicationExtensions.cs | 6 ++ 2 files changed, 6 insertions(+), 69 deletions(-) delete mode 100644 libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs diff --git a/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs b/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs deleted file mode 100644 index 0b64944783c..00000000000 --- a/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Threading; -using System.Threading.Tasks; -using Garnet.common; -using Garnet.networking; - -namespace Garnet.client -{ - /// - /// Mono-threaded remote client session for Garnet (a session makes a single network connection, and - /// expects mono-threaded client access, i.e., no concurrent invocations of API by client) - /// - public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageConsumer - { - static ReadOnlySpan GOSSIP => "GOSSIP"u8; - - /// - /// Send gossip message to corresponding node - /// - /// - /// - public Task ExecuteGossip(Memory byteArray) - { - var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - tcsQueue.Enqueue(tcs); - byte* curr = offset; - byte* next = offset; - int arraySize = 3; - - while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end)) - { - Flush(); - curr = offset; - } - offset = curr; - - //1 - while (!RespWriteUtils.TryWriteDirect(CLUSTER, ref curr, end)) - { - Flush(); - curr = offset; - } - offset = curr; - - //2 - while (!RespWriteUtils.TryWriteBulkString(GOSSIP, ref curr, end)) - { - Flush(); - curr = offset; - } - offset = curr; - - //3 - while (!RespWriteUtils.TryWriteBulkString(byteArray.Span, ref curr, end)) - { - Flush(); - curr = offset; - } - offset = curr; - - Flush(); - Interlocked.Increment(ref numCommands); - return tcs.Task; - } - } -} \ No newline at end of file diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs index da8c2a9c6ff..3beac1a77ff 100644 --- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs @@ -31,6 +31,7 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo /// /// /// + /// public Task ExecuteReplicaSync(string nodeId, string primary_replid, byte[] checkpointEntryData, long aofBeginAddress, long aofTailAddress) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); @@ -112,6 +113,7 @@ public Task ExecuteReplicaSync(string nodeId, string primary_replid, byt /// /// /// + /// public Task ExecuteSendCkptMetadata(Memory fileTokenBytes, int fileType, Memory data) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); @@ -178,6 +180,7 @@ public Task ExecuteSendCkptMetadata(Memory fileTokenBytes, int fil /// /// /// + /// public Task ExecuteSendFileSegments(Memory fileTokenBytes, int fileType, long startAddress, Span data, int segmentId = -1) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); @@ -263,6 +266,7 @@ public Task ExecuteSendFileSegments(Memory fileTokenBytes, int fil /// /// /// + /// public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool replayAOF, string primary_replid, byte[] checkpointEntryData, long beginAddress, long tailAddress) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); @@ -351,6 +355,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool re /// /// /// + /// public Task ExecuteAttachSync(byte[] syncMetadata) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); @@ -399,6 +404,7 @@ public Task ExecuteAttachSync(byte[] syncMetadata) /// /// /// + /// public void SetClusterSyncHeader(string sourceNodeId, bool isMainStore) { // Unlike Migration, where we don't know at the time of header initialization if we have a record or not, in Replication From 5ed62043d05992ba084c83c7b3cab92e1b9a9674 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 22 Sep 2025 17:53:01 -0700 Subject: [PATCH 13/28] fix server-side BeginRecoverReplica --- .../ReplicaOps/ReplicaReceiveCheckpoint.cs | 4 +--- .../Session/RespClusterReplicationCommands.cs | 14 ++++++-------- libs/server/Databases/DatabaseManagerBase.cs | 3 +-- libs/server/Databases/IDatabaseManager.cs | 3 +-- libs/server/Databases/MultiDatabaseManager.cs | 2 +- libs/server/Databases/SingleDatabaseManager.cs | 4 ++-- libs/server/StoreWrapper.cs | 5 ++--- 7 files changed, 14 insertions(+), 21 deletions(-) diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs index 081beb6b377..b4b51c80c0a 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs @@ -254,16 +254,15 @@ IDevice GetStoreHLogDevice() /// Process request from primary to start recovery process from the retrieved checkpoint. /// /// - /// /// /// /// /// /// + /// /// public long BeginReplicaRecover( bool recoverMainStoreFromToken, - bool recoverObjectStoreFromToken, bool replayAOF, string primaryReplicationId, CheckpointEntry remoteCheckpoint, @@ -284,7 +283,6 @@ public long BeginReplicaRecover( storeWrapper.RecoverCheckpoint( replicaRecover: true, recoverMainStoreFromToken, - recoverObjectStoreFromToken, remoteCheckpoint.metadata); if (replayAOF) diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index 619445b210c..9d44d2b2d31 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -348,26 +348,25 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters) invalidParameters = false; // Expecting exactly 7 arguments - if (parseState.Count != 7) + if (parseState.Count != 6) { invalidParameters = true; return true; } if (!parseState.TryGetBool(0, out var recoverMainStoreFromToken) || - !parseState.TryGetBool(1, out var recoverObjectStoreFromToken) || - !parseState.TryGetBool(2, out var replayAOF)) + !parseState.TryGetBool(1, out var replayAOF)) { while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_BOOLEAN, ref dcurr, dend)) SendAndReset(); return true; } - var primaryReplicaId = parseState.GetString(3); - var checkpointEntryBytes = parseState.GetArgSliceByRef(4).ToArray(); + var primaryReplicaId = parseState.GetString(2); + var checkpointEntryBytes = parseState.GetArgSliceByRef(3).ToArray(); - if (!parseState.TryGetLong(5, out var beginAddress) || - !parseState.TryGetLong(6, out var tailAddress)) + if (!parseState.TryGetLong(4, out var beginAddress) || + !parseState.TryGetLong(5, out var tailAddress)) { while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend)) SendAndReset(); @@ -377,7 +376,6 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters) var entry = CheckpointEntry.FromByteArray(checkpointEntryBytes); var replicationOffset = clusterProvider.replicationManager.BeginReplicaRecover( recoverMainStoreFromToken, - recoverObjectStoreFromToken, replayAOF, primaryReplicaId, entry, diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index d1f63d21e15..2a2d75a2fad 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -37,8 +37,7 @@ internal abstract class DatabaseManagerBase : IDatabaseManager public abstract void ResumeCheckpoints(int dbId); /// - public abstract void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, - bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null); + public abstract void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null); /// public abstract bool TakeCheckpoint(bool background, ILogger logger = null, CancellationToken token = default); diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs index d5e6c4aa5d4..709db854b1c 100644 --- a/libs/server/Databases/IDatabaseManager.cs +++ b/libs/server/Databases/IDatabaseManager.cs @@ -85,9 +85,8 @@ public interface IDatabaseManager : IDisposable /// /// /// - /// /// - public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null); + public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null); /// /// Take checkpoint of all active databases if checkpointing is not in progress diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs index 4b7670609d9..9759e4a4dc6 100644 --- a/libs/server/Databases/MultiDatabaseManager.cs +++ b/libs/server/Databases/MultiDatabaseManager.cs @@ -87,7 +87,7 @@ public MultiDatabaseManager(SingleDatabaseManager src) : } /// - public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) + public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null) { if (replicaRecover) throw new GarnetException( diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs index 1fa136f33e8..a7570118a2c 100644 --- a/libs/server/Databases/SingleDatabaseManager.cs +++ b/libs/server/Databases/SingleDatabaseManager.cs @@ -53,7 +53,7 @@ public override GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, o } /// - public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) + public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null) { long storeVersion = 0; try @@ -173,7 +173,7 @@ public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime, if (storeTailAddress.HasValue) defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value; - + defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow; } finally diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index cd824e16b87..49291bdc1e8 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -391,9 +391,8 @@ public async Task TakeOnDemandCheckpoint(DateTimeOffset entryTime, int dbId = 0) /// /// Recover checkpoint /// - public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, - bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null) - => databaseManager.RecoverCheckpoint(replicaRecover, recoverMainStoreFromToken, recoverObjectStoreFromToken, metadata); + public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null) + => databaseManager.RecoverCheckpoint(replicaRecover, recoverMainStoreFromToken, metadata); /// /// Mark the beginning of a checkpoint by taking and a lock to avoid concurrent checkpointing From 9a05c222a00fb0d71515cf96e1e526673460cf6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Paulus=20P=C3=A4rssinen?= Date: Tue, 23 Sep 2025 20:02:32 +0300 Subject: [PATCH 14/28] Add link to new 'Unsafe code best practices' -article (#1386) --- website/docs/dev/onboarding.md | 61 ++++++++++++++++++---------------- 1 file changed, 32 insertions(+), 29 deletions(-) diff --git a/website/docs/dev/onboarding.md b/website/docs/dev/onboarding.md index e787356a646..dfce20d40c0 100644 --- a/website/docs/dev/onboarding.md +++ b/website/docs/dev/onboarding.md @@ -33,12 +33,12 @@ For an introduction to Garnet and its capabilities, you can start with [Welcome ### Start hacking -1. Clone the repository - -```bash -git clone https://github.com/microsoft/garnet.git -``` - +1. Clone the repository + +```bash +git clone https://github.com/microsoft/garnet.git +``` + After cloning the repository you can either run the unit tests or run the server and use one of the RESP client suggested in Windows or Linux. 2. Run the tests suite @@ -47,13 +47,13 @@ After cloning the repository you can either run the unit tests or run the server dotnet test -c Release -l "console;verbosity=detailed" ``` -3. Run the server - -Using a size memory of 4 GB and index size of 64 MB: - -```bash -cd /main/GarnetServer/ -dotnet run -c Debug -f net8.0 -- --logger-level Trace -m 4g -i 64m +3. Run the server + +Using a size memory of 4 GB and index size of 64 MB: + +```bash +cd /main/GarnetServer/ +dotnet run -c Debug -f net8.0 -- --logger-level Trace -m 4g -i 64m ``` 4. Use the Memurai client in Windows to send commands to Garnet. A guide about how to install Memurai on Windows can be found [here](https://docs.memurai.com/en/installation.html). @@ -62,16 +62,16 @@ dotnet run -c Debug -f net8.0 -- --logger-level Trace -m 4g -i 64m 6. A third option is to install Redis-Insight on Windows. Follow the official guide [here](https://redis.com/redis-enterprise/redis-insight/#insight-form). -## Troubleshooting - -1. If you need to use TLS in Linux, follow the guide at: - - `/Garnet/test/testcerts/README.md` - -2. If you need to run the local device library, make sure to have these dependencies: - - ```bash - sudo apt install -y g++ libaio-dev uuid-dev libtbb-dev +## Troubleshooting + +1. If you need to use TLS in Linux, follow the guide at: + + `/Garnet/test/testcerts/README.md` + +2. If you need to run the local device library, make sure to have these dependencies: + + ```bash + sudo apt install -y g++ libaio-dev uuid-dev libtbb-dev ``` ## Garnet API development @@ -104,6 +104,8 @@ RESP representation: **Tsavorite** and **Garnet** rely heavily on these two types for allocating data in memory and then transfer it on the network layer. Understanding and familiarity with both of them will be very helpful for a better understanding of the code in general. * [Documentation about Span](https://learn.microsoft.com/en-us/dotnet/api/system.span-1?view=net-7.0) + + * [Unsafe code best practices](https://learn.microsoft.com/en-us/dotnet/standard/unsafe-code/best-practices) * [Use of pointers and unsafe code](https://learn.microsoft.com/en-us/dotnet/csharp/language-reference/unsafe-code) @@ -150,12 +152,12 @@ Any new feature, change to existing functionality or bug fixing needs to be done /// /// Iterates the set of keys in the main store. /// -/// The pattern to apply for filtering -/// When true the filter is omitted -/// The value of the cursor in the command request -/// Value of the cursor returned -/// The list of keys from the stores -/// The size of the batch of keys +/// The pattern to apply for filtering +/// When true the filter is omitted +/// The value of the cursor in the command request +/// Value of the cursor returned +/// The list of keys from the stores +/// The size of the batch of keys /// Type of key to filter out /// public bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out long storeCursor, out List Keys, long count = 10, Span type = default); @@ -188,3 +190,4 @@ Note that Tsavorite has its own solution file and test suite in the folder ` Date: Tue, 23 Sep 2025 11:12:02 -0700 Subject: [PATCH 15/28] [Bugfix] AOF Replay Double Replaying Finalize (#1372) * AOF Finalize Double Replay Fix * stupid whitespace * fmt * minor comment --------- Co-authored-by: Hamdaan Khalid Co-authored-by: Badrish Chandramouli --- libs/server/AOF/AofProcessor.cs | 2 +- libs/server/Custom/CustomRespCommands.cs | 4 +- .../Custom/CustomTransactionProcedure.cs | 2 + libs/server/Transaction/TransactionManager.cs | 12 ++++-- .../Garnet.test/AofFinalizeDoubleReplayTxn.cs | 28 +++++++++++++ test/Garnet.test/RespAofTests.cs | 39 +++++++++++++++++++ 6 files changed, 81 insertions(+), 6 deletions(-) create mode 100644 test/Garnet.test/AofFinalizeDoubleReplayTxn.cs diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index 2406e37244a..f4c21c5e7f5 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -373,7 +373,7 @@ void RunStoredProc(byte id, CustomProcedureInput customProcInput, byte* ptr) customProcInput.DeserializeFrom(curr); // Run the stored procedure with the reconstructed input - respServerSession.RunTransactionProc(id, ref customProcInput, ref output); + respServerSession.RunTransactionProc(id, ref customProcInput, ref output, isRecovering: true); } } diff --git a/libs/server/Custom/CustomRespCommands.cs b/libs/server/Custom/CustomRespCommands.cs index 87e5402c4ff..950e27a6e1d 100644 --- a/libs/server/Custom/CustomRespCommands.cs +++ b/libs/server/Custom/CustomRespCommands.cs @@ -53,11 +53,11 @@ private bool TryTransactionProc(byte id, CustomTransactionProcedure proc, int st return true; } - public bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, ref MemoryResult output) + public bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, ref MemoryResult output, bool isRecovering = false) { var proc = customCommandManagerSession .GetCustomTransactionProcedure(id, this, txnManager, scratchBufferAllocator, out _); - return txnManager.RunTransactionProc(id, ref procInput, proc, ref output); + return txnManager.RunTransactionProc(id, ref procInput, proc, ref output, isRecovering); } private void TryCustomProcedure(CustomProcedure proc, int startIdx = 0) diff --git a/libs/server/Custom/CustomTransactionProcedure.cs b/libs/server/Custom/CustomTransactionProcedure.cs index 451a39a2865..216364cae08 100644 --- a/libs/server/Custom/CustomTransactionProcedure.cs +++ b/libs/server/Custom/CustomTransactionProcedure.cs @@ -76,6 +76,8 @@ public abstract void Main(TGarnetApi api, ref CustomProcedureInput p /// /// Finalize transaction: runs after the transactions commits/aborts, allowed to read and write (non-transactionally) with per-key locks and produce output + /// NOTE: Finalize is considered post transaction processing and therefore is not executed at recovery time. Instead, the individual Tsavorite commands are logged and replayed through the AOF. + /// If you are not using AOF for persistence then this is implementation detail you can ignore. /// public virtual void Finalize(TGarnetApi api, ref CustomProcedureInput procInput, ref MemoryResult output) where TGarnetApi : IGarnetApi diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 6d7cb41faf7..adfe7975ab6 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -175,7 +175,7 @@ internal void Reset(bool isRunning) this.keyCount = 0; } - internal bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, CustomTransactionProcedure proc, ref MemoryResult output) + internal bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, CustomTransactionProcedure proc, ref MemoryResult output, bool isRecovering = false) { var running = false; scratchBufferAllocator.Reset(); @@ -227,8 +227,14 @@ internal bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, Cu { try { - // Run finalize procedure at the end - proc.Finalize(garnetTxFinalizeApi, ref procInput, ref output); + // Run finalize procedure at the end. + // If the transaction was invoked during AOF replay skip the finalize step altogether + // Finalize logs to AOF accordingly, so let the replay pick up the commits from AOF as + // part of normal AOF replay. + if (!isRecovering) + { + proc.Finalize(garnetTxFinalizeApi, ref procInput, ref output); + } } catch { } diff --git a/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs b/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs new file mode 100644 index 00000000000..ce33ffa35c7 --- /dev/null +++ b/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs @@ -0,0 +1,28 @@ +using Garnet.common; +using Garnet.server; +using Tsavorite.core; + +namespace Garnet.test +{ + // Test transaction used to make sure we are not doing double replay of items enqueued to AOF at finalize step + // AOFFINDOUBLEREP KEY + public class AofFinalizeDoubleReplayTxn : CustomTransactionProcedure + { + public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) + { + int offset = 0; + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); + return true; + } + public override void Main(TGarnetApi api, ref CustomProcedureInput procInput, ref MemoryResult output) + { + // No-op + } + + public override void Finalize(TGarnetApi api, ref CustomProcedureInput procInput, ref MemoryResult output) + { + int offset = 0; + api.Increment(GetNextArg(ref procInput, ref offset), out _); + } + } +} \ No newline at end of file diff --git a/test/Garnet.test/RespAofTests.cs b/test/Garnet.test/RespAofTests.cs index 15c13651a38..098ced7d5a4 100644 --- a/test/Garnet.test/RespAofTests.cs +++ b/test/Garnet.test/RespAofTests.cs @@ -830,6 +830,45 @@ public void AofCustomTxnRecoverTest() } } + // Tests that the transaction's finalize step is currently written once into AOF, and replayed twice during replay + [Test] + public void AofTransactionFinalizeStepTest() + { + const string txnName = "AOFFINDOUBLEREP"; + const string key = "key1"; + server.Dispose(false); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, enableAOF: true); + server.Register.NewTransactionProc(txnName, () => new AofFinalizeDoubleReplayTxn(), new RespCommandsInfo { Arity = 2 }); + server.Start(); + + int resultPostTxn = 0; + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db = redis.GetDatabase(0); + db.Execute(txnName, key); + + var res = db.StringGet(key); + resultPostTxn = (int)res; + } + + // so now commit AOF, kill server and force a replay + server.Store.CommitAOF(true); + server.Dispose(false); + + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, tryRecover: true, enableAOF: true); + server.Register.NewTransactionProc(txnName, () => new AofFinalizeDoubleReplayTxn(), new RespCommandsInfo { Arity = 2 }); + server.Start(); + + // post replay we should end up at the exact state. This test should break right away because currently we do double replay + using (var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig())) + { + var db = redis.GetDatabase(0); + var res = db.StringGet(key); + int resultAfterRecovery = (int)res; + ClassicAssert.AreEqual(resultPostTxn, resultAfterRecovery); + } + } + private static void ExpectedEtagTest(IDatabase db, string key, string expectedValue, long expected) { RedisResult res = db.Execute("GETWITHETAG", key); From 3ac77e82428014512359a7c6af82c42fbd51aff5 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Tue, 23 Sep 2025 11:49:32 -0700 Subject: [PATCH 16/28] Refactoring TxnKeyManager to use Key Specifications (#1381) * wip * wip * wip * wip * wip * wip * Updating info + docs jsons * Simplifying TKM * wip * test fix * format * bugfixes * some fixes * wip * tests fix * format * small rename --- libs/resources/RespCommandsDocs.json | 216 +++-- libs/resources/RespCommandsInfo.json | 682 ++++++++----- libs/server/Cluster/StoreType.cs | 5 + libs/server/Resp/BasicCommands.cs | 71 +- .../Resp/RespCommandInfoSimplifiedStructs.cs | 264 +++++ libs/server/Resp/RespCommandsInfo.cs | 47 +- libs/server/SessionParseStateExtensions.cs | 171 +++- libs/server/Transaction/TxnKeyManager.cs | 526 +--------- libs/server/Transaction/TxnRespCommands.cs | 15 +- .../CommandInfoUpdater/CommandDocsUpdater.cs | 2 +- .../CommandInfoUpdater/CommandInfoUpdater.cs | 30 +- playground/CommandInfoUpdater/CommonUtils.cs | 4 +- .../GarnetCommandsDocs.json | 901 ++++++++++++++---- .../GarnetCommandsInfo.json | 291 +++++- .../RespCommandInfoParser.cs | 8 +- .../CommandInfoUpdater/SupportedCommand.cs | 388 ++++---- test/Garnet.test/CustomRespCommandsInfo.json | 131 ++- test/Garnet.test/RespCommandTests.cs | 133 ++- 18 files changed, 2473 insertions(+), 1412 deletions(-) create mode 100644 libs/server/Resp/RespCommandInfoSimplifiedStructs.cs diff --git a/libs/resources/RespCommandsDocs.json b/libs/resources/RespCommandsDocs.json index e4d86c25bb6..be77703a3ed 100644 --- a/libs/resources/RespCommandsDocs.json +++ b/libs/resources/RespCommandsDocs.json @@ -41,13 +41,14 @@ { "Command": "ACL_GENPASS", "Name": "ACL|GENPASS", - "Summary": "Generate a pseudorandom secure password to use for ACL users", + "Summary": "Generates a pseudorandom, secure password that can be used to identify ACL users.", "Group": "Server", "Complexity": "O(1)", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "BITS", + "DisplayText": "bits", "Type": "Integer", "ArgumentFlags": "Optional" } @@ -127,21 +128,6 @@ } ] }, - { - "Command": "EXPDELSCAN", - "Name": "EXPDELSCAN", - "Summary": "Scans mutable records to delete expired ones.", - "Group": "Generic", - "Complexity": "O(1) for every call. O(N) for a complete iteration. N is the number of records in memory.", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "DBID", - "DisplayText": "dbid", - "Type": "Integer" - } - ] - }, { "Command": "APPEND", "Name": "APPEND", @@ -1383,6 +1369,13 @@ } ] }, + { + "Command": "CLUSTER_FLUSHALL", + "Name": "CLUSTER|FLUSHALL", + "Summary": "Sent by primary to replica to force to FLUSH its database", + "Group": "Cluster", + "Complexity": "O(1)" + }, { "Command": "CLUSTER_FORGET", "Name": "CLUSTER|FORGET", @@ -1925,22 +1918,6 @@ } ] }, - { - "Command": "DELIFEXPIM", - "Name": "DELIFEXPIM", - "Summary": "Deletes a key if it is in memory and expired.", - "Group": "Generic", - "Complexity": "O(1)", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandKeyArgument", - "Name": "KEY", - "DisplayText": "key", - "Type": "Key", - "KeySpecIndex": 0 - } - ] - }, { "Command": "DELIFGREATER", "Name": "DELIFGREATER", @@ -2097,6 +2074,21 @@ } ] }, + { + "Command": "EXPDELSCAN", + "Name": "EXPDELSCAN", + "Summary": "Scans mutable records to delete expired ones.", + "Group": "Generic", + "Complexity": "O(1) for every call. O(N) for a complete iteration. N is the number of records in memory.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer" + } + ] + }, { "Command": "EXPIRE", "Name": "EXPIRE", @@ -2536,7 +2528,7 @@ { "Command": "GEORADIUS", "Name": "GEORADIUS", - "Summary": "Query a sorted set representing a geospatial index to fetch members matching a given maximum distance from a point", + "Summary": "Queries a geospatial index for members within a distance from a coordinate, optionally stores the result.", "Group": "Geo", "Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.", "DocFlags": "Deprecated", @@ -2545,22 +2537,26 @@ { "TypeDiscriminator": "RespCommandKeyArgument", "Name": "KEY", + "DisplayText": "key", "Type": "Key", "KeySpecIndex": 0 }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "LONGITUDE", + "DisplayText": "longitude", "Type": "Double" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "LATITUDE", + "DisplayText": "latitude", "Type": "Double" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "RADIUS", + "DisplayText": "radius", "Type": "Double" }, { @@ -2571,24 +2567,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "M", + "DisplayText": "m", "Type": "PureToken", "Token": "M" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "KM", + "DisplayText": "km", "Type": "PureToken", "Token": "KM" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "FT", + "DisplayText": "ft", "Type": "PureToken", "Token": "FT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MI", + "DisplayText": "mi", "Type": "PureToken", "Token": "MI" } @@ -2597,6 +2597,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHCOORD", + "DisplayText": "withcoord", "Type": "PureToken", "Token": "WITHCOORD", "ArgumentFlags": "Optional" @@ -2604,6 +2605,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHDIST", + "DisplayText": "withdist", "Type": "PureToken", "Token": "WITHDIST", "ArgumentFlags": "Optional" @@ -2611,25 +2613,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHHASH", + "DisplayText": "withhash", "Type": "PureToken", "Token": "WITHHASH", "ArgumentFlags": "Optional" }, { "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "COUNT", + "Name": "COUNT-BLOCK", "Type": "Block", "ArgumentFlags": "Optional", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "COUNT", + "DisplayText": "count", "Type": "Integer", "Token": "COUNT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ANY", + "DisplayText": "any", "Type": "PureToken", "Token": "ANY", "ArgumentFlags": "Optional" @@ -2645,39 +2650,49 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ASC", + "DisplayText": "asc", "Type": "PureToken", "Token": "ASC" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "DESC", + "DisplayText": "desc", "Type": "PureToken", "Token": "DESC" } ] }, { - "TypeDiscriminator": "RespCommandKeyArgument", - "Name": "KEY", - "Type": "Key", - "Token": "STORE", - "ArgumentFlags": "Optional", - "KeySpecIndex": 1 - }, - { - "TypeDiscriminator": "RespCommandKeyArgument", - "Name": "KEY", - "Type": "Key", - "Token": "STOREDIST", + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "STORE", + "Type": "OneOf", "ArgumentFlags": "Optional", - "KeySpecIndex": 2 + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "STOREKEY", + "DisplayText": "key", + "Type": "Key", + "Token": "STORE", + "KeySpecIndex": 1 + }, + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "STOREDISTKEY", + "DisplayText": "key", + "Type": "Key", + "Token": "STOREDIST", + "KeySpecIndex": 2 + } + ] } ] }, { "Command": "GEORADIUS_RO", "Name": "GEORADIUS_RO", - "Summary": "A read-only variant for GEORADIUS", + "Summary": "Returns members from a geospatial index that are within a distance from a coordinate.", "Group": "Geo", "Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.", "DocFlags": "Deprecated", @@ -2686,22 +2701,26 @@ { "TypeDiscriminator": "RespCommandKeyArgument", "Name": "KEY", + "DisplayText": "key", "Type": "Key", "KeySpecIndex": 0 }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "LONGITUDE", + "DisplayText": "longitude", "Type": "Double" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "LATITUDE", + "DisplayText": "latitude", "Type": "Double" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "RADIUS", + "DisplayText": "radius", "Type": "Double" }, { @@ -2712,24 +2731,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "M", + "DisplayText": "m", "Type": "PureToken", "Token": "M" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "KM", + "DisplayText": "km", "Type": "PureToken", "Token": "KM" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "FT", + "DisplayText": "ft", "Type": "PureToken", "Token": "FT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MI", + "DisplayText": "mi", "Type": "PureToken", "Token": "MI" } @@ -2738,6 +2761,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHCOORD", + "DisplayText": "withcoord", "Type": "PureToken", "Token": "WITHCOORD", "ArgumentFlags": "Optional" @@ -2745,6 +2769,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHDIST", + "DisplayText": "withdist", "Type": "PureToken", "Token": "WITHDIST", "ArgumentFlags": "Optional" @@ -2752,25 +2777,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHHASH", + "DisplayText": "withhash", "Type": "PureToken", "Token": "WITHHASH", "ArgumentFlags": "Optional" }, { "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "COUNT", + "Name": "COUNT-BLOCK", "Type": "Block", "ArgumentFlags": "Optional", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "COUNT", + "DisplayText": "count", "Type": "Integer", "Token": "COUNT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ANY", + "DisplayText": "any", "Type": "PureToken", "Token": "ANY", "ArgumentFlags": "Optional" @@ -2786,12 +2814,14 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ASC", + "DisplayText": "asc", "Type": "PureToken", "Token": "ASC" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "DESC", + "DisplayText": "desc", "Type": "PureToken", "Token": "DESC" } @@ -2802,7 +2832,7 @@ { "Command": "GEORADIUSBYMEMBER", "Name": "GEORADIUSBYMEMBER", - "Summary": "Query a sorted set representing a geospatial index to fetch members matching a given maximum distance from a member", + "Summary": "Queries a geospatial index for members within a distance from a member, optionally stores the result.", "Group": "Geo", "Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.", "DocFlags": "Deprecated", @@ -2811,17 +2841,20 @@ { "TypeDiscriminator": "RespCommandKeyArgument", "Name": "KEY", + "DisplayText": "key", "Type": "Key", "KeySpecIndex": 0 }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MEMBER", + "DisplayText": "member", "Type": "String" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "RADIUS", + "DisplayText": "radius", "Type": "Double" }, { @@ -2832,24 +2865,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "M", + "DisplayText": "m", "Type": "PureToken", "Token": "M" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "KM", + "DisplayText": "km", "Type": "PureToken", "Token": "KM" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "FT", + "DisplayText": "ft", "Type": "PureToken", "Token": "FT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MI", + "DisplayText": "mi", "Type": "PureToken", "Token": "MI" } @@ -2858,6 +2895,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHCOORD", + "DisplayText": "withcoord", "Type": "PureToken", "Token": "WITHCOORD", "ArgumentFlags": "Optional" @@ -2865,6 +2903,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHDIST", + "DisplayText": "withdist", "Type": "PureToken", "Token": "WITHDIST", "ArgumentFlags": "Optional" @@ -2872,25 +2911,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHHASH", + "DisplayText": "withhash", "Type": "PureToken", "Token": "WITHHASH", "ArgumentFlags": "Optional" }, { "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "COUNT", + "Name": "COUNT-BLOCK", "Type": "Block", "ArgumentFlags": "Optional", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "COUNT", + "DisplayText": "count", "Type": "Integer", "Token": "COUNT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ANY", + "DisplayText": "any", "Type": "PureToken", "Token": "ANY", "ArgumentFlags": "Optional" @@ -2906,39 +2948,49 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ASC", + "DisplayText": "asc", "Type": "PureToken", "Token": "ASC" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "DESC", + "DisplayText": "desc", "Type": "PureToken", "Token": "DESC" } ] }, { - "TypeDiscriminator": "RespCommandKeyArgument", - "Name": "KEY", - "Type": "Key", - "Token": "STORE", - "ArgumentFlags": "Optional", - "KeySpecIndex": 1 - }, - { - "TypeDiscriminator": "RespCommandKeyArgument", - "Name": "KEY", - "Type": "Key", - "Token": "STOREDIST", + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "STORE", + "Type": "OneOf", "ArgumentFlags": "Optional", - "KeySpecIndex": 2 + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "STOREKEY", + "DisplayText": "key", + "Type": "Key", + "Token": "STORE", + "KeySpecIndex": 1 + }, + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "STOREDISTKEY", + "DisplayText": "key", + "Type": "Key", + "Token": "STOREDIST", + "KeySpecIndex": 2 + } + ] } ] }, { "Command": "GEORADIUSBYMEMBER_RO", "Name": "GEORADIUSBYMEMBER_RO", - "Summary": "A read-only variant for GEORADIUSBYMEMBER", + "Summary": "Returns members from a geospatial index that are within a distance from a member.", "Group": "Geo", "Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.", "DocFlags": "Deprecated", @@ -2947,17 +2999,20 @@ { "TypeDiscriminator": "RespCommandKeyArgument", "Name": "KEY", + "DisplayText": "key", "Type": "Key", "KeySpecIndex": 0 }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MEMBER", + "DisplayText": "member", "Type": "String" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "RADIUS", + "DisplayText": "radius", "Type": "Double" }, { @@ -2968,24 +3023,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "M", + "DisplayText": "m", "Type": "PureToken", "Token": "M" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "KM", + "DisplayText": "km", "Type": "PureToken", "Token": "KM" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "FT", + "DisplayText": "ft", "Type": "PureToken", "Token": "FT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "MI", + "DisplayText": "mi", "Type": "PureToken", "Token": "MI" } @@ -2994,6 +3053,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHCOORD", + "DisplayText": "withcoord", "Type": "PureToken", "Token": "WITHCOORD", "ArgumentFlags": "Optional" @@ -3001,6 +3061,7 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHDIST", + "DisplayText": "withdist", "Type": "PureToken", "Token": "WITHDIST", "ArgumentFlags": "Optional" @@ -3008,25 +3069,28 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "WITHHASH", + "DisplayText": "withhash", "Type": "PureToken", "Token": "WITHHASH", "ArgumentFlags": "Optional" }, { "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "COUNT", + "Name": "COUNT-BLOCK", "Type": "Block", "ArgumentFlags": "Optional", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "COUNT", + "DisplayText": "count", "Type": "Integer", "Token": "COUNT" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ANY", + "DisplayText": "any", "Type": "PureToken", "Token": "ANY", "ArgumentFlags": "Optional" @@ -3042,12 +3106,14 @@ { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "ASC", + "DisplayText": "asc", "Type": "PureToken", "Token": "ASC" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "DESC", + "DisplayText": "desc", "Type": "PureToken", "Token": "DESC" } @@ -4536,14 +4602,6 @@ "Type": "Integer", "Token": "COUNT", "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "NOVALUES", - "DisplayText": "novalues", - "Type": "PureToken", - "Token": "NOVALUES", - "ArgumentFlags": "Optional" } ] }, @@ -6933,7 +6991,8 @@ "Name": "NOGET", "DisplayText": "noget", "Type": "PureToken", - "Token": "NOGET" + "Token": "NOGET", + "ArgumentFlags": "Optional" } ] }, @@ -6990,7 +7049,8 @@ "Name": "NOGET", "DisplayText": "noget", "Type": "PureToken", - "Token": "NOGET" + "Token": "NOGET", + "ArgumentFlags": "Optional" } ] }, @@ -7562,18 +7622,20 @@ { "Command": "SWAPDB", "Name": "SWAPDB", - "Summary": "Swaps two Memurai databases", + "Summary": "Swaps two Redis 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", + "DisplayText": "index1", "Type": "Integer" }, { "TypeDiscriminator": "RespCommandBasicArgument", "Name": "INDEX2", + "DisplayText": "index2", "Type": "Integer" } ] diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index f381ba1548f..daa1acd29d3 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -23,6 +23,20 @@ "response_policy:all_succeeded" ] }, + { + "Command": "ACL_GENPASS", + "Name": "ACL|GENPASS", + "Arity": -2, + "Flags": "Loading, NoScript, Stale", + "AclCategories": "Slow" + }, + { + "Command": "ACL_GETUSER", + "Name": "ACL|GETUSER", + "Arity": 3, + "Flags": "Admin, Loading, NoScript, Stale", + "AclCategories": "Admin, Dangerous, Slow" + }, { "Command": "ACL_LIST", "Name": "ACL|LIST", @@ -72,33 +86,9 @@ "Arity": 2, "Flags": "Loading, NoScript, Stale", "AclCategories": "Slow" - }, - { - "Command": "ACL_GETUSER", - "Name": "ACL|GETUSER", - "Arity": 3, - "Flags": "Admin, Loading, NoScript, Stale", - "AclCategories": "Admin, Dangerous, Slow" - }, - { - "Command": "ACL_GENPASS", - "Name": "ACL|GENPASS", - "Arity": -2, - "Flags": "Loading, NoScript, Stale", - "AclCategories": "Slow" } ] }, - { - "Command": "EXPDELSCAN", - "Name": "EXPDELSCAN", - "Arity": -1, - "Flags": "Admin, NoMulti, NoScript, ReadOnly", - "FirstKey": 1, - "LastKey": 1, - "Step": 1, - "AclCategories": "Admin, Garnet" - }, { "Command": "APPEND", "Name": "APPEND", @@ -122,7 +112,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Main" }, { "Command": "ASKING", @@ -175,7 +166,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "BITFIELD", @@ -201,7 +193,8 @@ "Notes": "This command allows both access and modification of the key", "Flags": "RW, Access, Update, VariableFlags" } - ] + ], + "StoreType": "Main" }, { "Command": "BITFIELD_RO", @@ -226,7 +219,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "BITOP", @@ -264,7 +258,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "BITPOS", @@ -289,7 +284,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "BLMOVE", @@ -327,7 +323,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "BLMPOP", @@ -349,7 +346,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "BLPOP", @@ -374,7 +372,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "BRPOP", @@ -399,7 +398,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "BRPOPLPUSH", @@ -437,7 +437,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "BZMPOP", @@ -459,7 +460,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "BZPOPMAX", @@ -484,7 +486,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "BZPOPMIN", @@ -509,7 +512,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "CLIENT", @@ -727,6 +731,14 @@ "Flags": "Admin, NoMulti, NoScript", "AclCategories": "Admin, Dangerous, Slow, Garnet" }, + { + "Command": "CLUSTER_FLUSHALL", + "Name": "CLUSTER|FLUSHALL", + "IsInternal": true, + "Arity": 2, + "Flags": "Admin, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow, Garnet" + }, { "Command": "CLUSTER_FORGET", "Name": "CLUSTER|FORGET", @@ -966,14 +978,6 @@ "Arity": 4, "Flags": "Admin, NoMulti, NoScript", "AclCategories": "Admin, Dangerous, Slow, Garnet" - }, - { - "Command": "CLUSTER_FLUSHALL", - "Name": "CLUSTER|FLUSHALL", - "IsInternal": true, - "Arity": 2, - "Flags": "Admin, NoMulti, NoScript", - "AclCategories": "Admin, Dangerous, Slow, Garnet" } ] }, @@ -1085,14 +1089,16 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Read, Slow, Garnet" + "AclCategories": "Read, Slow, Garnet", + "StoreType": "Object" }, { "Command": "CustomObjCmd", "Name": "CustomObjCmd", "IsInternal": true, "Arity": 1, - "AclCategories": "Dangerous, Slow, Garnet, Custom" + "AclCategories": "Dangerous, Slow, Garnet, Custom", + "StoreType": "Object" }, { "Command": "CustomProcedure", @@ -1106,7 +1112,8 @@ "Name": "CustomRawStringCmd", "IsInternal": true, "Arity": 1, - "AclCategories": "Dangerous, Slow, Garnet, Custom" + "AclCategories": "Dangerous, Slow, Garnet, Custom", + "StoreType": "Main" }, { "Command": "CustomTxn", @@ -1156,7 +1163,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "DECRBY", @@ -1181,7 +1189,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "DEL", @@ -1210,7 +1219,8 @@ }, "Flags": "RM, Delete" } - ] + ], + "StoreType": "All" }, { "Command": "DELIFGREATER", @@ -1234,7 +1244,8 @@ }, "Flags": "RM, Delete" } - ] + ], + "StoreType": "Main" }, { "Command": "DISCARD", @@ -1269,7 +1280,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "All" }, { "Command": "ECHO", @@ -1357,7 +1369,18 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "All" + }, + { + "Command": "EXPDELSCAN", + "Name": "EXPDELSCAN", + "Arity": -1, + "Flags": "Admin, NoMulti, NoScript, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Admin, Garnet" }, { "Command": "EXPIRE", @@ -1382,7 +1405,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "EXPIREAT", @@ -1407,7 +1431,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "EXPIRETIME", @@ -1432,7 +1457,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "All" }, { "Command": "FAILOVER", @@ -1496,7 +1522,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "GEODIST", @@ -1521,7 +1548,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEOHASH", @@ -1546,7 +1574,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEOPOS", @@ -1571,7 +1600,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEORADIUS", @@ -1624,7 +1654,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "GEORADIUS_RO", @@ -1649,7 +1680,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEORADIUSBYMEMBER", @@ -1702,7 +1734,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "GEORADIUSBYMEMBER_RO", @@ -1727,7 +1760,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEOSEARCH", @@ -1752,7 +1786,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GEOSEARCHSTORE", @@ -1790,7 +1825,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "GET", @@ -1815,7 +1851,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "GETBIT", @@ -1840,7 +1877,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "GETDEL", @@ -1865,7 +1903,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Main" }, { "Command": "GETEX", @@ -1891,7 +1930,8 @@ "Notes": "RW and UPDATE because it changes the TTL", "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "GETIFNOTMATCH", @@ -1915,7 +1955,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "GETRANGE", @@ -1940,7 +1981,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "GETSET", @@ -1965,7 +2007,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "GETWITHETAG", @@ -1989,7 +2032,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "HCOLLECT", @@ -2014,7 +2058,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HDEL", @@ -2039,7 +2084,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "HELLO", @@ -2071,7 +2117,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "HEXPIRE", @@ -2096,7 +2143,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HEXPIREAT", @@ -2121,7 +2169,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HEXPIRETIME", @@ -2146,7 +2195,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HGET", @@ -2171,7 +2221,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HGETALL", @@ -2199,7 +2250,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HINCRBY", @@ -2224,7 +2276,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HINCRBYFLOAT", @@ -2249,7 +2302,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HKEYS", @@ -2277,7 +2331,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HLEN", @@ -2302,7 +2357,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "HMGET", @@ -2327,7 +2383,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HMSET", @@ -2352,7 +2409,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HPERSIST", @@ -2377,7 +2435,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HPEXPIRE", @@ -2402,7 +2461,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HPEXPIREAT", @@ -2427,7 +2487,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HPEXPIRETIME", @@ -2452,7 +2513,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HPTTL", @@ -2477,7 +2539,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HRANDFIELD", @@ -2505,7 +2568,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HSCAN", @@ -2533,7 +2597,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HSET", @@ -2558,7 +2623,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "HSETNX", @@ -2583,7 +2649,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "HSTRLEN", @@ -2608,7 +2675,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "HTTL", @@ -2633,7 +2701,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "HVALS", @@ -2661,7 +2730,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "INCR", @@ -2686,7 +2756,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "INCRBY", @@ -2711,7 +2782,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "INCRBYFLOAT", @@ -2736,7 +2808,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "INFO", @@ -2832,7 +2905,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "LINDEX", @@ -2857,7 +2931,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "LINSERT", @@ -2882,7 +2957,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "LLEN", @@ -2907,7 +2983,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "LMOVE", @@ -2945,7 +3022,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "LMPOP", @@ -2967,7 +3045,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "LPOP", @@ -2992,7 +3071,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "LPOS", @@ -3017,7 +3097,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "LPUSH", @@ -3042,7 +3123,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "LPUSHX", @@ -3067,7 +3149,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "LRANGE", @@ -3092,7 +3175,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "LREM", @@ -3117,7 +3201,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "LSET", @@ -3142,7 +3227,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "LTRIM", @@ -3167,7 +3253,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "MEMORY", @@ -3228,7 +3315,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "MIGRATE", @@ -3322,7 +3410,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "MSETNX", @@ -3347,7 +3436,8 @@ }, "Flags": "OW, Insert" } - ] + ], + "StoreType": "Main" }, { "Command": "MULTI", @@ -3379,7 +3469,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "PEXPIRE", @@ -3404,7 +3495,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "PEXPIREAT", @@ -3429,7 +3521,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "PEXPIRETIME", @@ -3454,7 +3547,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "All" }, { "Command": "PFADD", @@ -3479,7 +3573,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Main" }, { "Command": "PFCOUNT", @@ -3505,7 +3600,8 @@ "Notes": "RW because it may change the internal representation of the key, and propagate to replicas", "Flags": "RW, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "PFMERGE", @@ -3543,7 +3639,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "PING", @@ -3579,7 +3676,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "PSUBSCRIBE", @@ -3614,7 +3712,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "All" }, { "Command": "PUBLISH", @@ -3736,7 +3835,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "RENAMENX", @@ -3774,7 +3874,8 @@ }, "Flags": "OW, Insert" } - ] + ], + "StoreType": "All" }, { "Command": "REPLICAOF", @@ -3806,7 +3907,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "All" }, { "Command": "ROLE", @@ -3838,7 +3940,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "RPOPLPUSH", @@ -3876,7 +3979,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "RPUSH", @@ -3901,7 +4005,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "RPUSHX", @@ -3926,7 +4031,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "RUNTXP", @@ -3961,7 +4067,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "SAVE", @@ -3980,7 +4087,8 @@ "nondeterministic_output", "request_policy:special", "response_policy:special" - ] + ], + "StoreType": "All" }, { "Command": "SCARD", @@ -4005,7 +4113,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "SCRIPT", @@ -4074,7 +4183,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SDIFFSTORE", @@ -4112,7 +4222,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SECONDARYOF", @@ -4152,7 +4263,8 @@ "Notes": "RW and ACCESS due to the optional \u0060GET\u0060 argument", "Flags": "RW, Access, Update, VariableFlags" } - ] + ], + "StoreType": "Main" }, { "Command": "SETBIT", @@ -4177,7 +4289,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "SETEX", @@ -4202,7 +4315,8 @@ }, "Flags": "OW, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "SETIFGREATER", @@ -4226,7 +4340,8 @@ }, "Flags": "RW, Access, Update, VariableFlags" } - ] + ], + "StoreType": "Main" }, { "Command": "SETIFMATCH", @@ -4250,7 +4365,8 @@ }, "Flags": "RW, Access, Update, VariableFlags" } - ] + ], + "StoreType": "Main" }, { "Command": "SETNX", @@ -4275,7 +4391,8 @@ }, "Flags": "OW, Insert" } - ] + ], + "StoreType": "Main" }, { "Command": "SETRANGE", @@ -4300,7 +4417,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Main" }, { "Command": "SINTER", @@ -4328,7 +4446,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SINTERCARD", @@ -4350,7 +4469,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SINTERSTORE", @@ -4373,7 +4493,7 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "OW, Update" + "Flags": "RW, Update" }, { "BeginSearch": { @@ -4388,7 +4508,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SISMEMBER", @@ -4413,7 +4534,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { "Command": "SECONDARYOF", @@ -4497,7 +4619,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SMISMEMBER", @@ -4522,7 +4645,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SMOVE", @@ -4560,7 +4684,8 @@ }, "Flags": "RW, Insert" } - ] + ], + "StoreType": "Object" }, { "Command": "SPOP", @@ -4588,7 +4713,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "SPUBLISH", @@ -4641,7 +4767,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SREM", @@ -4666,7 +4793,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "SSCAN", @@ -4694,7 +4822,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SSUBSCRIBE", @@ -4744,7 +4873,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Main" }, { "Command": "SUBSCRIBE", @@ -4776,7 +4906,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Main" }, { "Command": "SUNION", @@ -4804,7 +4935,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SUNIONSTORE", @@ -4842,7 +4974,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "SWAPDB", @@ -4887,7 +5020,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "All" }, { "Command": "TYPE", @@ -4912,7 +5046,8 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "All" }, { "Command": "UNLINK", @@ -4941,7 +5076,8 @@ }, "Flags": "RM, Delete" } - ] + ], + "StoreType": "All" }, { "Command": "UNSUBSCRIBE", @@ -5055,7 +5191,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZCARD", @@ -5080,17 +5217,18 @@ }, "Flags": "RO" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZCOUNT", - "Name": "ZCOUNT", - "Arity": 4, - "Flags": "Fast, ReadOnly", + "Command": "ZCOLLECT", + "Name": "ZCOLLECT", + "Arity": 2, + "Flags": "Admin, Write", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, Read, SortedSet", + "AclCategories": "Admin, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5103,19 +5241,20 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RO, Access" + "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZCOLLECT", - "Name": "ZCOLLECT", - "Arity": 2, - "Flags": "Admin, Write", + "Command": "ZCOUNT", + "Name": "ZCOUNT", + "Arity": 4, + "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Write, Admin, Garnet", + "AclCategories": "Fast, Read, SortedSet", "KeySpecifications": [ { "BeginSearch": { @@ -5128,9 +5267,10 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RW, Access, Update" + "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZDIFF", @@ -5152,7 +5292,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZDIFFSTORE", @@ -5190,7 +5331,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZEXPIRE", @@ -5200,7 +5342,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5215,7 +5357,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZEXPIREAT", @@ -5225,7 +5368,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5240,7 +5383,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZEXPIRETIME", @@ -5250,7 +5394,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5265,7 +5409,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZINCRBY", @@ -5290,7 +5435,8 @@ }, "Flags": "RW, Access, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZINTER", @@ -5312,7 +5458,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZINTERCARD", @@ -5334,7 +5481,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZINTERSTORE", @@ -5372,7 +5520,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZLEXCOUNT", @@ -5397,7 +5546,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZMPOP", @@ -5419,7 +5569,8 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "ZMSCORE", @@ -5444,7 +5595,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZPERSIST", @@ -5454,7 +5606,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5469,7 +5621,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZPEXPIRE", @@ -5479,7 +5632,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5494,7 +5647,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZPEXPIREAT", @@ -5504,7 +5658,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5519,7 +5673,8 @@ }, "Flags": "RW, Update" } - ] + ], + "StoreType": "Object" }, { "Command": "ZPEXPIRETIME", @@ -5529,7 +5684,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5544,17 +5699,18 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZPTTL", - "Name": "ZPTTL", - "Arity": -5, - "Flags": "Fast, ReadOnly", + "Command": "ZPOPMAX", + "Name": "ZPOPMAX", + "Arity": -2, + "Flags": "Fast, Write", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, SortedSet, Write", "KeySpecifications": [ { "BeginSearch": { @@ -5567,13 +5723,14 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RO, Access" + "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZPOPMAX", - "Name": "ZPOPMAX", + "Command": "ZPOPMIN", + "Name": "ZPOPMIN", "Arity": -2, "Flags": "Fast, Write", "FirstKey": 1, @@ -5594,17 +5751,18 @@ }, "Flags": "RW, Access, Delete" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZPOPMIN", - "Name": "ZPOPMIN", - "Arity": -2, - "Flags": "Fast, Write", + "Command": "ZPTTL", + "Name": "ZPTTL", + "Arity": -5, + "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, SortedSet, Write", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5617,9 +5775,10 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RW, Access, Delete" + "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANDMEMBER", @@ -5647,7 +5806,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANGE", @@ -5672,7 +5832,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANGEBYLEX", @@ -5697,7 +5858,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANGEBYSCORE", @@ -5722,7 +5884,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANGESTORE", @@ -5760,7 +5923,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZRANK", @@ -5785,7 +5949,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREM", @@ -5810,7 +5975,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREMRANGEBYLEX", @@ -5835,7 +6001,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREMRANGEBYRANK", @@ -5860,7 +6027,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREMRANGEBYSCORE", @@ -5885,7 +6053,8 @@ }, "Flags": "RW, Delete" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREVRANGE", @@ -5910,7 +6079,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREVRANGEBYLEX", @@ -5935,7 +6105,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREVRANGEBYSCORE", @@ -5960,7 +6131,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZREVRANK", @@ -5985,7 +6157,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZSCAN", @@ -6013,17 +6186,18 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZTTL", - "Name": "ZTTL", - "Arity": -5, + "Command": "ZSCORE", + "Name": "ZSCORE", + "Arity": 3, "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet", "KeySpecifications": [ { "BeginSearch": { @@ -6038,17 +6212,18 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { - "Command": "ZSCORE", - "Name": "ZSCORE", - "Arity": 3, + "Command": "ZTTL", + "Name": "ZTTL", + "Arity": -5, "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, Read, SortedSet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -6063,7 +6238,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZUNION", @@ -6085,7 +6261,8 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" }, { "Command": "ZUNIONSTORE", @@ -6123,6 +6300,7 @@ }, "Flags": "RO, Access" } - ] + ], + "StoreType": "Object" } ] \ No newline at end of file diff --git a/libs/server/Cluster/StoreType.cs b/libs/server/Cluster/StoreType.cs index b841ba34c59..1efa27fb52e 100644 --- a/libs/server/Cluster/StoreType.cs +++ b/libs/server/Cluster/StoreType.cs @@ -10,6 +10,11 @@ namespace Garnet.server /// public enum StoreType : byte { + /// + /// No store specified + /// + None = 0, + /// /// Main (raw string) store /// diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs index a81fec8caeb..6cc37408b4a 100644 --- a/libs/server/Resp/BasicCommands.cs +++ b/libs/server/Resp/BasicCommands.cs @@ -1175,23 +1175,21 @@ private bool NetworkCOMMAND_GETKEYS() } var cmdName = parseState.GetString(0); - bool cmdFound = RespCommandsInfo.TryGetRespCommandInfo(cmdName, out var cmdInfo, true, true, logger) || - storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out cmdInfo); - if (!cmdFound) - { + // Try to parse command and get its simplified info + if (!TryGetSimpleCommandInfo(cmdName, out var simpleCmdInfo)) return AbortWithErrorMessage(CmdStrings.RESP_INVALID_COMMAND_SPECIFIED); - } - if (cmdInfo.KeySpecifications == null || cmdInfo.KeySpecifications.Length == 0) - { + // If command has no key specifications, abort with error + if (simpleCmdInfo.KeySpecs == null || simpleCmdInfo.KeySpecs.Length == 0) return AbortWithErrorMessage(CmdStrings.RESP_COMMAND_HAS_NO_KEY_ARGS); - } - - parseState.TryExtractKeysFromSpecs(cmdInfo.KeySpecifications, out var keys); + // Extract command keys from parse state and key specification + // An offset is applied to the parse state, as the command (and possibly subcommand) are included in the parse state. + var slicedParseState = parseState.Slice(simpleCmdInfo.IsSubCommand ? 2 : 1); + var keys = slicedParseState.ExtractCommandKeys(simpleCmdInfo); - while (!RespWriteUtils.TryWriteArrayLength(keys.Count, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteArrayLength(keys.Length, ref dcurr, dend)) SendAndReset(); foreach (var key in keys) @@ -1214,35 +1212,35 @@ private bool NetworkCOMMAND_GETKEYSANDFLAGS() } var cmdName = parseState.GetString(0); - bool cmdFound = RespCommandsInfo.TryGetRespCommandInfo(cmdName, out var cmdInfo, true, true, logger) || - storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out cmdInfo); - if (!cmdFound) - { + // Try to parse command and get its simplified info + if (!TryGetSimpleCommandInfo(cmdName, out var simpleCmdInfo)) return AbortWithErrorMessage(CmdStrings.RESP_INVALID_COMMAND_SPECIFIED); - } - if (cmdInfo.KeySpecifications == null || cmdInfo.KeySpecifications.Length == 0) - { + // If command has no key specifications, abort with error + if (simpleCmdInfo.KeySpecs == null || simpleCmdInfo.KeySpecs.Length == 0) return AbortWithErrorMessage(CmdStrings.RESP_COMMAND_HAS_NO_KEY_ARGS); - } - parseState.TryExtractKeysAndFlagsFromSpecs(cmdInfo.KeySpecifications, out var keys, out var flags); + // Extract command keys from parse state and key specification + // An offset is applied to the parse state, as the command (and possibly subcommand) are included in the parse state. + var slicedParseState = parseState.Slice(simpleCmdInfo.IsSubCommand ? 2 : 1); + var keysAndFlags = slicedParseState.ExtractCommandKeysAndFlags(simpleCmdInfo); - while (!RespWriteUtils.TryWriteArrayLength(keys.Count, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteArrayLength(keysAndFlags.Length, ref dcurr, dend)) SendAndReset(); - for (int i = 0; i < keys.Count; i++) + for (var i = 0; i < keysAndFlags.Length; i++) { while (!RespWriteUtils.TryWriteArrayLength(2, ref dcurr, dend)) SendAndReset(); - while (!RespWriteUtils.TryWriteBulkString(keys[i].Span, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteBulkString(keysAndFlags[i].Item1.Span, ref dcurr, dend)) SendAndReset(); - WriteSetLength(flags[i].Length); + var flags = EnumUtils.GetEnumDescriptions(keysAndFlags[i].Item2); + WriteSetLength(flags.Length); - foreach (var flag in flags[i]) + foreach (var flag in flags) { while (!RespWriteUtils.TryWriteBulkString(Encoding.ASCII.GetBytes(flag), ref dcurr, dend)) SendAndReset(); @@ -1762,6 +1760,29 @@ bool ParseGETAndKey(ref SpanByte key) return true; } + private bool TryGetSimpleCommandInfo(string cmdName, out SimpleRespCommandInfo simpleCmdInfo) + { + simpleCmdInfo = SimpleRespCommandInfo.Default; + + // Try to parse known command from name and obtain its command info + if (!Enum.TryParse(cmdName, true, out var cmd) || + !RespCommandsInfo.TryGetSimpleRespCommandInfo(cmd, out simpleCmdInfo, logger)) + { + // If we no known command or info was found, attempt to find custom command + if (storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out var cmdInfo)) + { + cmdInfo.PopulateSimpleCommandInfo(ref simpleCmdInfo); + } + else + { + // No matching command was found + return false; + } + } + + return true; + } + static void SetResult(int c, ref int firstPending, ref (GarnetStatus, SpanByteAndMemory)[] outputArr, GarnetStatus status, SpanByteAndMemory output) { diff --git a/libs/server/Resp/RespCommandInfoSimplifiedStructs.cs b/libs/server/Resp/RespCommandInfoSimplifiedStructs.cs new file mode 100644 index 00000000000..4cafd9515c7 --- /dev/null +++ b/libs/server/Resp/RespCommandInfoSimplifiedStructs.cs @@ -0,0 +1,264 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Runtime.InteropServices; +using System.Text; + +namespace Garnet.server +{ + /// + /// Represents a simplified version of RESP command's information + /// + public struct SimpleRespCommandInfo + { + /// + /// Command Arity + /// + public sbyte Arity; + + /// + /// True if command is allowed in a transaction context + /// + public bool AllowedInTxn; + + /// + /// True if command has sub-commands + /// + public bool IsParent; + + /// + /// True if command is a sub-command + /// + public bool IsSubCommand; + + /// + /// Simplified command key specifications + /// + public SimpleRespKeySpec[] KeySpecs; + + /// + /// Store type that the command operates on (None/Main/Object/All). Default: None for commands without key arguments. + /// + public StoreType StoreType; + + /// + /// Default SimpleRespCommandInfo + /// + public static SimpleRespCommandInfo Default = new(); + } + + /// + /// Represents a simplified version of a single key specification of a RESP command + /// + public struct SimpleRespKeySpecBeginSearch + { + /// + /// Keyword that precedes the keys in the command arguments + /// + public byte[] Keyword; + + /// + /// Index of first key or the index at which to start searching for keyword (if begin search is of keyword type) + /// + public int Index; + + /// + /// If true - begin search is of type index, otherwise begin search is of type keyword + /// + public bool IsIndexType; + + /// + /// Set begin search of type index + /// + /// Index of first key + public SimpleRespKeySpecBeginSearch(int index) + { + IsIndexType = true; + Index = index; + } + + /// + /// Set begin search of type keyword + /// + /// Keyword that precedes the keys in the command arguments + /// Index at which to start searching for keyword + public SimpleRespKeySpecBeginSearch(string keyword, int startIdx) + { + Index = startIdx; + Keyword = Encoding.UTF8.GetBytes(keyword); + } + } + + /// + /// Represents a simplified version of a single key specification of a RESP command + /// + [StructLayout(LayoutKind.Explicit, Size = Size)] + public struct SimpleRespKeySpecFindKeys + { + /// + /// Size of struct + /// + public const int Size = 10; + + /// + /// The index (relative to begin search) of the argument containing the number of keys + /// + [FieldOffset(0)] + public int KeyNumIndex; + + /// + /// the index (relative to begin search) of the first key + /// + [FieldOffset(4)] + public int FirstKey; + + /// + /// The index (relative to begin search) of the last key argument or limit - stops the key search by a factor. + /// 0 and 1 mean no limit. 2 means half of the remaining arguments, 3 means a third, and so on. + /// Limit is used if IsRangeLimitType is set. + /// + [FieldOffset(0)] + public int LastKeyOrLimit; + + /// + /// The number of arguments that should be skipped, after finding a key, to find the next one. + /// + [FieldOffset(4)] + public int KeyStep; + + /// + /// If true - find keys is of type range, otherwise find keys is of type keynum + /// + [FieldOffset(8)] + public bool IsRangeType; + + /// + /// If true - find keys is of type range and limit is used, otherwise find keys is of type range and last key is used. + /// + [FieldOffset(9)] + public bool IsRangeLimitType; + + /// + /// Set find keys of type range + /// + /// The number of arguments that should be skipped, after finding a key, to find the next one + /// The index of the last key argument or the limit + /// If preceding argument represents a limit + public SimpleRespKeySpecFindKeys(int keyStep, int lastKeyOrLimit, bool isLimit) + { + IsRangeType = true; + KeyStep = keyStep; + LastKeyOrLimit = lastKeyOrLimit; + IsRangeLimitType = isLimit; + } + + /// + /// Set find keys of type keynum + /// + /// The index of the argument containing the number of keys + /// The index of the first key + /// The number of arguments that should be skipped, after finding a key, to find the next one + public SimpleRespKeySpecFindKeys(int keyNumIndex, int firstKey, int keyStep) + { + KeyNumIndex = keyNumIndex; + FirstKey = firstKey; + KeyStep = keyStep; + } + } + + /// + /// Represents a simplified version of a single key specification of a RESP command + /// + public struct SimpleRespKeySpec + { + /// + /// Begin search specification + /// + public SimpleRespKeySpecBeginSearch BeginSearch; + + /// + /// Find keys specification + /// + public SimpleRespKeySpecFindKeys FindKeys; + + /// + /// Key specification flags + /// + public KeySpecificationFlags Flags; + } + + /// + /// Extension methods for obtaining simplified RESP command info structs + /// + public static class RespCommandInfoExtensions + { + /// + /// Populates a SimpleRespCommandInfo struct from a RespCommandsInfo instance + /// + /// The source RespCommandsInfo + /// The destination SimpleRespCommandInfo + public static void PopulateSimpleCommandInfo(this RespCommandsInfo cmdInfo, ref SimpleRespCommandInfo simpleCmdInfo) + { + var arity = cmdInfo.Arity; + + // Verify that arity is in the signed byte range (-128 to 127) + Debug.Assert(arity is <= sbyte.MaxValue and >= sbyte.MinValue); + + simpleCmdInfo.Arity = (sbyte)arity; + simpleCmdInfo.AllowedInTxn = (cmdInfo.Flags & RespCommandFlags.NoMulti) == 0; + simpleCmdInfo.IsParent = (cmdInfo.SubCommands?.Length ?? 0) > 0; + simpleCmdInfo.IsSubCommand = cmdInfo.Parent != null; + simpleCmdInfo.StoreType = cmdInfo.StoreType; + + if (cmdInfo.KeySpecifications != null) + { + var tmpSimpleKeySpecs = new List(); + + foreach (var keySpec in cmdInfo.KeySpecifications) + { + if (keySpec.TryGetSimpleKeySpec(out var simpleKeySpec)) + tmpSimpleKeySpecs.Add(simpleKeySpec); + } + + simpleCmdInfo.KeySpecs = tmpSimpleKeySpecs.ToArray(); + } + } + + /// + /// Tries to convert a RespCommandKeySpecification to a SimpleRespKeySpec + /// + /// The source RespCommandKeySpecification + /// The resulting SimpleRespKeySpec + /// True if successful + public static bool TryGetSimpleKeySpec(this RespCommandKeySpecification keySpec, out SimpleRespKeySpec simpleKeySpec) + { + simpleKeySpec = new SimpleRespKeySpec(); + + if (keySpec.BeginSearch is BeginSearchUnknown || keySpec.FindKeys is FindKeysUnknown) + return false; + + simpleKeySpec.BeginSearch = keySpec.BeginSearch switch + { + BeginSearchIndex bsi => new SimpleRespKeySpecBeginSearch(bsi.Index), + BeginSearchKeyword bsk => new SimpleRespKeySpecBeginSearch(bsk.Keyword, bsk.StartFrom), + _ => throw new NotSupportedException() + }; + + simpleKeySpec.FindKeys = keySpec.FindKeys switch + { + FindKeysRange fkr => fkr.LastKey == -1 + ? new SimpleRespKeySpecFindKeys(fkr.KeyStep, fkr.Limit, true) + : new SimpleRespKeySpecFindKeys(fkr.KeyStep, fkr.LastKey, false), + FindKeysKeyNum fkk => new SimpleRespKeySpecFindKeys(fkk.KeyNumIdx, fkk.FirstKey, fkk.KeyStep), + _ => throw new NotSupportedException() + }; + + simpleKeySpec.Flags = keySpec.Flags; + + return true; + } + } +} \ No newline at end of file diff --git a/libs/server/Resp/RespCommandsInfo.cs b/libs/server/Resp/RespCommandsInfo.cs index 1c936f9cdf0..db91b3532ba 100644 --- a/libs/server/Resp/RespCommandsInfo.cs +++ b/libs/server/Resp/RespCommandsInfo.cs @@ -5,7 +5,6 @@ using System.Collections.Generic; using System.Collections.Immutable; using System.Collections.ObjectModel; -using System.Diagnostics; using System.Linq; using System.Numerics; using System.Text.Json.Serialization; @@ -15,37 +14,6 @@ namespace Garnet.server { - /// - /// Represents a simplified version of RESP command's information - /// - public struct SimpleRespCommandInfo - { - /// - /// Command Arity - /// - public sbyte Arity; - - /// - /// If command is allowed in a transaction context - /// - public bool AllowedInTxn; - - /// - /// If command has sub-commands - /// - public bool IsParent; - - /// - /// If command is a sub-command - /// - public bool IsSubCommand; - - /// - /// Default SimpleRespCommandInfo - /// - public static SimpleRespCommandInfo Default = new(); - } - /// /// Represents a RESP command's information /// @@ -120,6 +88,11 @@ public RespAclCategories AclCategories /// public RespCommandKeySpecification[] KeySpecifications { get; init; } + /// + /// Store type that the command operates on (None/Main/Object/All). Default: None for commands without key arguments. + /// + public StoreType StoreType { get; set; } + /// public RespCommandsInfo[] SubCommands { get; init; } @@ -197,15 +170,7 @@ private static bool TryInitializeRespCommandsInfo(ILogger logger = null) continue; } - var arity = cmdInfo.Arity; - - // Verify that arity is in the signed byte range (-128 to 127) - Debug.Assert(arity <= sbyte.MaxValue && arity >= sbyte.MinValue); - - tmpSimpleRespCommandInfo[cmdId].Arity = (sbyte)arity; - tmpSimpleRespCommandInfo[cmdId].AllowedInTxn = (cmdInfo.Flags & RespCommandFlags.NoMulti) == 0; - tmpSimpleRespCommandInfo[cmdId].IsParent = (cmdInfo.SubCommands?.Length ?? 0) > 0; - tmpSimpleRespCommandInfo[cmdId].IsSubCommand = cmdInfo.Parent != null; + cmdInfo.PopulateSimpleCommandInfo(ref tmpSimpleRespCommandInfo[cmdId]); } var tmpAllSubCommandsInfo = new Dictionary(StringComparer.OrdinalIgnoreCase); diff --git a/libs/server/SessionParseStateExtensions.cs b/libs/server/SessionParseStateExtensions.cs index a535cac4a60..19b37ac06b5 100644 --- a/libs/server/SessionParseStateExtensions.cs +++ b/libs/server/SessionParseStateExtensions.cs @@ -3,6 +3,8 @@ using System; using System.Collections.Generic; +using System.Diagnostics; +using System.Linq; using System.Text; using Garnet.common; @@ -848,50 +850,53 @@ internal static bool TryGetTimeout(this SessionParseState parseState, int idx, o /// Tries to extract keys from the key specifications in the given RespCommandsInfo. /// /// The SessionParseState instance. - /// The RespCommandKeySpecification array contains the key specification - /// The list to store extracted keys. - /// True if keys were successfully extracted, otherwise false. - internal static bool TryExtractKeysFromSpecs(this ref SessionParseState state, RespCommandKeySpecification[] keySpecs, out List keys) + /// The command's simplified info + /// The extracted keys + internal static ArgSlice[] ExtractCommandKeys(this ref SessionParseState state, SimpleRespCommandInfo commandInfo) { - keys = new(); + var keysIndexes = new List<(ArgSlice Key, int Index)>(); - foreach (var spec in keySpecs) - { - if (!ExtractKeysFromSpec(ref state, keys, spec)) - { - return false; - } - } + foreach (var spec in commandInfo.KeySpecs) + TryAppendKeysFromSpec(ref state, spec, commandInfo.IsSubCommand, keysIndexes); - return true; + return keysIndexes.OrderBy(k => k.Index).Select(k => k.Key).ToArray(); } /// /// Tries to extract keys and their associated flags from the key specifications in the given RespCommandsInfo. /// /// The SessionParseState instance. - /// The RespCommandKeySpecification array containing the key specifications. - /// The list to store extracted keys. - /// The list to store associated flags for each key. - /// True if keys and flags were successfully extracted, otherwise false. - internal static bool TryExtractKeysAndFlagsFromSpecs(this ref SessionParseState state, RespCommandKeySpecification[] keySpecs, out List keys, out List flags) + /// The command's simplified info + /// The extracted keys and flags + internal static (ArgSlice, KeySpecificationFlags)[] ExtractCommandKeysAndFlags(this ref SessionParseState state, SimpleRespCommandInfo commandInfo) { - keys = new(); - flags = new(); + var keysFlagsIndexes = new List<(ArgSlice Key, KeySpecificationFlags Flags, int Index)>(); + + foreach (var spec in commandInfo.KeySpecs) + TryAppendKeysAndFlagsFromSpec(ref state, spec, commandInfo.IsSubCommand, keysFlagsIndexes); + + return keysFlagsIndexes.OrderBy(k => k.Index).Select(k => (k.Key, k.Flags)).ToArray(); + } + + /// + /// Extracts keys from the given key specification in the provided SessionParseState. + /// + /// The SessionParseState instance. + /// The key specification to use for extraction. + /// True if command is a sub-command + /// The list to store extracted keys and their matching indexes + private static bool TryAppendKeysFromSpec(ref SessionParseState parseState, SimpleRespKeySpec keySpec, bool isSubCommand, List<(ArgSlice Key, int Index)> keysToIndexes) + { + if (!parseState.TryGetKeySearchArgsFromSimpleKeySpec(keySpec, isSubCommand, out var searchArgs)) + return false; - foreach (var spec in keySpecs) + for (var i = searchArgs.firstIdx; i <= searchArgs.lastIdx; i += searchArgs.step) { - var prevKeyCount = keys.Count; - if (!ExtractKeysFromSpec(ref state, keys, spec)) - { - return false; - } + var key = parseState.GetArgSliceByRef(i); + if (key.Length == 0) + continue; - var keyFlags = spec.RespFormatFlags; - for (int i = prevKeyCount; i < keys.Count; i++) - { - flags.Add(keyFlags); - } + keysToIndexes.Add((key, i)); } return true; @@ -900,30 +905,108 @@ internal static bool TryExtractKeysAndFlagsFromSpecs(this ref SessionParseState /// /// Extracts keys from the given key specification in the provided SessionParseState. /// - /// The SessionParseState instance. - /// The list to store extracted keys. - /// The key specification to use for extraction. - /// True if keys were successfully extracted, otherwise false. - private static bool ExtractKeysFromSpec(ref SessionParseState state, List keys, RespCommandKeySpecification spec) + /// The SessionParseState instance. + /// The key specification to use for extraction. + /// True if command is a sub-command + /// The list to store extracted keys and flags and their indexes + private static bool TryAppendKeysAndFlagsFromSpec(ref SessionParseState parseState, SimpleRespKeySpec keySpec, bool isSubCommand, List<(ArgSlice Key, KeySpecificationFlags Flags, int Index)> keysAndFlags) { - int startIndex = 0; + if (!parseState.TryGetKeySearchArgsFromSimpleKeySpec(keySpec, isSubCommand, out var searchArgs)) + return false; - if (spec.BeginSearch is BeginSearchKeySpecMethodBase bsKeyword) + for (var i = searchArgs.firstIdx; i <= searchArgs.lastIdx; i += searchArgs.step) { - if (!bsKeyword.TryGetStartIndex(ref state, out startIndex)) + var key = parseState.GetArgSliceByRef(i); + if (key.Length == 0) + continue; + + keysAndFlags.Add((key, keySpec.Flags, i)); + } + + return true; + } + + /// + /// Extracts the first, last, and step arguments for key searching based on a simplified RESP key specification and the current parse state. + /// + /// The current parse state + /// The simplified key specification + /// True if command is a sub-command + /// First, last, and step arguments for key searching + /// + internal static bool TryGetKeySearchArgsFromSimpleKeySpec(this ref SessionParseState parseState, SimpleRespKeySpec keySpec, bool isSubCommand, out (int firstIdx, int lastIdx, int step) searchArgs) + { + searchArgs = (-1, -1, -1); + + // Determine the starting index for searching keys + var beginSearchIdx = keySpec.BeginSearch.Index < 0 + ? parseState.Count + keySpec.BeginSearch.Index + : keySpec.BeginSearch.Index - (isSubCommand ? 2 : 1); + + if (beginSearchIdx < 0 || beginSearchIdx >= parseState.Count) + return false; + + var firstKeyIdx = -1; + + // If the begin search is an index type - use the specified index as a constant + if (keySpec.BeginSearch.IsIndexType) + { + firstKeyIdx = beginSearchIdx; + } + // If the begin search is a keyword type - search for the keyword in the parse state, starting at the specified index + else + { + var step = keySpec.BeginSearch.Index < 0 ? -1 : 1; + for (var i = beginSearchIdx; i < parseState.Count; i += step) { - return false; + if (parseState.GetArgSliceByRef(i).ReadOnlySpan + .EqualsUpperCaseSpanIgnoringCase(keySpec.BeginSearch.Keyword)) + { + // The begin search index is the argument immediately after the keyword + firstKeyIdx = i + 1; + break; + } } } - if (startIndex < 0 || startIndex >= state.Count) - return false; + // Next, determine the first, last, and step arguments for key searching based on the find keys specification + var keyStep = keySpec.FindKeys.KeyStep; + int lastKeyIdx; - if (spec.FindKeys is FindKeysKeySpecMethodBase findKey) + if (keySpec.FindKeys.IsRangeType) + { + // If the find keys is of type range with limit, the last key index is determined by the limit factor + // 0 and 1 mean no limit, 2 means half of the remaining arguments, 3 means a third, and so on. + if (keySpec.FindKeys.IsRangeLimitType) + { + var limit = keySpec.FindKeys.LastKeyOrLimit; + var keyNum = 1 + ((parseState.Count - 1 - firstKeyIdx) / keyStep); + lastKeyIdx = limit is 0 or 1 ? firstKeyIdx + ((keyNum - 1) * keyStep) + : firstKeyIdx + (((keyNum / limit) - 1) * keyStep); + } + // If the find keys is of type range with last key, the last key index is determined by the specified last key index relative to the begin search index + else + { + lastKeyIdx = keySpec.FindKeys.LastKeyOrLimit; + lastKeyIdx = lastKeyIdx < 0 ? lastKeyIdx + parseState.Count : firstKeyIdx + lastKeyIdx; + } + } + // If the find keys is of type keynum, the last key index is determined by the number of keys specified at the key number index relative to the begin search index + else { - findKey.ExtractKeys(ref state, startIndex, keys); + var keyNumIdx = beginSearchIdx + keySpec.FindKeys.KeyNumIndex; + Debug.Assert(keyNumIdx >= 0 && keyNumIdx < parseState.Count); + + var keyNumFound = parseState.TryGetInt(keyNumIdx, out var keyNum); + Debug.Assert(keyNumFound); + + firstKeyIdx += keySpec.FindKeys.FirstKey; + lastKeyIdx = firstKeyIdx + ((keyNum - 1) * keyStep); } + Debug.Assert(lastKeyIdx < parseState.Count); + + searchArgs = (firstKeyIdx, lastKeyIdx, keyStep); return true; } } diff --git a/libs/server/Transaction/TxnKeyManager.cs b/libs/server/Transaction/TxnKeyManager.cs index e309e8157e1..f8089664799 100644 --- a/libs/server/Transaction/TxnKeyManager.cs +++ b/libs/server/Transaction/TxnKeyManager.cs @@ -1,7 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; using Garnet.common; using Tsavorite.core; @@ -52,523 +51,36 @@ public unsafe void VerifyKeyOwnership(ArgSlice key, LockType type) if (!respSession.clusterSession.NetworkIterativeSlotVerify(key, readOnly, respSession.SessionAsking)) { this.state = TxnState.Aborted; - return; - } - } - - /// - /// Returns a number of skipped args - /// - internal int GetKeys(RespCommand command, int inputCount, out ReadOnlySpan error) - { - error = CmdStrings.RESP_ERR_GENERIC_UNK_CMD; - return command switch - { - RespCommand.APPEND => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.BITCOUNT => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.BITFIELD => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.BITFIELD_RO => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.BITOP => SingleWriteKeyListReadKeys(inputCount, false, offset: 1), - RespCommand.BITPOS => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.COSCAN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.DECR => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.DECRBY => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.DEL => ListKeys(inputCount, StoreType.All, LockType.Exclusive), - RespCommand.DELIFGREATER => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.EXISTS => ListKeys(inputCount, StoreType.All, LockType.Shared), - RespCommand.EXPIRE => SingleKey(StoreType.All, LockType.Exclusive), - RespCommand.EXPIREAT => SingleKey(StoreType.All, LockType.Exclusive), - RespCommand.EXPIRETIME => SingleKey(StoreType.All, LockType.Shared), - RespCommand.GEOADD => SortedSetObjectKeys(command, inputCount), - RespCommand.GEODIST => SortedSetObjectKeys(command, inputCount), - RespCommand.GEOHASH => SortedSetObjectKeys(command, inputCount), - RespCommand.GEOPOS => SortedSetObjectKeys(command, inputCount), - RespCommand.GEORADIUS => SortedSetObjectKeys(command, inputCount), - RespCommand.GEORADIUS_RO => SortedSetObjectKeys(command, inputCount), - RespCommand.GEORADIUSBYMEMBER => SortedSetObjectKeys(command, inputCount), - RespCommand.GEORADIUSBYMEMBER_RO => SortedSetObjectKeys(command, inputCount), - RespCommand.GEOSEARCH => SortedSetObjectKeys(command, inputCount), - RespCommand.GEOSEARCHSTORE => SortedSetObjectKeys(command, inputCount), - RespCommand.GET => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.GETBIT => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.GETDEL => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.GETEX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.GETIFNOTMATCH => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.GETRANGE => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.GETSET => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.GETWITHETAG => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.HDEL => HashObjectKeys(command), - RespCommand.HEXISTS => HashObjectKeys(command), - RespCommand.HEXPIRE => HashObjectKeys(command), - RespCommand.HEXPIREAT => HashObjectKeys(command), - RespCommand.HEXPIRETIME => HashObjectKeys(command), - RespCommand.HGET => HashObjectKeys(command), - RespCommand.HGETALL => HashObjectKeys(command), - RespCommand.HINCRBY => HashObjectKeys(command), - RespCommand.HINCRBYFLOAT => HashObjectKeys(command), - RespCommand.HKEYS => HashObjectKeys(command), - RespCommand.HLEN => HashObjectKeys(command), - RespCommand.HMGET => HashObjectKeys(command), - RespCommand.HMSET => HashObjectKeys(command), - RespCommand.HPERSIST => HashObjectKeys(command), - RespCommand.HPEXPIRE => HashObjectKeys(command), - RespCommand.HPEXPIREAT => HashObjectKeys(command), - RespCommand.HPEXPIRETIME => HashObjectKeys(command), - RespCommand.HPTTL => HashObjectKeys(command), - RespCommand.HRANDFIELD => HashObjectKeys(command), - RespCommand.HSCAN => HashObjectKeys(command), - RespCommand.HSET => HashObjectKeys(command), - RespCommand.HSETNX => HashObjectKeys(command), - RespCommand.HSTRLEN => HashObjectKeys(command), - RespCommand.HTTL => HashObjectKeys(command), - RespCommand.HVALS => HashObjectKeys(command), - RespCommand.INCR => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.INCRBY => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.INCRBYFLOAT => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.LCS => ListKeys(2, StoreType.Main, LockType.Shared), - RespCommand.LINDEX => ListObjectKeys(command), - RespCommand.LINSERT => ListObjectKeys(command), - RespCommand.LLEN => ListObjectKeys(command), - RespCommand.LMOVE => ListObjectKeys(command), - RespCommand.LMPOP => ListObjectKeys(command), - RespCommand.LPOP => ListObjectKeys(command), - RespCommand.LPOS => ListObjectKeys(command), - RespCommand.LPUSH => ListObjectKeys(command), - RespCommand.LPUSHX => ListObjectKeys(command), - RespCommand.LRANGE => ListObjectKeys(command), - RespCommand.LREM => ListObjectKeys(command), - RespCommand.LSET => ListObjectKeys(command), - RespCommand.LTRIM => ListObjectKeys(command), - RespCommand.MGET => ListKeys(inputCount, StoreType.Main, LockType.Shared), - RespCommand.MSET => MSETKeys(inputCount, LockType.Exclusive), - RespCommand.MSETNX => MSETKeys(inputCount, LockType.Exclusive), - RespCommand.PERSIST => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.PEXPIRE => SingleKey(StoreType.All, LockType.Exclusive), - RespCommand.PEXPIREAT => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.PEXPIRETIME => SingleKey(StoreType.All, LockType.Shared), - RespCommand.PFADD => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.PFCOUNT => ListKeys(inputCount, StoreType.Main, LockType.Shared), - RespCommand.PFMERGE => SingleWriteKeyListReadKeys(inputCount, false), - RespCommand.PSETEX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.PTTL => SingleKey(StoreType.All, LockType.Shared), - RespCommand.RENAME => SingleKey(StoreType.All, LockType.Exclusive), - RespCommand.RENAMENX => SingleKey(StoreType.All, LockType.Exclusive), - RespCommand.RPOP => ListObjectKeys(command), - RespCommand.RPOPLPUSH => ListObjectKeys(command), - RespCommand.RPUSH => ListObjectKeys(command), - RespCommand.RPUSHX => ListObjectKeys(command), - RespCommand.SADD => SetObjectKeys(command, inputCount), - RespCommand.SCARD => SetObjectKeys(command, inputCount), - RespCommand.SDIFF => SetObjectKeys(command, inputCount), - RespCommand.SDIFFSTORE => SetObjectKeys(command, inputCount), - RespCommand.SET => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETBIT => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETEX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETEXNX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETEXXX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETIFGREATER => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETIFMATCH => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETNX => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SETRANGE => SingleKey(StoreType.Main, LockType.Exclusive), - RespCommand.SINTER => SetObjectKeys(command, inputCount), - RespCommand.SINTERCARD => SetObjectKeys(command, inputCount), - RespCommand.SINTERSTORE => SetObjectKeys(command, inputCount), - RespCommand.SISMEMBER => SetObjectKeys(command, inputCount), - RespCommand.SMEMBERS => SetObjectKeys(command, inputCount), - RespCommand.SMISMEMBER => SetObjectKeys(command, inputCount), - RespCommand.SMOVE => SetObjectKeys(command, inputCount), - RespCommand.SPOP => SetObjectKeys(command, inputCount), - RespCommand.SRANDMEMBER => SetObjectKeys(command, inputCount), - RespCommand.SREM => SetObjectKeys(command, inputCount), - RespCommand.SSCAN => SetObjectKeys(command, inputCount), - RespCommand.STRLEN => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.SUBSTR => SingleKey(StoreType.Main, LockType.Shared), - RespCommand.SUNION => SetObjectKeys(command, inputCount), - RespCommand.SUNIONSTORE => SetObjectKeys(command, inputCount), - RespCommand.TTL => SingleKey(StoreType.All, LockType.Shared), - RespCommand.TYPE => SingleKey(StoreType.All, LockType.Shared), - RespCommand.UNLINK => ListKeys(inputCount, StoreType.All, LockType.Exclusive), - RespCommand.ZADD => SortedSetObjectKeys(command, inputCount), - RespCommand.ZCARD => SortedSetObjectKeys(command, inputCount), - RespCommand.ZCOUNT => SortedSetObjectKeys(command, inputCount), - RespCommand.ZDIFF => SortedSetObjectKeys(command, inputCount), - RespCommand.ZDIFFSTORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZEXPIRE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZEXPIREAT => SortedSetObjectKeys(command, inputCount), - RespCommand.ZEXPIRETIME => SortedSetObjectKeys(command, inputCount), - RespCommand.ZINCRBY => SortedSetObjectKeys(command, inputCount), - RespCommand.ZINTER => SortedSetObjectKeys(command, inputCount), - RespCommand.ZINTERCARD => SortedSetObjectKeys(command, inputCount), - RespCommand.ZINTERSTORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZLEXCOUNT => SortedSetObjectKeys(command, inputCount), - RespCommand.ZMPOP => SortedSetObjectKeys(command, inputCount), - RespCommand.ZMSCORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPERSIST => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPEXPIRE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPEXPIREAT => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPEXPIRETIME => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPOPMAX => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPOPMIN => SortedSetObjectKeys(command, inputCount), - RespCommand.ZPTTL => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANDMEMBER => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANGE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANGEBYLEX => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANGEBYSCORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANGESTORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZRANK => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREM => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREMRANGEBYLEX => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREMRANGEBYRANK => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREMRANGEBYSCORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREVRANGE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREVRANGEBYLEX => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREVRANGEBYSCORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZREVRANK => SortedSetObjectKeys(command, inputCount), - RespCommand.ZSCAN => SortedSetObjectKeys(command, inputCount), - RespCommand.ZSCORE => SortedSetObjectKeys(command, inputCount), - RespCommand.ZTTL => SortedSetObjectKeys(command, inputCount), - RespCommand.ZUNION => SortedSetObjectKeys(command, inputCount), - RespCommand.ZUNIONSTORE => SortedSetObjectKeys(command, inputCount), - _ => OtherCommands(command, out error) - }; - } - - private int OtherCommands(RespCommand command, out ReadOnlySpan error) - { - error = CmdStrings.RESP_ERR_GENERIC_UNK_CMD; - if (command == RespCommand.DEBUG) - { - if (respSession.CanRunDebug()) - return 1; - - error = System.Text.Encoding.ASCII.GetBytes(string.Format( - CmdStrings.GenericErrCommandDisallowedWithOption, RespCommand.DEBUG, "enable-debug-command")); - return -1; } - - return command switch - { - RespCommand.CLIENT => 1, - RespCommand.CONFIG => 1, - RespCommand.ECHO => 1, - RespCommand.PING => 1, - RespCommand.PUBLISH => 1, - RespCommand.SELECT => 1, - RespCommand.SPUBLISH => 1, - RespCommand.SWAPDB => 1, - RespCommand.TIME => 1, - _ => -1 - }; - } - - private int GeoCommands(RespCommand command, int inputCount) - { - var idx = 0; - - // GEOSEARCHSTORE dest key.... - // While all other commands here start with GEOsomething key... - if (command == RespCommand.GEOSEARCHSTORE) - { - var destinationKey = respSession.parseState.GetArgSliceByRef(idx++); - SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); - SaveKeyArgSlice(destinationKey); - } - - // Either this is GEOSEARCHSTORE, and index 1 is sourcekey, or some other command and index 0 is sourcekey. - var key = respSession.parseState.GetArgSliceByRef(idx++); - SaveKeyEntryToLock(key, true, LockType.Shared); - SaveKeyArgSlice(key); - - switch (command) - { - case RespCommand.GEOSEARCH: - case RespCommand.GEORADIUS_RO: - case RespCommand.GEORADIUSBYMEMBER_RO: - return 1; - case RespCommand.GEOSEARCHSTORE: - return 2; - case RespCommand.GEORADIUS: - case RespCommand.GEORADIUSBYMEMBER: - // These commands may or may not store a result - for (var i = idx; i < inputCount - 1; ++i) - { - var span = respSession.parseState.GetArgSliceByRef(i).ReadOnlySpan; - - if (span.EqualsUpperCaseSpanIgnoringCase(CmdStrings.STORE) || - span.EqualsUpperCaseSpanIgnoringCase(CmdStrings.STOREDIST)) - { - var destinationKey = respSession.parseState.GetArgSliceByRef(i + 1); - SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); - SaveKeyArgSlice(destinationKey); - break; - } - } - - return 1; - default: - // Should never reach here. - throw new NotSupportedException(); - } - } - - private int SortedSetObjectKeys(RespCommand command, int inputCount) - { - return command switch - { - RespCommand.GEOADD => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.GEODIST => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.GEOHASH => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.GEOPOS => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.GEORADIUS => GeoCommands(RespCommand.GEORADIUS, inputCount), - RespCommand.GEORADIUS_RO => GeoCommands(RespCommand.GEORADIUS_RO, inputCount), - RespCommand.GEORADIUSBYMEMBER => GeoCommands(RespCommand.GEORADIUSBYMEMBER, inputCount), - RespCommand.GEORADIUSBYMEMBER_RO => GeoCommands(RespCommand.GEORADIUSBYMEMBER_RO, inputCount), - RespCommand.GEOSEARCH => GeoCommands(RespCommand.GEOSEARCH, inputCount), - RespCommand.GEOSEARCHSTORE => GeoCommands(RespCommand.GEOSEARCHSTORE, inputCount), - RespCommand.ZADD => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZCARD => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZCOUNT => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZDIFF => ListReadKeysWithCount(LockType.Shared), - RespCommand.ZDIFFSTORE => SingleWriteKeyListReadKeysWithCount(inputCount), - RespCommand.ZEXPIRE => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZEXPIREAT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZEXPIRETIME => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZINCRBY => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZINTER => ListReadKeysWithCount(LockType.Shared), - RespCommand.ZINTERCARD => ListReadKeysWithCount(LockType.Shared), - RespCommand.ZINTERSTORE => SingleWriteKeyListReadKeysWithCount(inputCount), - RespCommand.ZLEXCOUNT => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZMPOP => ListReadKeysWithCount(LockType.Exclusive), - RespCommand.ZMSCORE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZPERSIST => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZPEXPIRE => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZPEXPIREAT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZPEXPIRETIME => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZPOPMAX => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZPOPMIN => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZPTTL => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZRANDMEMBER => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZRANGE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZRANGEBYLEX => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZRANGEBYSCORE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZRANGESTORE => SingleWriteKeyListReadKeys(2), - RespCommand.ZRANK => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZREM => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZREMRANGEBYLEX => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZREMRANGEBYRANK => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZREMRANGEBYSCORE => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.ZREVRANGE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZREVRANGEBYLEX => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZREVRANGEBYSCORE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZREVRANK => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZSCAN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZSCORE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZTTL => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.ZUNION => ListReadKeysWithCount(LockType.Shared), - RespCommand.ZUNIONSTORE => SingleWriteKeyListReadKeysWithCount(inputCount), - _ => -1 - }; - } - - private int ListObjectKeys(RespCommand command) - { - return command switch - { - RespCommand.LINDEX => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.LINSERT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LLEN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.LMOVE => ListKeys(2, StoreType.Object, LockType.Exclusive), - RespCommand.LMPOP => ListReadKeysWithCount(LockType.Exclusive), - RespCommand.LPOP => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LPOS => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.LPUSH => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LPUSHX => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LRANGE => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.LREM => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LSET => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.LTRIM => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.RPOP => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.RPOPLPUSH => ListKeys(2, StoreType.Object, LockType.Exclusive), - RespCommand.RPUSH => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.RPUSHX => SingleKey(StoreType.Object, LockType.Exclusive), - _ => -1 - }; - } - - private int HashObjectKeys(RespCommand command) - { - return command switch - { - RespCommand.HDEL => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HEXISTS => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HEXPIRE => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HEXPIREAT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HEXPIRETIME => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HGET => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HGETALL => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HINCRBY => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HINCRBYFLOAT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HKEYS => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HLEN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HMGET => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HMSET => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HPERSIST => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HPEXPIRE => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HPEXPIREAT => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HPEXPIRETIME => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HPTTL => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HRANDFIELD => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HSCAN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HSET => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HSETNX => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.HSTRLEN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HTTL => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.HVALS => SingleKey(StoreType.Object, LockType.Shared), - _ => -1 - }; - } - - private int SetObjectKeys(RespCommand command, int inputCount) - { - return command switch - { - RespCommand.SADD => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.SCARD => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SDIFF => ListKeys(inputCount, StoreType.Object, LockType.Shared), - RespCommand.SDIFFSTORE => SingleWriteKeyListReadKeys(inputCount), - RespCommand.SINTER => ListKeys(inputCount, StoreType.Object, LockType.Shared), - RespCommand.SINTERCARD => ListReadKeysWithCount(LockType.Shared), - RespCommand.SINTERSTORE => SingleWriteKeyListReadKeys(inputCount), - RespCommand.SISMEMBER => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SMEMBERS => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SMISMEMBER => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SMOVE => ListKeys(2, StoreType.Object, LockType.Exclusive), - RespCommand.SPOP => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.SRANDMEMBER => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SREM => SingleKey(StoreType.Object, LockType.Exclusive), - RespCommand.SSCAN => SingleKey(StoreType.Object, LockType.Shared), - RespCommand.SUNION => ListKeys(inputCount, StoreType.Object, LockType.Shared), - RespCommand.SUNIONSTORE => SingleWriteKeyListReadKeys(inputCount), - _ => -1 - }; } /// - /// Returns a single for commands that have a single key + /// Locks keys according to command's key specifications /// - private int SingleKey(StoreType storeType, LockType type) + /// Simplified command info + internal void LockKeys(SimpleRespCommandInfo cmdInfo) { - var key = respSession.parseState.GetArgSliceByRef(0); - if (storeType is StoreType.Main or StoreType.All) - SaveKeyEntryToLock(key, false, type); - if (storeType is StoreType.Object or StoreType.All && !objectStoreBasicContext.IsNull) - SaveKeyEntryToLock(key, true, type); - SaveKeyArgSlice(key); - return 1; - } - - /// - /// Returns a list of keys for commands: MGET, DEL, UNLINK - /// - private int ListKeys(int inputCount, StoreType storeType, LockType type) - { - for (var i = 0; i < inputCount; i++) - { - var key = respSession.parseState.GetArgSliceByRef(i); - if (storeType is StoreType.Main or StoreType.All) - SaveKeyEntryToLock(key, false, type); - if (storeType is StoreType.Object or StoreType.All && !objectStoreBasicContext.IsNull) - SaveKeyEntryToLock(key, true, type); - SaveKeyArgSlice(key); - } - return inputCount; - } - - /// - /// Returns a list of keys for LMPOP command - /// - private int ListReadKeysWithCount(LockType type, bool isObject = true) - { - if (respSession.parseState.Count == 0) - return -2; - - if (!respSession.parseState.TryGetInt(0, out var numKeys)) - return -2; - - if (numKeys + 1 > respSession.parseState.Count) - return -2; - - for (var i = 1; i < numKeys + 1; i++) - { - var key = respSession.parseState.GetArgSliceByRef(i); - SaveKeyEntryToLock(key, isObject, type); - SaveKeyArgSlice(key); - } - return numKeys; - } - - /// - /// Returns a list of keys for MSET commands - /// - private int MSETKeys(int inputCount, LockType type, bool isObject = true) - { - for (var i = 0; i < inputCount; i += 2) - { - var key = respSession.parseState.GetArgSliceByRef(i); - SaveKeyEntryToLock(key, isObject, type); - SaveKeyArgSlice(key); - } - return inputCount; - } - - /// - /// Returns a list of keys for set *STORE commands (e.g. SUNIONSTORE, SINTERSTORE etc.) - /// Where the first key's value is written to and the rest of the keys' values are read from. - /// - private int SingleWriteKeyListReadKeys(int inputCount, bool isObject = true, int offset = 0) - { - if (inputCount <= offset) - return 0; - - var key = respSession.parseState.GetArgSliceByRef(offset); - SaveKeyEntryToLock(key, isObject, LockType.Exclusive); - SaveKeyArgSlice(key); - - for (var i = offset + 1; i < inputCount; i++) - { - key = respSession.parseState.GetArgSliceByRef(i); - SaveKeyEntryToLock(key, isObject, LockType.Shared); - SaveKeyArgSlice(key); - } - - return inputCount; - } + if (cmdInfo.KeySpecs == null || cmdInfo.KeySpecs.Length == 0) + return; - /// - /// Returns a list of keys for complex *STORE commands (e.g. ZUNIONSTORE, ZINTERSTORE etc.) - /// Where the first key's value is written to and the rest of the keys' values are read from. - /// We can get number of read keys by checking the second argument. - /// - private int SingleWriteKeyListReadKeysWithCount(int inputCount, bool isObject = true) - { - if (inputCount > 0) + foreach (var keySpec in cmdInfo.KeySpecs) { - var key = respSession.parseState.GetArgSliceByRef(0); - SaveKeyEntryToLock(key, isObject, LockType.Exclusive); - SaveKeyArgSlice(key); - } + if (!respSession.parseState.TryGetKeySearchArgsFromSimpleKeySpec(keySpec, cmdInfo.IsSubCommand, out var searchArgs)) + continue; - if ((inputCount < 2) || !respSession.parseState.TryGetInt(1, out var numKeysArg)) - return -2; + var isReadOnly = (keySpec.Flags & KeySpecificationFlags.RO) == KeySpecificationFlags.RO; + var lockType = isReadOnly ? LockType.Shared : LockType.Exclusive; - for (var i = 2; i < inputCount && i < numKeysArg + 2; i++) - { - var key = respSession.parseState.GetArgSliceByRef(i); - SaveKeyEntryToLock(key, isObject, LockType.Shared); - SaveKeyArgSlice(key); + for (var currIdx = searchArgs.firstIdx; currIdx <= searchArgs.lastIdx; currIdx += searchArgs.step) + { + var key = respSession.parseState.GetArgSliceByRef(currIdx); + if (cmdInfo.StoreType is StoreType.Main or StoreType.All) + SaveKeyEntryToLock(key, false, lockType); + if (cmdInfo.StoreType is StoreType.Object or StoreType.All && !objectStoreBasicContext.IsNull) + SaveKeyEntryToLock(key, true, lockType); + SaveKeyArgSlice(key); + } } - - return inputCount; } } } \ No newline at end of file diff --git a/libs/server/Transaction/TxnRespCommands.cs b/libs/server/Transaction/TxnRespCommands.cs index 761c6a8ebc8..e51eaf8612c 100644 --- a/libs/server/Transaction/TxnRespCommands.cs +++ b/libs/server/Transaction/TxnRespCommands.cs @@ -166,23 +166,18 @@ private bool NetworkSKIP(RespCommand cmd) } } - // Get and add keys to txn key list - int skipped = txnManager.GetKeys(cmd, count, out ReadOnlySpan error); - - if (skipped < 0) + if (cmd == RespCommand.DEBUG && !CanRunDebug()) { - // We ran out of data in network buffer, let caller handler it - if (skipped == -2) return false; - - // We found an unsupported command, abort - while (!RespWriteUtils.TryWriteError(error, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(System.Text.Encoding.ASCII.GetBytes(string.Format( + CmdStrings.GenericErrCommandDisallowedWithOption, RespCommand.DEBUG, "enable-debug-command")), ref dcurr, dend)) SendAndReset(); txnManager.Abort(); - return true; } + txnManager.LockKeys(commandInfo); + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_QUEUED, ref dcurr, dend)) SendAndReset(); diff --git a/playground/CommandInfoUpdater/CommandDocsUpdater.cs b/playground/CommandInfoUpdater/CommandDocsUpdater.cs index 2aab4d4ebf0..9d77a94df1c 100644 --- a/playground/CommandInfoUpdater/CommandDocsUpdater.cs +++ b/playground/CommandInfoUpdater/CommandDocsUpdater.cs @@ -70,7 +70,7 @@ public static bool TryUpdateCommandDocs(string outputDir, int respServerPort, IP IDictionary queriedCommandsDocs = new Dictionary(); var commandsToQuery = commandsToAdd.Keys.Select(k => k.Command) - .Where(c => updatedCommandsInfo.ContainsKey(c) && !updatedCommandsInfo[c].IsInternal).ToArray(); + .Where(c => updatedCommandsInfo.ContainsKey(c) || (updatedCommandsInfo[c].SubCommands?.Length > 0 && !updatedCommandsInfo[c].IsInternal)).ToArray(); if (commandsToQuery.Length > 0) { for (var i = 0; i < commandsToQuery.Length; i += QUERY_CMD_BATCH_SIZE) diff --git a/playground/CommandInfoUpdater/CommandInfoUpdater.cs b/playground/CommandInfoUpdater/CommandInfoUpdater.cs index a96d70dda2c..ca9cc0947af 100644 --- a/playground/CommandInfoUpdater/CommandInfoUpdater.cs +++ b/playground/CommandInfoUpdater/CommandInfoUpdater.cs @@ -15,7 +15,7 @@ namespace CommandInfoUpdater /// public class CommandInfoUpdater { - const int QUERY_CMD_BATCH_SIZE = 10; + const int QUERY_CMD_BATCH_SIZE = 1; private static readonly string CommandInfoFileName = "RespCommandsInfo.json"; private static readonly string GarnetCommandInfoJsonPath = "GarnetCommandsInfo.json"; @@ -62,7 +62,7 @@ public static bool TryUpdateCommandInfo(string outputDir, int respServerPort, IP IDictionary queriedCommandsInfo = new Dictionary(); var commandsToQuery = commandsToAdd.Keys.Select(k => k.Command) - .Where(c => !garnetCommandsInfo.ContainsKey(c) || !garnetCommandsInfo[c].IsInternal).ToArray(); + .Where(c => !garnetCommandsInfo.ContainsKey(c) || (garnetCommandsInfo[c].SubCommands?.Length > 0 && !garnetCommandsInfo[c].IsInternal)).ToArray(); if (commandsToQuery.Length > 0) { @@ -131,6 +131,26 @@ public static bool TryUpdateCommandInfo(string outputDir, int respServerPort, IP } } + // Update store types + foreach (var sc in commandsToAdd.Keys) + { + if (!additionalCommandsInfo.TryGetValue(sc.Command, out var commandInfo)) + continue; + + commandInfo.StoreType = sc.StoreType; + + if (commandInfo.SubCommands == null) + continue; + + foreach (var subCommandInfo in commandInfo.SubCommands) + { + if (sc.SubCommands.TryGetValue(subCommandInfo.Name, out var scSubCommand)) + { + subCommandInfo.StoreType = scSubCommand.StoreType; + } + } + } + updatedCommandsInfo = GetUpdatedCommandsInfo(existingCommandsInfo, commandsToAdd, commandsToRemove, additionalCommandsInfo); @@ -175,9 +195,9 @@ private static unsafe bool TryGetCommandsInfo(string[] commandsToQuery, int resp } // Get a map of supported commands to Garnet's RespCommand & ArrayCommand for the parser - var supportedCommands = new ReadOnlyDictionary( + var supportedCommands = new ReadOnlyDictionary( SupportedCommand.SupportedCommandsFlattenedMap.ToDictionary(kvp => kvp.Key, - kvp => kvp.Value.RespCommand, StringComparer.OrdinalIgnoreCase)); + kvp => (kvp.Value.RespCommand, kvp.Value.StoreType), StringComparer.OrdinalIgnoreCase)); // Parse the response fixed (byte* respPtr = response) @@ -259,6 +279,7 @@ private static IReadOnlyDictionary GetUpdatedCommandsI AclCategories = existingCommand.AclCategories, Tips = existingCommand.Tips, KeySpecifications = existingCommand.KeySpecifications, + StoreType = existingCommand.StoreType, SubCommands = remainingSubCommands == null || remainingSubCommands.Length == 0 ? null : [.. existingCommand.SubCommands.Where(sc => remainingSubCommands.Contains(sc.Name))] @@ -315,6 +336,7 @@ private static IReadOnlyDictionary GetUpdatedCommandsI AclCategories = baseCommand.AclCategories, Tips = baseCommand.Tips, KeySpecifications = baseCommand.KeySpecifications, + StoreType = baseCommand.StoreType, SubCommands = updatedSubCommands?.ToArray() }; diff --git a/playground/CommandInfoUpdater/CommonUtils.cs b/playground/CommandInfoUpdater/CommonUtils.cs index f3b28a6da3f..aa669dbbb8e 100644 --- a/playground/CommandInfoUpdater/CommonUtils.cs +++ b/playground/CommandInfoUpdater/CommonUtils.cs @@ -130,7 +130,7 @@ .. supportedCommand.SubCommands if (subCommandsToAdd.Length > 0) { commandsToAdd.Add( - new SupportedCommand(supportedCommand.Command, supportedCommand.RespCommand, subCommandsToAdd), false); + new SupportedCommand(supportedCommand.Command, supportedCommand.RespCommand, supportedCommand.StoreType, subCommandsToAdd), false); } } @@ -162,7 +162,7 @@ .. supportedCommand.SubCommands if (subCommandsToRemove.Length > 0) { commandsToRemove.Add( - new SupportedCommand(existingCommand.Key, existingCommand.Value.Command, subCommandsToRemove), false); + new SupportedCommand(existingCommand.Key, existingCommand.Value.Command, StoreType.None, subCommandsToRemove), false); } } diff --git a/playground/CommandInfoUpdater/GarnetCommandsDocs.json b/playground/CommandInfoUpdater/GarnetCommandsDocs.json index 039746b3aad..2ce569c4f83 100644 --- a/playground/CommandInfoUpdater/GarnetCommandsDocs.json +++ b/playground/CommandInfoUpdater/GarnetCommandsDocs.json @@ -40,164 +40,672 @@ ] }, { - "Command": "CLIENT_KILL", - "Name": "CLIENT|KILL", - "Summary": "Terminates open connections.", + "Command": "CLIENT", + "Name": "CLIENT", + "Summary": "A container for client connection commands.", "Group": "Connection", - "Complexity": "O(N) where N is the number of client connections", + "Complexity": "Depends on subcommand.", + "SubCommands": [ + { + "Command": "CLIENT_KILL", + "Name": "CLIENT|KILL", + "Summary": "Terminates open connections.", + "Group": "Connection", + "Complexity": "O(N) where N is the number of client connections", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FILTER", + "Type": "OneOf", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "OLD-FORMAT", + "DisplayText": "ip:port", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "NEW-FORMAT", + "Type": "OneOf", + "ArgumentFlags": "Multiple", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "CLIENT-ID", + "DisplayText": "client-id", + "Type": "Integer", + "Token": "ID", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "CLIENT-TYPE", + "Type": "OneOf", + "Token": "TYPE", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NORMAL", + "DisplayText": "normal", + "Type": "PureToken", + "Token": "NORMAL" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MASTER", + "DisplayText": "master", + "Type": "PureToken", + "Token": "MASTER" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "SLAVE", + "DisplayText": "slave", + "Type": "PureToken", + "Token": "SLAVE" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "REPLICA", + "DisplayText": "replica", + "Type": "PureToken", + "Token": "REPLICA" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "PUBSUB", + "DisplayText": "pubsub", + "Type": "PureToken", + "Token": "PUBSUB" + } + ] + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "USERNAME", + "DisplayText": "username", + "Type": "String", + "Token": "USER", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "ADDR", + "DisplayText": "ip:port", + "Type": "String", + "Token": "ADDR", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "LADDR", + "DisplayText": "ip:port", + "Type": "String", + "Token": "LADDR", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "SKIPME", + "Type": "OneOf", + "Token": "SKIPME", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "YES", + "DisplayText": "yes", + "Type": "PureToken", + "Token": "YES" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NO", + "DisplayText": "no", + "Type": "PureToken", + "Token": "NO" + } + ] + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MAXAGE", + "DisplayText": "maxage", + "Type": "Integer", + "Token": "MAXAGE", + "ArgumentFlags": "Optional" + } + ] + } + ] + } + ] + } + ] + }, + { + "Command": "COMMITAOF", + "Name": "COMMITAOF", + "Group": "Server", + "Summary": "Commit to append-only file.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "DBID", + "DisplayText": "dbid", + "Type": "Integer", + "Token": "DBID", + "ArgumentFlags": "Optional" + } + ] + }, + { + "Command": "COSCAN", + "Name": "COSCAN", + "Group": "Generic", + "Summary": "Iterates over members of a collection object.", + "Complexity": "O(1) for every call. O(N) for a complete iteration, including enough command calls for the cursor to return back to 0. N is the number of elements inside the collection.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "CURSOR", + "DisplayText": "cursor", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "PATTERN", + "DisplayText": "pattern", + "Type": "Pattern", + "Token": "MATCH", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "COUNT", + "DisplayText": "count", + "Type": "Integer", + "Token": "COUNT", + "ArgumentFlags": "Optional" + } + ] + }, + { + "Command": "DELIFGREATER", + "Name": "DELIFGREATER", + "Summary": "Deletes a key only if the provided Etag is strictly greater than the existing Etag for the key.", + "Group": "Generic", + "Complexity": "O(1)", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "ETAG", + "DisplayText": "etag", + "Type": "Integer" + } + ] + }, + { + "Command": "FORCEGC", + "Name": "FORCEGC", + "Summary": "Forces garbage collection.", + "Group": "Server" + }, + { + "Command": "HCOLLECT", + "Name": "HCOLLECT", + "Summary": "Manually trigger deletion of expired fields from memory", + "Group": "Hash" + }, + { + "Command": "HEXPIRE", + "Name": "HEXPIRE", + "Summary": "Set expiry for hash field using relative time to expire (seconds)", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "SECONDS", + "DisplayText": "seconds", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "CONDITION", + "Type": "OneOf", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NX", + "DisplayText": "nx", + "Type": "PureToken", + "Token": "NX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "XX", + "DisplayText": "xx", + "Type": "PureToken", + "Token": "XX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "GT", + "DisplayText": "gt", + "Type": "PureToken", + "Token": "GT" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "LT", + "DisplayText": "lt", + "Type": "PureToken", + "Token": "LT" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] + }, + { + "Command": "HEXPIREAT", + "Name": "HEXPIREAT", + "Summary": "Set expiry for hash field using an absolute Unix timestamp (seconds)", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "UNIX-TIME-SECONDS", + "DisplayText": "unix-time-seconds", + "Type": "UnixTime" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "CONDITION", + "Type": "OneOf", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NX", + "DisplayText": "nx", + "Type": "PureToken", + "Token": "NX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "XX", + "DisplayText": "xx", + "Type": "PureToken", + "Token": "XX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "GT", + "DisplayText": "gt", + "Type": "PureToken", + "Token": "GT" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "LT", + "DisplayText": "lt", + "Type": "PureToken", + "Token": "LT" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] + }, + { + "Command": "HEXPIRETIME", + "Name": "HEXPIRETIME", + "Summary": "Returns the expiration time of a hash field as a Unix timestamp, in seconds.", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] + }, + { + "Command": "HPERSIST", + "Name": "HPERSIST", + "Summary": "Removes the expiration time for each specified field", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] + }, + { + "Command": "HPEXPIRE", + "Name": "HPEXPIRE", + "Summary": "Set expiry for hash field using relative time to expire (milliseconds)", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MILLISECONDS", + "DisplayText": "milliseconds", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "CONDITION", + "Type": "OneOf", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NX", + "DisplayText": "nx", + "Type": "PureToken", + "Token": "NX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "XX", + "DisplayText": "xx", + "Type": "PureToken", + "Token": "XX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "GT", + "DisplayText": "gt", + "Type": "PureToken", + "Token": "GT" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "LT", + "DisplayText": "lt", + "Type": "PureToken", + "Token": "LT" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] + }, + { + "Command": "HPEXPIREAT", + "Name": "HPEXPIREAT", + "Summary": "Set expiry for hash field using an absolute Unix timestamp (milliseconds)", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "UNIX-TIME-MILLISECONDS", + "DisplayText": "unix-time-milliseconds", + "Type": "UnixTime" + }, { "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "FILTER", + "Name": "CONDITION", "Type": "OneOf", + "ArgumentFlags": "Optional", "Arguments": [ { "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "OLD-FORMAT", - "DisplayText": "ip:port", - "Type": "String" + "Name": "NX", + "DisplayText": "nx", + "Type": "PureToken", + "Token": "NX" }, { - "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "NEW-FORMAT", - "Type": "OneOf", - "ArgumentFlags": "Multiple", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "CLIENT-ID", - "DisplayText": "client-id", - "Type": "Integer", - "Token": "ID", - "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "CLIENT-TYPE", - "Type": "OneOf", - "Token": "TYPE", - "ArgumentFlags": "Optional", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "NORMAL", - "DisplayText": "normal", - "Type": "PureToken", - "Token": "NORMAL" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "MASTER", - "DisplayText": "master", - "Type": "PureToken", - "Token": "MASTER" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "SLAVE", - "DisplayText": "slave", - "Type": "PureToken", - "Token": "SLAVE" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "REPLICA", - "DisplayText": "replica", - "Type": "PureToken", - "Token": "REPLICA" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "PUBSUB", - "DisplayText": "pubsub", - "Type": "PureToken", - "Token": "PUBSUB" - } - ] - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "USERNAME", - "DisplayText": "username", - "Type": "String", - "Token": "USER", - "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "ADDR", - "DisplayText": "ip:port", - "Type": "String", - "Token": "ADDR", - "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "LADDR", - "DisplayText": "ip:port", - "Type": "String", - "Token": "LADDR", - "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "SKIPME", - "Type": "OneOf", - "Token": "SKIPME", - "ArgumentFlags": "Optional", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "YES", - "DisplayText": "yes", - "Type": "PureToken", - "Token": "YES" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "NO", - "DisplayText": "no", - "Type": "PureToken", - "Token": "NO" - } - ] - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "MAXAGE", - "DisplayText": "maxage", - "Type": "Integer", - "Token": "MAXAGE", - "ArgumentFlags": "Optional" - } - ] + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "XX", + "DisplayText": "xx", + "Type": "PureToken", + "Token": "XX" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "GT", + "DisplayText": "gt", + "Type": "PureToken", + "Token": "GT" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "LT", + "DisplayText": "lt", + "Type": "PureToken", + "Token": "LT" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" } ] } ] }, { - "Command": "COMMITAOF", - "Name": "COMMITAOF", - "Group": "Server", - "Summary": "Commit to append-only file.", + "Command": "HPEXPIRETIME", + "Name": "HPEXPIRETIME", + "Summary": "Returns the expiration time of a hash field as a Unix timestamp, in msec.", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", "Arguments": [ { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "DBID", - "DisplayText": "dbid", - "Type": "Integer", - "Token": "DBID", - "ArgumentFlags": "Optional" + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] } ] }, { - "Command": "COSCAN", - "Name": "COSCAN", - "Group": "Generic", - "Summary": "Iterates over members of a collection object.", - "Complexity": "O(1) for every call. O(N) for a complete iteration, including enough command calls for the cursor to return back to 0. N is the number of elements inside the collection.", + "Command": "HPTTL", + "Name": "HPTTL", + "Summary": "Returns the TTL in milliseconds of a hash field.", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", "Arguments": [ { "TypeDiscriminator": "RespCommandKeyArgument", @@ -207,35 +715,34 @@ "KeySpecIndex": 0 }, { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "CURSOR", - "DisplayText": "cursor", - "Type": "Integer" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "PATTERN", - "DisplayText": "pattern", - "Type": "Pattern", - "Token": "MATCH", - "ArgumentFlags": "Optional" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "COUNT", - "DisplayText": "count", - "Type": "Integer", - "Token": "COUNT", - "ArgumentFlags": "Optional" + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] } ] }, { - "Command": "DELIFGREATER", - "Name": "DELIFGREATER", - "Summary": "Deletes a key only if the provided Etag is strictly greater than the existing Etag for the key.", - "Group": "Generic", - "Complexity": "O(1)", + "Command": "HTTL", + "Name": "HTTL", + "Summary": "Returns the TTL in seconds of a hash field.", + "Group": "Hash", + "Complexity": "O(N) where N is the number of specified fields", "Arguments": [ { "TypeDiscriminator": "RespCommandKeyArgument", @@ -245,25 +752,28 @@ "KeySpecIndex": 0 }, { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "ETAG", - "DisplayText": "etag", - "Type": "Integer" + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "FIELDS", + "Type": "Block", + "Token": "FIELDS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMFIELDS", + "DisplayText": "numfields", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] } ] }, - { - "Command": "FORCEGC", - "Name": "FORCEGC", - "Summary": "Forces garbage collection.", - "Group": "Server" - }, - { - "Command": "HCOLLECT", - "Name": "HCOLLECT", - "Summary": "Manually trigger deletion of expired fields from memory", - "Group": "Hash" - }, { "Command": "SECONDARYOF", "Name": "SECONDARYOF", @@ -1441,5 +1951,42 @@ ] } ] + }, + { + "Command": "ZTTL", + "Name": "ZTTL", + "Summary": "Returns the TTL in seconds of a sorted set member.", + "Group": "SortedSet", + "Complexity": "O(N) where N is the number of specified members", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "MEMBERS", + "Type": "Block", + "Token": "MEMBERS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMMEMBERS", + "DisplayText": "nummembers", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MEMBER", + "DisplayText": "member", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + } + ] } ] \ No newline at end of file diff --git a/playground/CommandInfoUpdater/GarnetCommandsInfo.json b/playground/CommandInfoUpdater/GarnetCommandsInfo.json index 9d8e4473ab3..52786d649d8 100644 --- a/playground/CommandInfoUpdater/GarnetCommandsInfo.json +++ b/playground/CommandInfoUpdater/GarnetCommandsInfo.json @@ -51,6 +51,14 @@ "KeySpecifications": null, "SubCommands": null }, + { + "Command": "CLUSTER_ATTACH_SYNC", + "Name": "CLUSTER|ATTACH_SYNC", + "IsInternal": true, + "Arity": 3, + "Flags": "Admin, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow, Garnet" + }, { "Command": "CLUSTER_BANLIST", "Name": "CLUSTER|BANLIST", @@ -321,6 +329,14 @@ "AclCategories": "Admin, Dangerous, Slow, Garnet", "KeySpecifications": null, "SubCommands": null + }, + { + "Command": "CLUSTER_SYNC", + "Name": "CLUSTER|SYNC", + "IsInternal": true, + "Arity": 4, + "Flags": "Admin, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow, Garnet" } ] }, @@ -443,31 +459,6 @@ "KeySpecifications": null, "SubCommands": null }, - { - "Command": "HCOLLECT", - "Name": "HCOLLECT", - "Arity": 2, - "Flags": "Admin, Write", - "FirstKey": 1, - "LastKey": 1, - "Step": 1, - "AclCategories": "Admin, Hash, Write, Garnet", - "KeySpecifications": [ - { - "BeginSearch": { - "TypeDiscriminator": "BeginSearchIndex", - "Index": 1 - }, - "FindKeys": { - "TypeDiscriminator": "FindKeysRange", - "LastKey": 0, - "KeyStep": 1, - "Limit": 0 - }, - "Flags": "RW, Access, Update" - } - ] - }, { "Command": "LATENCY", "Name": "LATENCY", @@ -690,6 +681,256 @@ } ] }, + { + "Command": "HCOLLECT", + "Name": "HCOLLECT", + "Arity": 2, + "Flags": "Admin, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Admin, Hash, Write, Garnet", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Access, Update" + } + ] + }, + { + "Command": "HEXPIRE", + "Name": "HEXPIRE", + "Arity": -6, + "Flags": "DenyOom, Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Write", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Update" + } + ] + }, + { + "Command": "HEXPIREAT", + "Name": "HEXPIREAT", + "Arity": -6, + "Flags": "DenyOom, Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Write", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Update" + } + ] + }, + { + "Command": "HPERSIST", + "Name": "HPERSIST", + "Arity": -5, + "Flags": "Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Write", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Update" + } + ] + }, + { + "Command": "HEXPIRETIME", + "Name": "HEXPIRETIME", + "Arity": -5, + "Flags": "Fast, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Read", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ] + }, + { + "Command": "HPEXPIRE", + "Name": "HPEXPIRE", + "Arity": -6, + "Flags": "DenyOom, Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Write", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Update" + } + ] + }, + { + "Command": "HPEXPIREAT", + "Name": "HPEXPIREAT", + "Arity": -6, + "Flags": "DenyOom, Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Write", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Update" + } + ] + }, + { + "Command": "HPEXPIRETIME", + "Name": "HPEXPIRETIME", + "Arity": -5, + "Flags": "Fast, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Read", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ] + }, + { + "Command": "HPTTL", + "Name": "HPTTL", + "Arity": -5, + "Flags": "Fast, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Read", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ] + }, + { + "Command": "HTTL", + "Name": "HTTL", + "Arity": -5, + "Flags": "Fast, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Hash, Fast, Read", + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ] + }, { "Command": "LASTSAVE", "Name": "LASTSAVE", diff --git a/playground/CommandInfoUpdater/RespCommandInfoParser.cs b/playground/CommandInfoUpdater/RespCommandInfoParser.cs index 1927b730cde..c34a4469f89 100644 --- a/playground/CommandInfoUpdater/RespCommandInfoParser.cs +++ b/playground/CommandInfoUpdater/RespCommandInfoParser.cs @@ -15,11 +15,11 @@ public class RespCommandInfoParser /// /// Pointer to current RESP chunk to read /// Pointer to end of RESP chunk to read - /// Mapping between command name and Garnet RespCommand and ArrayCommand values + /// Mapping between command name, Garnet RespCommand and StoreType /// Parsed RespCommandsInfo object /// Name of parent command, null if none /// True if parsing successful - public static unsafe bool TryReadFromResp(ref byte* ptr, byte* end, IReadOnlyDictionary supportedCommands, out RespCommandsInfo commandInfo, string parentCommand = null) + public static unsafe bool TryReadFromResp(ref byte* ptr, byte* end, IReadOnlyDictionary supportedCommands, out RespCommandsInfo commandInfo, string parentCommand = null) { commandInfo = default; @@ -98,9 +98,10 @@ public static unsafe bool TryReadFromResp(ref byte* ptr, byte* end, IReadOnlyDic subCommands.Add(commandInfo); } + var supportedCommand = supportedCommands.GetValueOrDefault(name, (RespCommand.NONE, StoreType.None)); commandInfo = new RespCommandsInfo() { - Command = supportedCommands.GetValueOrDefault(name, RespCommand.NONE), + Command = supportedCommand.Item1, Name = name.ToUpper(), IsInternal = false, Arity = arity, @@ -111,6 +112,7 @@ public static unsafe bool TryReadFromResp(ref byte* ptr, byte* end, IReadOnlyDic AclCategories = aclCategories, Tips = tips.Length == 0 ? null : tips, KeySpecifications = keySpecifications.Length == 0 ? null : keySpecifications, + StoreType = supportedCommand.Item2, SubCommands = subCommands.Count == 0 ? null : [.. subCommands.OrderBy(sc => sc.Name)] }; diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index 89d9a8da159..a1a61b79234 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -12,7 +12,7 @@ namespace CommandInfoUpdater public class SupportedCommand { private static readonly SupportedCommand[] AllSupportedCommands = [ - new("ACL", RespCommand.ACL, + new("ACL", RespCommand.ACL, StoreType.None, [ new("ACL|CAT", RespCommand.ACL_CAT), new("ACL|DELUSER", RespCommand.ACL_DELUSER), @@ -26,25 +26,25 @@ public class SupportedCommand new("ACL|WHOAMI", RespCommand.ACL_WHOAMI), ]), new("EXPDELSCAN", RespCommand.EXPDELSCAN), - new("APPEND", RespCommand.APPEND), + new("APPEND", RespCommand.APPEND, StoreType.Main), new("ASKING", RespCommand.ASKING), new("ASYNC", RespCommand.ASYNC), new("AUTH", RespCommand.AUTH), new("BGSAVE", RespCommand.BGSAVE), - new("BITCOUNT", RespCommand.BITCOUNT), - new("BITFIELD", RespCommand.BITFIELD), - new("BITFIELD_RO", RespCommand.BITFIELD_RO), - new("BITOP", RespCommand.BITOP), - new("BITPOS", RespCommand.BITPOS), - new("BLPOP", RespCommand.BLPOP), - new("BRPOP", RespCommand.BRPOP), - new("BLMOVE", RespCommand.BLMOVE), - new("BRPOPLPUSH", RespCommand.BRPOPLPUSH), - new("BZMPOP", RespCommand.BZMPOP), - new("BZPOPMAX", RespCommand.BZPOPMAX), - new("BZPOPMIN", RespCommand.BZPOPMIN), - new("BLMPOP", RespCommand.BLMPOP), - new("CLIENT", RespCommand.CLIENT, + new("BITCOUNT", RespCommand.BITCOUNT, StoreType.Main), + new("BITFIELD", RespCommand.BITFIELD, StoreType.Main), + new("BITFIELD_RO", RespCommand.BITFIELD_RO, StoreType.Main), + new("BITOP", RespCommand.BITOP, StoreType.Main), + new("BITPOS", RespCommand.BITPOS, StoreType.Main), + new("BLPOP", RespCommand.BLPOP, StoreType.Object), + new("BRPOP", RespCommand.BRPOP, StoreType.Object), + new("BLMOVE", RespCommand.BLMOVE, StoreType.Object), + new("BRPOPLPUSH", RespCommand.BRPOPLPUSH, StoreType.Object), + new("BZMPOP", RespCommand.BZMPOP, StoreType.Object), + new("BZPOPMAX", RespCommand.BZPOPMAX, StoreType.Object), + new("BZPOPMIN", RespCommand.BZPOPMIN, StoreType.Object), + new("BLMPOP", RespCommand.BLMPOP, StoreType.Object), + new("CLIENT", RespCommand.CLIENT, StoreType.None, [ new("CLIENT|ID", RespCommand.CLIENT_ID), new("CLIENT|INFO", RespCommand.CLIENT_INFO), @@ -55,7 +55,7 @@ public class SupportedCommand new("CLIENT|SETINFO", RespCommand.CLIENT_SETINFO), new("CLIENT|UNBLOCK", RespCommand.CLIENT_UNBLOCK), ]), - new("CLUSTER", RespCommand.CLUSTER, + new("CLUSTER", RespCommand.CLUSTER, StoreType.None, [ new("CLUSTER|ADDSLOTS", RespCommand.CLUSTER_ADDSLOTS), new("CLUSTER|ADDSLOTSRANGE", RespCommand.CLUSTER_ADDSLOTSRANGE), @@ -103,7 +103,7 @@ public class SupportedCommand new("CLUSTER|SLOTSTATE", RespCommand.CLUSTER_SLOTSTATE), new("CLUSTER|SYNC", RespCommand.CLUSTER_SYNC), ]), - new("COMMAND", RespCommand.COMMAND, + new("COMMAND", RespCommand.COMMAND, StoreType.None, [ new("COMMAND|INFO", RespCommand.COMMAND_INFO), new("COMMAND|COUNT", RespCommand.COMMAND_COUNT), @@ -112,135 +112,134 @@ public class SupportedCommand new("COMMAND|GETKEYSANDFLAGS", RespCommand.COMMAND_GETKEYSANDFLAGS), ]), new("COMMITAOF", RespCommand.COMMITAOF), - new("CONFIG", RespCommand.CONFIG, + new("CONFIG", RespCommand.CONFIG, StoreType.None, [ new("CONFIG|GET", RespCommand.CONFIG_GET), new("CONFIG|SET", RespCommand.CONFIG_SET), new("CONFIG|REWRITE", RespCommand.CONFIG_REWRITE), ]), - new("COSCAN", RespCommand.COSCAN), - new("CustomRawStringCmd", RespCommand.CustomRawStringCmd), - new("CustomObjCmd", RespCommand.CustomObjCmd), + new("COSCAN", RespCommand.COSCAN, StoreType.Object), + new("CustomRawStringCmd", RespCommand.CustomRawStringCmd, StoreType.Main), + new("CustomObjCmd", RespCommand.CustomObjCmd, StoreType.Object), new("CustomTxn", RespCommand.CustomTxn), new("CustomProcedure", RespCommand.CustomProcedure), new("DBSIZE", RespCommand.DBSIZE), new("DEBUG", RespCommand.DEBUG), - new("DECR", RespCommand.DECR), - new("DECRBY", RespCommand.DECRBY), - new("DEL", RespCommand.DEL), - new("DELIFEXPIM", RespCommand.DELIFEXPIM), - new("DELIFGREATER", RespCommand.DELIFGREATER), + new("DECR", RespCommand.DECR, StoreType.Main), + new("DECRBY", RespCommand.DECRBY, StoreType.Main), + new("DEL", RespCommand.DEL, StoreType.All), + new("DELIFGREATER", RespCommand.DELIFGREATER, StoreType.Main), new("DISCARD", RespCommand.DISCARD), - new("DUMP", RespCommand.DUMP), + new("DUMP", RespCommand.DUMP, StoreType.All), new("ECHO", RespCommand.ECHO), new("EXEC", RespCommand.EXEC), - new("EXISTS", RespCommand.EXISTS), - new("EXPIRE", RespCommand.EXPIRE), - new("EXPIREAT", RespCommand.EXPIREAT), - new("EXPIRETIME", RespCommand.EXPIRETIME), + new("EXISTS", RespCommand.EXISTS, StoreType.All), + new("EXPIRE", RespCommand.EXPIRE, StoreType.All), + new("EXPIREAT", RespCommand.EXPIREAT, StoreType.All), + new("EXPIRETIME", RespCommand.EXPIRETIME, StoreType.All), new("FAILOVER", RespCommand.FAILOVER), new("FLUSHALL", RespCommand.FLUSHALL), new("FLUSHDB", RespCommand.FLUSHDB), new("FORCEGC", RespCommand.FORCEGC), - new("GEOADD", RespCommand.GEOADD), - new("GEODIST", RespCommand.GEODIST), - new("GEOHASH", RespCommand.GEOHASH), - new("GEOPOS", RespCommand.GEOPOS), - new("GEORADIUS", RespCommand.GEORADIUS), - new("GEORADIUS_RO", RespCommand.GEORADIUS_RO), - new("GEORADIUSBYMEMBER", RespCommand.GEORADIUSBYMEMBER), - new("GEORADIUSBYMEMBER_RO", RespCommand.GEORADIUSBYMEMBER_RO), - new("GEOSEARCH", RespCommand.GEOSEARCH), - new("GEOSEARCHSTORE", RespCommand.GEOSEARCHSTORE), - new("GET", RespCommand.GET), - new("GETEX", RespCommand.GETEX), - new("GETBIT", RespCommand.GETBIT), - new("GETDEL", RespCommand.GETDEL), - new("GETIFNOTMATCH", RespCommand.GETIFNOTMATCH), - new("GETRANGE", RespCommand.GETRANGE), - new("GETWITHETAG", RespCommand.GETWITHETAG), - new("GETSET", RespCommand.GETSET), - new("HCOLLECT", RespCommand.HCOLLECT), - new("HDEL", RespCommand.HDEL), + new("GEOADD", RespCommand.GEOADD, StoreType.Object), + new("GEODIST", RespCommand.GEODIST, StoreType.Object), + new("GEOHASH", RespCommand.GEOHASH, StoreType.Object), + new("GEOPOS", RespCommand.GEOPOS, StoreType.Object), + new("GEORADIUS", RespCommand.GEORADIUS, StoreType.Object), + new("GEORADIUS_RO", RespCommand.GEORADIUS_RO, StoreType.Object), + new("GEORADIUSBYMEMBER", RespCommand.GEORADIUSBYMEMBER, StoreType.Object), + new("GEORADIUSBYMEMBER_RO", RespCommand.GEORADIUSBYMEMBER_RO, StoreType.Object), + new("GEOSEARCH", RespCommand.GEOSEARCH, StoreType.Object), + new("GEOSEARCHSTORE", RespCommand.GEOSEARCHSTORE, StoreType.Object), + new("GET", RespCommand.GET, StoreType.Main), + new("GETEX", RespCommand.GETEX, StoreType.Main), + new("GETBIT", RespCommand.GETBIT, StoreType.Main), + new("GETDEL", RespCommand.GETDEL, StoreType.Main), + new("GETIFNOTMATCH", RespCommand.GETIFNOTMATCH, StoreType.Main), + new("GETRANGE", RespCommand.GETRANGE, StoreType.Main), + new("GETWITHETAG", RespCommand.GETWITHETAG, StoreType.Main), + new("GETSET", RespCommand.GETSET, StoreType.Main), + new("HCOLLECT", RespCommand.HCOLLECT, StoreType.Object), + new("HDEL", RespCommand.HDEL, StoreType.Object), new("HELLO", RespCommand.HELLO), - new("HEXISTS", RespCommand.HEXISTS), - new("HEXPIRE", RespCommand.HEXPIRE), - new("HPEXPIRE", RespCommand.HPEXPIRE), - new("HEXPIREAT", RespCommand.HEXPIREAT), - new("HPEXPIREAT", RespCommand.HPEXPIREAT), - new("HTTL", RespCommand.HTTL), - new("HPTTL", RespCommand.HPTTL), - new("HEXPIRETIME", RespCommand.HEXPIRETIME), - new("HPEXPIRETIME", RespCommand.HPEXPIRETIME), - new("HPERSIST", RespCommand.HPERSIST), - new("HGET", RespCommand.HGET), - new("HGETALL", RespCommand.HGETALL), - new("HINCRBY", RespCommand.HINCRBY), - new("HINCRBYFLOAT", RespCommand.HINCRBYFLOAT), - new("HKEYS", RespCommand.HKEYS), - new("HLEN", RespCommand.HLEN), - new("HMGET", RespCommand.HMGET), - new("HMSET", RespCommand.HMSET), - new("HRANDFIELD", RespCommand.HRANDFIELD), - new("HSCAN", RespCommand.HSCAN), - new("HSET", RespCommand.HSET), - new("HSETNX", RespCommand.HSETNX), - new("HSTRLEN", RespCommand.HSTRLEN), - new("HVALS", RespCommand.HVALS), - new("INCR", RespCommand.INCR), - new("INCRBY", RespCommand.INCRBY), - new("INCRBYFLOAT", RespCommand.INCRBYFLOAT), + new("HEXISTS", RespCommand.HEXISTS, StoreType.Object), + new("HEXPIRE", RespCommand.HEXPIRE, StoreType.Object), + new("HPEXPIRE", RespCommand.HPEXPIRE, StoreType.Object), + new("HEXPIREAT", RespCommand.HEXPIREAT, StoreType.Object), + new("HPEXPIREAT", RespCommand.HPEXPIREAT, StoreType.Object), + new("HTTL", RespCommand.HTTL, StoreType.Object), + new("HPTTL", RespCommand.HPTTL, StoreType.Object), + new("HEXPIRETIME", RespCommand.HEXPIRETIME, StoreType.Object), + new("HPEXPIRETIME", RespCommand.HPEXPIRETIME, StoreType.Object), + new("HPERSIST", RespCommand.HPERSIST, StoreType.Object), + new("HGET", RespCommand.HGET, StoreType.Object), + new("HGETALL", RespCommand.HGETALL, StoreType.Object), + new("HINCRBY", RespCommand.HINCRBY, StoreType.Object), + new("HINCRBYFLOAT", RespCommand.HINCRBYFLOAT, StoreType.Object), + new("HKEYS", RespCommand.HKEYS, StoreType.Object), + new("HLEN", RespCommand.HLEN, StoreType.Object), + new("HMGET", RespCommand.HMGET, StoreType.Object), + new("HMSET", RespCommand.HMSET, StoreType.Object), + new("HRANDFIELD", RespCommand.HRANDFIELD, StoreType.Object), + new("HSCAN", RespCommand.HSCAN, StoreType.Object), + new("HSET", RespCommand.HSET, StoreType.Object), + new("HSETNX", RespCommand.HSETNX, StoreType.Object), + new("HSTRLEN", RespCommand.HSTRLEN, StoreType.Object), + new("HVALS", RespCommand.HVALS, StoreType.Object), + new("INCR", RespCommand.INCR, StoreType.Main), + new("INCRBY", RespCommand.INCRBY, StoreType.Main), + new("INCRBYFLOAT", RespCommand.INCRBYFLOAT, StoreType.Main), new("INFO", RespCommand.INFO), new("KEYS", RespCommand.KEYS), - new("LCS", RespCommand.LCS), + new("LCS", RespCommand.LCS, StoreType.Main), new("LASTSAVE", RespCommand.LASTSAVE), - new("LATENCY", RespCommand.LATENCY, + new("LATENCY", RespCommand.LATENCY, StoreType.None, [ new("LATENCY|HELP", RespCommand.LATENCY_HELP), new("LATENCY|HISTOGRAM", RespCommand.LATENCY_HISTOGRAM), new("LATENCY|RESET", RespCommand.LATENCY_RESET), ]), - new("LINDEX", RespCommand.LINDEX), - new("LINSERT", RespCommand.LINSERT), - new("LLEN", RespCommand.LLEN), - new("LMOVE", RespCommand.LMOVE), - new("LMPOP", RespCommand.LMPOP), - new("LPOP", RespCommand.LPOP), - new("LPOS", RespCommand.LPOS), - new("LPUSH", RespCommand.LPUSH), - new("LPUSHX", RespCommand.LPUSHX), - new("LRANGE", RespCommand.LRANGE), - new("LREM", RespCommand.LREM), - new("LSET", RespCommand.LSET), - new("LTRIM", RespCommand.LTRIM), - new("MEMORY", RespCommand.MEMORY, + new("LINDEX", RespCommand.LINDEX, StoreType.Object), + new("LINSERT", RespCommand.LINSERT, StoreType.Object), + new("LLEN", RespCommand.LLEN, StoreType.Object), + new("LMOVE", RespCommand.LMOVE, StoreType.Object), + new("LMPOP", RespCommand.LMPOP, StoreType.Object), + new("LPOP", RespCommand.LPOP, StoreType.Object), + new("LPOS", RespCommand.LPOS, StoreType.Object), + new("LPUSH", RespCommand.LPUSH, StoreType.Object), + new("LPUSHX", RespCommand.LPUSHX, StoreType.Object), + new("LRANGE", RespCommand.LRANGE, StoreType.Object), + new("LREM", RespCommand.LREM, StoreType.Object), + new("LSET", RespCommand.LSET, StoreType.Object), + new("LTRIM", RespCommand.LTRIM, StoreType.Object), + new("MEMORY", RespCommand.MEMORY, StoreType.None, [ new("MEMORY|USAGE", RespCommand.MEMORY_USAGE), ]), - new("MGET", RespCommand.MGET), + new("MGET", RespCommand.MGET, StoreType.Main), new("MIGRATE", RespCommand.MIGRATE), new("PURGEBP", RespCommand.PURGEBP), - new("MODULE", RespCommand.MODULE, + new("MODULE", RespCommand.MODULE, StoreType.None, [ new("MODULE|LOADCS", RespCommand.MODULE_LOADCS), ]), new("MONITOR", RespCommand.MONITOR), - new("MSET", RespCommand.MSET), - new("MSETNX", RespCommand.MSETNX), + new("MSET", RespCommand.MSET, StoreType.Main), + new("MSETNX", RespCommand.MSETNX, StoreType.Main), new("MULTI", RespCommand.MULTI), - new("PERSIST", RespCommand.PERSIST), - new("PEXPIRE", RespCommand.PEXPIRE), - new("PEXPIREAT", RespCommand.PEXPIREAT), - new("PEXPIRETIME", RespCommand.PEXPIRETIME), - new("PFADD", RespCommand.PFADD), - new("PFCOUNT", RespCommand.PFCOUNT), - new("PFMERGE", RespCommand.PFMERGE), + new("PERSIST", RespCommand.PERSIST, StoreType.All), + new("PEXPIRE", RespCommand.PEXPIRE, StoreType.All), + new("PEXPIREAT", RespCommand.PEXPIREAT, StoreType.All), + new("PEXPIRETIME", RespCommand.PEXPIRETIME, StoreType.All), + new("PFADD", RespCommand.PFADD, StoreType.Main), + new("PFCOUNT", RespCommand.PFCOUNT, StoreType.Main), + new("PFMERGE", RespCommand.PFMERGE, StoreType.Main), new("PING", RespCommand.PING), - new("PSETEX", RespCommand.PSETEX), + new("PSETEX", RespCommand.PSETEX, StoreType.Main), new("PSUBSCRIBE", RespCommand.PSUBSCRIBE), - new("PTTL", RespCommand.PTTL), + new("PTTL", RespCommand.PTTL, StoreType.All), new("PUBLISH", RespCommand.PUBLISH), - new("PUBSUB", RespCommand.PUBSUB, + new("PUBSUB", RespCommand.PUBSUB, StoreType.None, [ new("PUBSUB|CHANNELS", RespCommand.PUBSUB_CHANNELS), new("PUBSUB|NUMPAT", RespCommand.PUBSUB_NUMPAT), @@ -251,112 +250,112 @@ public class SupportedCommand new("QUIT", RespCommand.QUIT), new("READONLY", RespCommand.READONLY), new("READWRITE", RespCommand.READWRITE), - new("RENAME", RespCommand.RENAME), - new("RESTORE", RespCommand.RESTORE), - new("RENAMENX", RespCommand.RENAMENX), + new("RENAME", RespCommand.RENAME, StoreType.All), + new("RESTORE", RespCommand.RESTORE, StoreType.All), + new("RENAMENX", RespCommand.RENAMENX, StoreType.All), new("REPLICAOF", RespCommand.REPLICAOF), new("ROLE", RespCommand.ROLE), - new("RPOP", RespCommand.RPOP), - new("RPOPLPUSH", RespCommand.RPOPLPUSH), - new("RPUSH", RespCommand.RPUSH), - new("RPUSHX", RespCommand.RPUSHX), + new("RPOP", RespCommand.RPOP, StoreType.Object), + new("RPOPLPUSH", RespCommand.RPOPLPUSH, StoreType.Object), + new("RPUSH", RespCommand.RPUSH, StoreType.Object), + new("RPUSHX", RespCommand.RPUSHX, StoreType.Object), new("RUNTXP", RespCommand.RUNTXP), - new("SADD", RespCommand.SADD), - new("SCARD", RespCommand.SCARD), + new("SADD", RespCommand.SADD, StoreType.Object), + new("SCARD", RespCommand.SCARD, StoreType.Object), new("SAVE", RespCommand.SAVE), - new("SCAN", RespCommand.SCAN), - new("SDIFF", RespCommand.SDIFF), - new("SDIFFSTORE", RespCommand.SDIFFSTORE), + new("SCAN", RespCommand.SCAN, StoreType.All), + new("SDIFF", RespCommand.SDIFF, StoreType.Object), + new("SDIFFSTORE", RespCommand.SDIFFSTORE, StoreType.Object), new("SECONDARYOF", RespCommand.SECONDARYOF), new("SELECT", RespCommand.SELECT), - new("SET", RespCommand.SET), - new("SETBIT", RespCommand.SETBIT), - new("SETEX", RespCommand.SETEX), - new("SETIFMATCH", RespCommand.SETIFMATCH), - new("SETIFGREATER", RespCommand.SETIFGREATER), - new("SETNX", RespCommand.SETNX), - new("SETRANGE", RespCommand.SETRANGE), - new("SISMEMBER", RespCommand.SISMEMBER), + new("SET", RespCommand.SET, StoreType.Main), + new("SETBIT", RespCommand.SETBIT, StoreType.Main), + new("SETEX", RespCommand.SETEX, StoreType.Main), + new("SETIFMATCH", RespCommand.SETIFMATCH, StoreType.Main), + new("SETIFGREATER", RespCommand.SETIFGREATER, StoreType.Main), + new("SETNX", RespCommand.SETNX, StoreType.Main), + new("SETRANGE", RespCommand.SETRANGE, StoreType.Main), + new("SISMEMBER", RespCommand.SISMEMBER, StoreType.Object), new("SLAVEOF", RespCommand.SECONDARYOF), - new("SLOWLOG", RespCommand.SLOWLOG, + new("SLOWLOG", RespCommand.SLOWLOG, StoreType.None, [ new("SLOWLOG|GET", RespCommand.SLOWLOG_GET), new("SLOWLOG|LEN", RespCommand.SLOWLOG_LEN), new("SLOWLOG|RESET", RespCommand.SLOWLOG_RESET), new("SLOWLOG|HELP", RespCommand.SLOWLOG_HELP), ]), - new("SMEMBERS", RespCommand.SMEMBERS), - new("SMISMEMBER", RespCommand.SMISMEMBER), - new("SMOVE", RespCommand.SMOVE), - new("SPOP", RespCommand.SPOP), + new("SMEMBERS", RespCommand.SMEMBERS, StoreType.Object), + new("SMISMEMBER", RespCommand.SMISMEMBER, StoreType.Object), + new("SMOVE", RespCommand.SMOVE, StoreType.Object), + new("SPOP", RespCommand.SPOP, StoreType.Object), new("SPUBLISH", RespCommand.SPUBLISH), - new("SRANDMEMBER", RespCommand.SRANDMEMBER), - new("SREM", RespCommand.SREM), - new("SSCAN", RespCommand.SSCAN), - new("STRLEN", RespCommand.STRLEN), + new("SRANDMEMBER", RespCommand.SRANDMEMBER, StoreType.Object), + new("SREM", RespCommand.SREM, StoreType.Object), + new("SSCAN", RespCommand.SSCAN, StoreType.Object), + new("STRLEN", RespCommand.STRLEN, StoreType.Main), new("SUBSCRIBE", RespCommand.SUBSCRIBE), new("SSUBSCRIBE", RespCommand.SSUBSCRIBE), - new("SUBSTR", RespCommand.SUBSTR), - new("SUNION", RespCommand.SUNION), - new("SUNIONSTORE", RespCommand.SUNIONSTORE), - new("SINTER", RespCommand.SINTER), - new("SINTERCARD", RespCommand.SINTERCARD), - new("SINTERSTORE", RespCommand.SINTERSTORE), + new("SUBSTR", RespCommand.SUBSTR, StoreType.Main), + new("SUNION", RespCommand.SUNION, StoreType.Object), + new("SUNIONSTORE", RespCommand.SUNIONSTORE, StoreType.Object), + new("SINTER", RespCommand.SINTER, StoreType.Object), + new("SINTERCARD", RespCommand.SINTERCARD, StoreType.Object), + new("SINTERSTORE", RespCommand.SINTERSTORE, StoreType.Object), new("SWAPDB", RespCommand.SWAPDB), new("TIME", RespCommand.TIME), - new("TTL", RespCommand.TTL), - new("TYPE", RespCommand.TYPE), - new("UNLINK", RespCommand.UNLINK), + new("TTL", RespCommand.TTL, StoreType.All), + new("TYPE", RespCommand.TYPE, StoreType.All), + new("UNLINK", RespCommand.UNLINK, StoreType.All), new("UNSUBSCRIBE", RespCommand.UNSUBSCRIBE), new("UNWATCH", RespCommand.UNWATCH), new("WATCH", RespCommand.WATCH), new("WATCHMS", RespCommand.WATCHMS), new("WATCHOS", RespCommand.WATCHOS), - new("ZADD", RespCommand.ZADD), - new("ZCARD", RespCommand.ZCARD), - new("ZCOUNT", RespCommand.ZCOUNT), - new("ZDIFF", RespCommand.ZDIFF), - new("ZDIFFSTORE", RespCommand.ZDIFFSTORE), - new("ZINCRBY", RespCommand.ZINCRBY), - new("ZINTER", RespCommand.ZINTER), - new("ZINTERCARD", RespCommand.ZINTERCARD), - new("ZINTERSTORE", RespCommand.ZINTERSTORE), - new("ZLEXCOUNT", RespCommand.ZLEXCOUNT), - new("ZMSCORE", RespCommand.ZMSCORE), - new("ZMPOP", RespCommand.ZMPOP), - new("ZPOPMAX", RespCommand.ZPOPMAX), - new("ZPOPMIN", RespCommand.ZPOPMIN), - new("ZRANDMEMBER", RespCommand.ZRANDMEMBER), - new("ZRANGE", RespCommand.ZRANGE), - new("ZRANGEBYLEX", RespCommand.ZRANGEBYLEX), - new("ZRANGEBYSCORE", RespCommand.ZRANGEBYSCORE), - new("ZRANGESTORE", RespCommand.ZRANGESTORE), - new("ZRANK", RespCommand.ZRANK), - new("ZREM", RespCommand.ZREM), - new("ZREMRANGEBYLEX", RespCommand.ZREMRANGEBYLEX), - new("ZREMRANGEBYRANK", RespCommand.ZREMRANGEBYRANK), - new("ZREMRANGEBYSCORE", RespCommand.ZREMRANGEBYSCORE), - new("ZREVRANGE", RespCommand.ZREVRANGE), - new("ZREVRANGEBYLEX", RespCommand.ZREVRANGEBYLEX), - new("ZREVRANGEBYSCORE", RespCommand.ZREVRANGEBYSCORE), - new("ZREVRANK", RespCommand.ZREVRANK), - new("ZSCAN", RespCommand.ZSCAN), - new("ZSCORE", RespCommand.ZSCORE), - new("ZEXPIRE", RespCommand.HEXPIRE), - new("ZPEXPIRE", RespCommand.HPEXPIRE), - new("ZEXPIREAT", RespCommand.HEXPIREAT), - new("ZPEXPIREAT", RespCommand.HPEXPIREAT), - new("ZTTL", RespCommand.HTTL), - new("ZPTTL", RespCommand.HPTTL), - new("ZEXPIRETIME", RespCommand.HEXPIRETIME), - new("ZPEXPIRETIME", RespCommand.HPEXPIRETIME), - new("ZPERSIST", RespCommand.HPERSIST), - new("ZCOLLECT", RespCommand.HPERSIST), - new("ZUNION", RespCommand.ZUNION), - new("ZUNIONSTORE", RespCommand.ZUNIONSTORE), + new("ZADD", RespCommand.ZADD, StoreType.Object), + new("ZCARD", RespCommand.ZCARD, StoreType.Object), + new("ZCOUNT", RespCommand.ZCOUNT, StoreType.Object), + new("ZDIFF", RespCommand.ZDIFF, StoreType.Object), + new("ZDIFFSTORE", RespCommand.ZDIFFSTORE, StoreType.Object), + new("ZINCRBY", RespCommand.ZINCRBY, StoreType.Object), + new("ZINTER", RespCommand.ZINTER, StoreType.Object), + new("ZINTERCARD", RespCommand.ZINTERCARD, StoreType.Object), + new("ZINTERSTORE", RespCommand.ZINTERSTORE, StoreType.Object), + new("ZLEXCOUNT", RespCommand.ZLEXCOUNT, StoreType.Object), + new("ZMSCORE", RespCommand.ZMSCORE, StoreType.Object), + new("ZMPOP", RespCommand.ZMPOP, StoreType.Object), + new("ZPOPMAX", RespCommand.ZPOPMAX, StoreType.Object), + new("ZPOPMIN", RespCommand.ZPOPMIN, StoreType.Object), + new("ZRANDMEMBER", RespCommand.ZRANDMEMBER, StoreType.Object), + new("ZRANGE", RespCommand.ZRANGE, StoreType.Object), + new("ZRANGEBYLEX", RespCommand.ZRANGEBYLEX, StoreType.Object), + new("ZRANGEBYSCORE", RespCommand.ZRANGEBYSCORE, StoreType.Object), + new("ZRANGESTORE", RespCommand.ZRANGESTORE, StoreType.Object), + new("ZRANK", RespCommand.ZRANK, StoreType.Object), + new("ZREM", RespCommand.ZREM, StoreType.Object), + new("ZREMRANGEBYLEX", RespCommand.ZREMRANGEBYLEX, StoreType.Object), + new("ZREMRANGEBYRANK", RespCommand.ZREMRANGEBYRANK, StoreType.Object), + new("ZREMRANGEBYSCORE", RespCommand.ZREMRANGEBYSCORE, StoreType.Object), + new("ZREVRANGE", RespCommand.ZREVRANGE, StoreType.Object), + new("ZREVRANGEBYLEX", RespCommand.ZREVRANGEBYLEX, StoreType.Object), + new("ZREVRANGEBYSCORE", RespCommand.ZREVRANGEBYSCORE, StoreType.Object), + new("ZREVRANK", RespCommand.ZREVRANK, StoreType.Object), + new("ZSCAN", RespCommand.ZSCAN, StoreType.Object), + new("ZSCORE", RespCommand.ZSCORE, StoreType.Object), + new("ZEXPIRE", RespCommand.ZEXPIRE, StoreType.Object), + new("ZPEXPIRE", RespCommand.ZPEXPIRE, StoreType.Object), + new("ZEXPIREAT", RespCommand.ZEXPIREAT, StoreType.Object), + new("ZPEXPIREAT", RespCommand.ZPEXPIREAT, StoreType.Object), + new("ZTTL", RespCommand.ZTTL, StoreType.Object), + new("ZPTTL", RespCommand.ZPTTL, StoreType.Object), + new("ZEXPIRETIME", RespCommand.ZEXPIRETIME, StoreType.Object), + new("ZPEXPIRETIME", RespCommand.ZPEXPIRETIME, StoreType.Object), + new("ZPERSIST", RespCommand.ZPERSIST, StoreType.Object), + new("ZCOLLECT", RespCommand.ZCOLLECT, StoreType.Object), + new("ZUNION", RespCommand.ZUNION, StoreType.Object), + new("ZUNIONSTORE", RespCommand.ZUNIONSTORE, StoreType.Object), new("EVAL", RespCommand.EVAL), new("EVALSHA", RespCommand.EVALSHA), - new("SCRIPT", RespCommand.SCRIPT, + new("SCRIPT", RespCommand.SCRIPT, StoreType.None, [ new("SCRIPT|EXISTS", RespCommand.SCRIPT_EXISTS), new("SCRIPT|FLUSH", RespCommand.SCRIPT_FLUSH), @@ -419,6 +418,11 @@ public class SupportedCommand /// public RespCommand RespCommand { get; set; } + /// + /// Store type that the command operates on (None/Main/Object/All). Default: None for commands without key arguments. + /// + public StoreType StoreType { get; set; } + /// /// Default constructor provided for JSON serialization /// @@ -432,12 +436,14 @@ public SupportedCommand() /// /// Supported command name /// RESP Command enum + /// Store type that the command operates on (None/Main/Object/All). Default: None for commands without key arguments. /// List of supported sub-command names (optional) - public SupportedCommand(string command, RespCommand respCommand = RespCommand.NONE, IEnumerable subCommands = null) : this() + public SupportedCommand(string command, RespCommand respCommand = RespCommand.NONE, StoreType storeType = StoreType.None, IEnumerable subCommands = null) : this() { Command = command; SubCommands = subCommands?.ToDictionary(sc => sc.Command, sc => sc); RespCommand = respCommand; + StoreType = storeType; } } } \ No newline at end of file diff --git a/test/Garnet.test/CustomRespCommandsInfo.json b/test/Garnet.test/CustomRespCommandsInfo.json index 72524a6fb6d..6e631b285ad 100644 --- a/test/Garnet.test/CustomRespCommandsInfo.json +++ b/test/Garnet.test/CustomRespCommandsInfo.json @@ -9,7 +9,22 @@ "Step": 1, "AclCategories": "Read", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RM, Delete" + } + ], + "StoreType": "Main", "SubCommands": null }, { @@ -22,7 +37,22 @@ "Step": 1, "AclCategories": "Read", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 2 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": -1, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ], + "StoreType": "Main", "SubCommands": null }, { @@ -35,7 +65,22 @@ "Step": 1, "AclCategories": "Read", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ], + "StoreType": "Object", "SubCommands": null }, { @@ -48,7 +93,22 @@ "Step": 1, "AclCategories": "Write", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Access, Update" + } + ], + "StoreType": "Object", "SubCommands": null }, { @@ -61,7 +121,35 @@ "Step": 1, "AclCategories": "Write", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 2 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 1, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "OW, Update" + }, + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RO, Access" + } + ], + "StoreType": "Main", "SubCommands": null }, { @@ -74,7 +162,22 @@ "Step": 1, "AclCategories": "Write", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Access, Update" + } + ], + "StoreType": "Main", "SubCommands": null }, { @@ -87,7 +190,21 @@ "Step": 1, "AclCategories": "String, Write", "Tips": null, - "KeySpecifications": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Access, Update" + } + ], "SubCommands": null } ] \ No newline at end of file diff --git a/test/Garnet.test/RespCommandTests.cs b/test/Garnet.test/RespCommandTests.cs index f1a87a68414..742e737e62c 100644 --- a/test/Garnet.test/RespCommandTests.cs +++ b/test/Garnet.test/RespCommandTests.cs @@ -218,7 +218,7 @@ public void CommandTest() ClassicAssert.AreEqual(externalRespCommandsInfo.Count, results.Length); // Register custom commands - var customCommandsRegistered = RegisterCustomCommands(); + var customCommandsRegistered = RegisterCustomCommands(["DELIFM", "MGETIFPM", "MYDICTSET", "SETIFPM", "SETWPIFPGT"]); // Dynamically register custom commands var customCommandsRegisteredDyn = DynamicallyRegisterCustomCommands(db); @@ -246,7 +246,7 @@ public void CommandInfoTest() ClassicAssert.AreEqual(externalRespCommandsInfo.Count, results.Length); // Register custom commands - var customCommandsRegistered = RegisterCustomCommands(); + var customCommandsRegistered = RegisterCustomCommands(["DELIFM", "MGETIFPM", "MYDICTSET", "SETIFPM", "SETWPIFPGT"]); // Dynamically register custom commands var customCommandsRegisteredDyn = DynamicallyRegisterCustomCommands(db); @@ -329,7 +329,7 @@ public void CommandCountTest() ClassicAssert.AreEqual(externalRespCommandsInfo.Count, commandCount); // Register custom commands - var customCommandsRegistered = RegisterCustomCommands(); + var customCommandsRegistered = RegisterCustomCommands(["DELIFM", "MGETIFPM", "MYDICTSET", "SETIFPM", "SETWPIFPGT"]); // Dynamically register custom commands var customCommandsRegisteredDyn = DynamicallyRegisterCustomCommands(db); @@ -548,16 +548,32 @@ public void AofIndependentCommandsTest() } } - private string[] RegisterCustomCommands() + private string[] RegisterCustomCommands(List registerCommandsSubset = null) { - var registeredCommands = new[] { "SETIFPM", "MYDICTSET", "MGETIFPM" }; + var commands = new HashSet { "DELIFM", "MGETIFPM", "MYDICTGET", "MYDICTSET", "READWRITETX", "SETIFPM", "SETWPIFPGT" }; + if (registerCommandsSubset != null) + { + commands.IntersectWith(registerCommandsSubset); + CollectionAssert.IsNotEmpty(commands); + } var factory = new MyDictFactory(); - server.Register.NewCommand("SETIFPM", CommandType.ReadModifyWrite, new SetIfPMCustomCommand(), respCustomCommandsInfo["SETIFPM"], respCustomCommandsDocs["SETIFPM"]); - server.Register.NewCommand("MYDICTSET", CommandType.ReadModifyWrite, factory, new MyDictSet(), respCustomCommandsInfo["MYDICTSET"], respCustomCommandsDocs["MYDICTSET"]); - server.Register.NewTransactionProc("MGETIFPM", () => new MGetIfPM(), respCustomCommandsInfo["MGETIFPM"], respCustomCommandsDocs["MGETIFPM"]); - - return registeredCommands; + if (commands.Contains("DELIFM")) + server.Register.NewCommand("DELIFM", CommandType.ReadModifyWrite, new DeleteIfMatchCustomCommand(), respCustomCommandsInfo["DELIFM"], respCustomCommandsDocs["DELIFM"]); + if (commands.Contains("MGETIFPM")) + server.Register.NewTransactionProc("MGETIFPM", () => new MGetIfPM(), respCustomCommandsInfo["MGETIFPM"], respCustomCommandsDocs["MGETIFPM"]); + if (commands.Contains("MYDICTGET")) + server.Register.NewCommand("MYDICTGET", CommandType.Read, factory, new MyDictGet(), respCustomCommandsInfo["MYDICTGET"], respCustomCommandsDocs["MYDICTGET"]); + if (commands.Contains("MYDICTSET")) + server.Register.NewCommand("MYDICTSET", CommandType.ReadModifyWrite, factory, new MyDictSet(), respCustomCommandsInfo["MYDICTSET"], respCustomCommandsDocs["MYDICTSET"]); + if (commands.Contains("READWRITETX")) + server.Register.NewTransactionProc("READWRITETX", () => new ReadWriteTxn(), respCustomCommandsInfo["READWRITETX"], respCustomCommandsDocs["READWRITETX"]); + if (commands.Contains("SETIFPM")) + server.Register.NewCommand("SETIFPM", CommandType.ReadModifyWrite, new SetIfPMCustomCommand(), respCustomCommandsInfo["SETIFPM"], respCustomCommandsDocs["SETIFPM"]); + if (commands.Contains("SETWPIFPGT")) + server.Register.NewCommand("SETWPIFPGT", CommandType.ReadModifyWrite, new SetWPIFPGTCustomCommand(), respCustomCommandsInfo["SETWPIFPGT"], respCustomCommandsDocs["SETWPIFPGT"]); + + return commands.ToArray(); } private (string, string, string) CreateTestLibrary() @@ -695,67 +711,92 @@ private void VerifyCommandDocs(string cmdName, RedisResult result) /// Test COMMAND GETKEYS command with various command signatures /// [Test] - [TestCase("SET", new[] { "mykey" }, new[] { "mykey", "value" }, Description = "Simple SET command")] - [TestCase("MSET", new[] { "key1", "key2" }, new[] { "key1", "value1", "key2", "value2" }, Description = "Multiple SET pairs")] - [TestCase("MGET", new[] { "key1", "key2", "key3" }, new[] { "key1", "key2", "key3" }, Description = "Multiple GET keys")] - [TestCase("ZUNIONSTORE", new[] { "destination", "key1", "key2" }, new[] { "destination", "2", "key1", "key2" }, Description = "ZUNIONSTORE with multiple source keys")] - [TestCase("EVAL", new[] { "key1", "key2" }, new[] { "return redis.call('GET', KEYS[1])", "2", "key1", "key2" }, Description = "EVAL with multiple keys")] - [TestCase("EXPIRE", new[] { "mykey" }, new[] { "mykey", "100", "NX" }, Description = "EXPIRE with NX option")] - [TestCase("MIGRATE", new[] { "key1", "key2" }, new[] { "127.0.0.1", "6379", "", "0", "5000", "KEYS", "key1", "key2" }, Description = "MIGRATE with multiple keys")] - [TestCase("GEOSEARCHSTORE", new[] { "dst", "src" }, new[] { "dst", "src", "FROMMEMBER", "member", "COUNT", "10", "ASC" }, Description = "GEOSEARCHSTORE with options")] - public void CommandGetKeysTest(string command, string[] expectedKeys, string[] args) + [TestCase("SET", new[] { "mykey", "value" }, false, new[] { "mykey" })] + [TestCase("MSET", new[] { "key1", "value1", "key2", "value2" }, false, new[] { "key1", "key2" })] + [TestCase("MGET", new[] { "key1", "key2", "key3" }, false, new[] { "key1", "key2", "key3" })] + [TestCase("ZUNIONSTORE", new[] { "destination", "2", "key1", "key2" }, false, new[] { "destination", "key1", "key2" })] + [TestCase("EVAL", new[] { "return redis.call('GET', KEYS[1])", "2", "key1", "key2" }, false, new[] { "key1", "key2" })] + [TestCase("EXPIRE", new[] { "mykey", "100", "NX" }, false, new[] { "mykey" })] + [TestCase("MIGRATE", new[] { "127.0.0.1", "6379", "", "0", "5000", "KEYS", "key1", "key2" }, false, new[] { "key1", "key2" })] + [TestCase("GEOSEARCHSTORE", new[] { "dst", "src", "FROMMEMBER", "member", "COUNT", "10", "ASC" }, false, new[] { "dst", "src" })] + [TestCase("DELIFM", new[] { "mykey", "value" }, true, new[] { "mykey" })] + [TestCase("MGETIFPM", new[] { "prefix", "key1", "key2", "key3" }, true, new[] { "key1", "key2", "key3" })] + [TestCase("MYDICTGET", new[] { "mykey", "key1" }, true, new[] { "mykey" })] + [TestCase("MYDICTSET", new[] { "mykey", "key1", "val1" }, true, new[] { "mykey" })] + [TestCase("READWRITETX", new[] { "readkey", "writekey1", "writekey2" }, true, new[] { "readkey", "writekey1", "writekey2" })] + [TestCase("SETIFPM", new[] { "mykey", "myvalue", "prefix" }, true, new[] { "mykey" })] + [TestCase("SETWPIFPGT", new[] { "mykey", "myvalue", "prefix" }, true, new[] { "mykey" })] + public void CommandGetKeysTest(string command, string[] args, bool isCustomCmd, string[] expectedKeys) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); - var cmdArgs = new object[] { "GETKEYS", command }.Union(args).ToArray(); - var results = (RedisResult[])db.Execute("COMMAND", cmdArgs); + if (isCustomCmd) + RegisterCustomCommands(); + + var results = (RedisResult[])db.Execute("COMMAND", new object[] { "GETKEYS", command }.Union(args).ToArray()); ClassicAssert.IsNotNull(results); - ClassicAssert.AreEqual(expectedKeys.Length, results.Length); + ClassicAssert.AreEqual(expectedKeys.Length, results!.Length); - for (var i = 0; i < expectedKeys.Length; i++) - { - ClassicAssert.AreEqual(expectedKeys[i], results[i].ToString()); - } + var actualKeys = results.Select(r => r.ToString()).ToArray(); + CollectionAssert.AreEqual(expectedKeys, actualKeys); } /// /// Test COMMAND GETKEYSANDFLAGS command with various command signatures /// [Test] - [TestCase("SET", "mykey", "RW access update variable_flags", new[] { "mykey", "value" }, Description = "Simple SET command")] - [TestCase("MSET", "key1,key2", "OW update|OW update", new[] { "key1", "value1", "key2", "value2" }, Description = "Multiple SET pairs")] - [TestCase("MGET", "key1,key2,key3", "RO access|RO access|RO access", new[] { "key1", "key2", "key3" }, Description = "Multiple GET keys")] - [TestCase("ZUNIONSTORE", "destination,key1,key2", "OW update|RO access|RO access", new[] { "destination", "2", "key1", "key2" }, Description = "ZUNIONSTORE with multiple source keys")] - [TestCase("EVAL", "key1,key2", "RW access update|RW access update", new[] { "return redis.call('GET', KEYS[1])", "2", "key1", "key2" }, Description = "EVAL with multiple keys")] - [TestCase("EXPIRE", "mykey", "RW update", new[] { "mykey", "100", "NX" }, Description = "EXPIRE with NX option")] - [TestCase("MIGRATE", "key1,key2", "RW access delete incomplete|RW access delete incomplete", new[] { "127.0.0.1", "6379", "", "0", "5000", "KEYS", "key1", "key2" }, Description = "MIGRATE with multiple keys")] - [TestCase("GEOSEARCHSTORE", "dst,src", "OW update|RO access", new[] { "dst", "src", "FROMMEMBER", "member", "COUNT", "10", "ASC" }, Description = "GEOSEARCHSTORE with options")] - public void CommandGetKeysAndFlagsTest(string command, string expectedKeysStr, string expectedFlagsStr, string[] args) + [TestCase("SET", new[] { "mykey", "value" }, false, new[] { "mykey" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Update | KeySpecificationFlags.VariableFlags })] + [TestCase("MSET", new[] { "key1", "value1", "key2", "value2" }, false, new[] { "key1", "key2" }, new[] { KeySpecificationFlags.OW | KeySpecificationFlags.Update })] + [TestCase("MGET", new[] { "key1", "key2", "key3" }, false, new[] { "key1", "key2", "key3" }, new[] { KeySpecificationFlags.RO | KeySpecificationFlags.Access })] + [TestCase("ZUNIONSTORE", new[] { "destination", "2", "key1", "key2" }, false, new[] { "destination", "key1", "key2" }, new[] { KeySpecificationFlags.OW | KeySpecificationFlags.Update, KeySpecificationFlags.RO | KeySpecificationFlags.Access, KeySpecificationFlags.RO | KeySpecificationFlags.Access })] + [TestCase("EVAL", new[] { "return redis.call('GET', KEYS[1])", "2", "key1", "key2" }, false, new[] { "key1", "key2" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Update })] + [TestCase("EXPIRE", new[] { "mykey", "100", "NX" }, false, new[] { "mykey" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Update })] + [TestCase("MIGRATE", new[] { "127.0.0.1", "6379", "", "0", "5000", "KEYS", "key1", "key2" }, false, new[] { "key1", "key2" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Delete | KeySpecificationFlags.Incomplete })] + [TestCase("GEOSEARCHSTORE", new[] { "dst", "src", "FROMMEMBER", "member", "COUNT", "10", "ASC" }, false, new[] { "dst", "src" }, new[] { KeySpecificationFlags.OW | KeySpecificationFlags.Update, KeySpecificationFlags.RO | KeySpecificationFlags.Access })] + [TestCase("DELIFM", new[] { "mykey", "value" }, true, new[] { "mykey" }, new[] { KeySpecificationFlags.RM | KeySpecificationFlags.Delete })] + [TestCase("MGETIFPM", new[] { "prefix", "key1", "key2", "key3" }, true, new[] { "key1", "key2", "key3" }, new[] { KeySpecificationFlags.RO | KeySpecificationFlags.Access })] + [TestCase("MYDICTGET", new[] { "mykey", "key1" }, true, new[] { "mykey" }, new[] { KeySpecificationFlags.RO | KeySpecificationFlags.Access })] + [TestCase("MYDICTSET", new[] { "mykey", "key1", "val1" }, true, new[] { "mykey" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Update })] + [TestCase("READWRITETX", new[] { "readkey", "writekey1", "writekey2" }, true, new[] { "readkey", "writekey1", "writekey2" }, new[] { KeySpecificationFlags.RO | KeySpecificationFlags.Access, KeySpecificationFlags.OW | KeySpecificationFlags.Update, KeySpecificationFlags.OW | KeySpecificationFlags.Update })] + [TestCase("SETIFPM", new[] { "mykey", "myvalue", "prefix" }, true, new[] { "mykey" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Update })] + [TestCase("SETWPIFPGT", new[] { "mykey", "myvalue", "prefix" }, true, new[] { "mykey" }, new[] { KeySpecificationFlags.RW | KeySpecificationFlags.Access | KeySpecificationFlags.Update })] + public void CommandGetKeysAndFlagsTest(string command, string[] args, bool isCustomCmd, string[] expectedKeys, KeySpecificationFlags[] keySpecFlags) { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); - var expectedKeys = expectedKeysStr.Split(','); - var expectedFlags = expectedFlagsStr.Split('|') - .Select(f => f.Split(' ', StringSplitOptions.RemoveEmptyEntries)) - .ToArray(); + if (isCustomCmd) + RegisterCustomCommands(); - var cmdArgs = new object[] { "GETKEYSANDFLAGS", command }.Union(args).ToArray(); - var results = (RedisResult[])db.Execute("COMMAND", cmdArgs); + var results = (RedisResult[])db.Execute("COMMAND", new object[] { "GETKEYSANDFLAGS", command }.Union(args).ToArray()); ClassicAssert.IsNotNull(results); - ClassicAssert.AreEqual(expectedKeys.Length, results.Length); + ClassicAssert.AreEqual(expectedKeys.Length, results!.Length); + + var expectedFlags = keySpecFlags.Select(EnumUtils.GetEnumDescriptions).ToArray(); + ClassicAssert.IsTrue(expectedKeys.Length == expectedFlags.Length || expectedFlags.Length == 1); + + // If we are given a single flags argument, we multiply it for all keys + if (expectedFlags.Length == 1 && expectedKeys.Length > 1) + { + expectedFlags = Enumerable.Range(0, expectedKeys.Length).Select(_ => expectedFlags[0]).ToArray(); + } for (var i = 0; i < expectedKeys.Length; i++) { var keyInfo = (RedisResult[])results[i]; - ClassicAssert.AreEqual(2, keyInfo.Length); - ClassicAssert.AreEqual(expectedKeys[i], keyInfo[0].ToString()); + ClassicAssert.IsNotNull(keyInfo); + + ClassicAssert.AreEqual(2, keyInfo!.Length); + var actualKey = keyInfo[0].ToString(); + ClassicAssert.AreEqual(expectedKeys[i], actualKey); - var flags = ((RedisResult[])keyInfo[1]).Select(f => f.ToString()).ToArray(); - CollectionAssert.AreEquivalent(expectedFlags[i], flags); + ClassicAssert.IsNotNull((RedisResult[])keyInfo[1]); + var actualFlags = ((RedisResult[])keyInfo[1])!.Select(r => r.ToString()).ToArray(); + ClassicAssert.IsNotNull(actualFlags); + CollectionAssert.AreEquivalent(expectedFlags[i], actualFlags); } } From 8a547bb9e5418ebbdd5bd19d67746c62cae17feb Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Tue, 23 Sep 2025 16:23:01 -0700 Subject: [PATCH 17/28] wip --- libs/server/API/GarnetApi.cs | 6 - libs/server/API/GarnetApiUnifiedCommands.cs | 10 +- libs/server/API/IGarnetApi.cs | 6 +- libs/server/Resp/KeyAdminCommands.cs | 12 +- .../Functions/ObjectStore/RMWMethods.cs | 6 + .../Common/ArrayKeyIterationFunctions.cs | 200 +++++------------- .../Storage/Session/MainStore/MainStoreOps.cs | 45 ---- test/Garnet.test/RespTests.cs | 65 +++--- 8 files changed, 106 insertions(+), 244 deletions(-) diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index 739eb85171e..fcb493761ac 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -192,12 +192,6 @@ public GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool #endregion - #region PERSIST - /// - public unsafe GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType = StoreType.All) - => storageSession.PERSIST(key, storeType, ref context, ref objectContext); - #endregion - #region Increment (INCR, INCRBY, DECR, DECRBY) /// public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output) diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index 64a0373064f..8c7d738d5bf 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -73,7 +73,15 @@ public GarnetStatus DELETE(PinnedSpanByte key) /// public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext); - + + #endregion + + #region PERSIST + + /// + public unsafe GarnetStatus PERSIST(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) + => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext); + #endregion } } diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index deae876cf21..431ec266339 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -228,13 +228,15 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s #endregion #region PERSIST + /// /// PERSIST /// /// Key - /// Store type: main, object, or both + /// + /// /// - GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType = StoreType.All); + GarnetStatus PERSIST(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output); #endregion #region Increment (INCR, INCRBY, DECR, DECRBY) diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index 4212e71401d..80fb33e7c3a 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -495,12 +495,18 @@ private bool NetworkPERSIST(ref TGarnetApi storageApi) } var key = parseState.GetArgSliceByRef(0); - var status = storageApi.PERSIST(key); + + // Prepare input + var input = new UnifiedStoreInput(RespCommand.PERSIST); + + // Prepare GarnetUnifiedStoreOutput output + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + + var status = storageApi.PERSIST(key, ref input, ref output); if (status == GarnetStatus.OK) { - while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_RETURN_VAL_1, ref dcurr, dend)) - SendAndReset(); + ProcessOutput(output.SpanByteAndMemory); } else { diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index 9eba297345b..dd2b71a7a81 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -99,6 +99,12 @@ public void PostInitialUpdater(ref LogRecord dstLogRecord, in RecordSizeInfo siz /// public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) { + if (!logRecord.Info.ValueIsObject) + { + output.OutputFlags |= OutputFlags.WrongType; + return true; + } + if (InPlaceUpdaterWorker(ref logRecord, in sizeInfo, ref input, ref output, ref rmwInfo, out long sizeChange)) { if (!logRecord.Info.Modified) diff --git a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs index 3ab38e722f3..62396525862 100644 --- a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs +++ b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs @@ -13,22 +13,18 @@ namespace Garnet.server sealed partial class StorageSession : IDisposable { // These contain classes so instantiate once and re-initialize - private ArrayKeyIterationFunctions.MainStoreGetDBSize mainStoreDbSizeFuncs; - private ArrayKeyIterationFunctions.ObjectStoreGetDBSize objectStoreDbSizeFuncs; + private ArrayKeyIterationFunctions.UnifiedStoreGetDBSize unifiedStoreDbSizeFuncs; - // Iterators for SCAN command - private ArrayKeyIterationFunctions.MainStoreGetDBKeys mainStoreDbScanFuncs; - private ArrayKeyIterationFunctions.ObjectStoreGetDBKeys objStoreDbScanFuncs; + // Iterator for SCAN command + private ArrayKeyIterationFunctions.UnifiedStoreGetDBKeys unifiedStoreDbScanFuncs; - // Iterators for expired key deletion + // Iterator for expired key deletion private ArrayKeyIterationFunctions.MainStoreExpiredKeyDeletionScan expiredKeyDeletionScanFuncs; - // Iterators for KEYS command - private ArrayKeyIterationFunctions.MainStoreGetDBKeys mainStoreDbKeysFuncs; - private ArrayKeyIterationFunctions.ObjectStoreGetDBKeys objStoreDbKeysFuncs; + // Iterator for KEYS command + private ArrayKeyIterationFunctions.UnifiedStoreGetDBKeys unifiedStoreDbKeysFuncs; long lastScanCursor; - List objStoreKeys; List Keys; /// @@ -45,13 +41,9 @@ sealed partial class StorageSession : IDisposable /// internal unsafe bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, out long storeCursor, out List keys, long count = 10, ReadOnlySpan typeObject = default) { - const long IsObjectStoreCursor = 1L << LogAddress.kAddressBits; Keys ??= new(); Keys.Clear(); - objStoreKeys ??= new(); - objStoreKeys.Clear(); - keys = Keys; Type matchType = null; @@ -81,35 +73,16 @@ internal unsafe bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, } } - byte* patternPtr = patternB.ToPointer(); + var patternPtr = patternB.ToPointer(); - mainStoreDbScanFuncs ??= new(); - mainStoreDbScanFuncs.Initialize(Keys, allKeys ? null : patternPtr, patternB.Length); - objStoreDbScanFuncs ??= new(); - objStoreDbScanFuncs.Initialize(objStoreKeys, allKeys ? null : patternPtr, patternB.Length, matchType); + unifiedStoreDbScanFuncs ??= new(); + unifiedStoreDbScanFuncs.Initialize(Keys, allKeys ? null : patternPtr, patternB.Length, matchType); storeCursor = cursor; - long remainingCount = count; + var remainingCount = count; - // Cursor is zero or not an object store address - // Scan main store only for string or default key type - if ((cursor & IsObjectStoreCursor) == 0 && (typeObject.IsEmpty || typeObject.SequenceEqual(CmdStrings.STRING) || typeObject.SequenceEqual(CmdStrings.stringt))) - { - basicContext.Session.ScanCursor(ref storeCursor, count, mainStoreDbScanFuncs, validateCursor: cursor != 0 && cursor != lastScanCursor); - remainingCount -= Keys.Count; - } - - // Scan object store with the type parameter - // Check the cursor value corresponds to the object store - if (!objectStoreBasicContext.IsNull && remainingCount > 0 && (typeObject.IsEmpty || (!typeObject.SequenceEqual(CmdStrings.STRING) && !typeObject.SequenceEqual(CmdStrings.stringt)))) - { - var validateCursor = storeCursor != 0 && storeCursor != lastScanCursor; - storeCursor &= ~IsObjectStoreCursor; - objectStoreBasicContext.Session.ScanCursor(ref storeCursor, remainingCount, objStoreDbScanFuncs, validateCursor: validateCursor); - if (storeCursor != 0) - storeCursor |= IsObjectStoreCursor; - Keys.AddRange(objStoreKeys); - } + unifiedStoreBasicContext.Session.ScanCursor(ref storeCursor, count, unifiedStoreDbScanFuncs, validateCursor: cursor != 0 && cursor != lastScanCursor); + remainingCount -= Keys.Count; remainingCount -= Keys.Count; lastScanCursor = storeCursor; return true; @@ -122,7 +95,7 @@ internal unsafe bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, { expiredKeyDeletionScanFuncs ??= new(); expiredKeyDeletionScanFuncs.Initialize(this); - _ = basicContext.Session.ScanCursor(ref fromAddress, untilAddress, expiredKeyDeletionScanFuncs); + _ = unifiedStoreBasicContext.Session.ScanCursor(ref fromAddress, untilAddress, expiredKeyDeletionScanFuncs); return (expiredKeyDeletionScanFuncs.deletedCount, expiredKeyDeletionScanFuncs.totalCount); } @@ -159,7 +132,7 @@ internal ITsavoriteScanIterator IterateMainStore() /// /// internal bool IterateObjectStore(ref TScanFunctions scanFunctions, ref long cursor, long untilAddress = -1, long maxAddress = long.MaxValue, bool validateCursor = false, bool includeTombstones = false) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => objectStoreBasicContext.Session.IterateLookup(ref scanFunctions, ref cursor, untilAddress, validateCursor: validateCursor, maxAddress: maxAddress, resetCursor: false, includeTombstones: includeTombstones); /// @@ -168,6 +141,27 @@ internal bool IterateObjectStore(ref TScanFunctions scanFunction internal ITsavoriteScanIterator IterateObjectStore() => objectStoreBasicContext.Session.Iterate(); + /// + /// Iterate the contents of the unified store (push-based) + /// + /// + /// + /// + /// + /// + /// + /// + /// + internal bool IterateUnifiedStore(ref TScanFunctions scanFunctions, ref long cursor, long untilAddress = -1, long maxAddress = long.MaxValue, bool validateCursor = false, bool includeTombstones = false) + where TScanFunctions : IScanIteratorFunctions + => unifiedStoreBasicContext.Session.IterateLookup(ref scanFunctions, ref cursor, untilAddress, validateCursor: validateCursor, maxAddress: maxAddress, resetCursor: false, includeTombstones: includeTombstones); + + /// + /// Iterate the contents of the unified store (pull based) + /// + internal ITsavoriteScanIterator IterateUnifiedStore() + => unifiedStoreBasicContext.Session.Iterate(); + /// /// Get a list of the keys in the store and object store when using pattern /// @@ -179,16 +173,9 @@ internal unsafe List DBKeys(PinnedSpanByte pattern) var allKeys = *pattern.ToPointer() == '*' && pattern.Length == 1; - mainStoreDbKeysFuncs ??= new(); - mainStoreDbKeysFuncs.Initialize(Keys, allKeys ? null : pattern.ToPointer(), pattern.Length); - basicContext.Session.Iterate(ref mainStoreDbKeysFuncs); - - if (!objectStoreBasicContext.IsNull) - { - objStoreDbKeysFuncs ??= new(); - objStoreDbKeysFuncs.Initialize(Keys, allKeys ? null : pattern.ToPointer(), pattern.Length, matchType: null); - objectStoreBasicContext.Session.Iterate(ref objStoreDbKeysFuncs); - } + unifiedStoreDbKeysFuncs ??= new(); + unifiedStoreDbKeysFuncs.Initialize(Keys, allKeys ? null : pattern.ToPointer(), pattern.Length); + unifiedStoreBasicContext.Session.Iterate(ref unifiedStoreDbKeysFuncs); return Keys; } @@ -199,21 +186,12 @@ internal unsafe List DBKeys(PinnedSpanByte pattern) /// internal int DbSize() { - mainStoreDbSizeFuncs ??= new(); - mainStoreDbSizeFuncs.Initialize(); + unifiedStoreDbSizeFuncs ??= new(); + unifiedStoreDbSizeFuncs.Initialize(); long cursor = 0; - basicContext.Session.ScanCursor(ref cursor, long.MaxValue, mainStoreDbSizeFuncs); - int count = mainStoreDbSizeFuncs.Count; - if (objectStoreBasicContext.Session != null) - { - objectStoreDbSizeFuncs ??= new(); - objectStoreDbSizeFuncs.Initialize(); - cursor = 0; - _ = objectStoreBasicContext.Session.ScanCursor(ref cursor, long.MaxValue, objectStoreDbSizeFuncs); - count += objectStoreDbSizeFuncs.Count; - } + unifiedStoreBasicContext.Session.ScanCursor(ref cursor, long.MaxValue, unifiedStoreDbSizeFuncs); - return count; + return unifiedStoreDbSizeFuncs.Count; } internal static unsafe class ArrayKeyIterationFunctions @@ -235,20 +213,17 @@ internal void Initialize(List keys, byte* patternB, int length, Type mat } } - internal sealed class ObjectStoreExpiredKeyDeletionScan : ExpiredKeysBase - { - protected override bool DeleteIfExpiredInMemory(in TSourceLogRecord logRecord, RecordMetadata recordMetadata) - { - var input = new ObjectInput(new RespInputHeader(GarnetObjectType.DelIfExpIm)); - var output = new GarnetObjectStoreOutput(); - return GarnetStatus.OK == storageSession.RMW_ObjectStore(logRecord.Key, ref input, ref output, ref storageSession.objectStoreBasicContext); - } - } - internal sealed class MainStoreExpiredKeyDeletionScan : ExpiredKeysBase { protected override bool DeleteIfExpiredInMemory(in TSourceLogRecord logRecord, RecordMetadata recordMetadata) { + if (logRecord.Info.ValueIsObject) + { + var objInput = new ObjectInput(new RespInputHeader(GarnetObjectType.DelIfExpIm)); + var output = new GarnetObjectStoreOutput(); + return GarnetStatus.OK == storageSession.RMW_ObjectStore(logRecord.Key, ref objInput, ref output, ref storageSession.objectStoreBasicContext); + } + var input = new RawStringInput(RespCommand.DELIFEXPIM); return GarnetStatus.OK == storageSession.DEL_Conditional(PinnedSpanByte.FromPinnedSpan(logRecord.Key), ref input, ref storageSession.basicContext); } @@ -294,56 +269,11 @@ public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } } - internal sealed class MainStoreGetDBKeys : IScanIteratorFunctions + internal sealed class UnifiedStoreGetDBKeys : IScanIteratorFunctions { private readonly GetDBKeysInfo info; - internal MainStoreGetDBKeys() => info = new(); - - internal void Initialize(List keys, byte* patternB, int length) - => info.Initialize(keys, patternB, length); - - public bool Reader(in TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - where TSourceLogRecord : ISourceLogRecord - { - var key = logRecord.Key; - - if (CheckExpiry(in logRecord)) - { - cursorRecordResult = CursorRecordResult.Skip; - return true; - } - - if (info.patternB != null) - { - bool ok; - if (logRecord.IsPinnedKey) - ok = GlobUtils.Match(info.patternB, info.patternLength, logRecord.PinnedKeyPointer, key.Length, true); - else - fixed (byte* keyPtr = key) - ok = GlobUtils.Match(info.patternB, info.patternLength, keyPtr, key.Length, true); - if (!ok) - { - cursorRecordResult = CursorRecordResult.Skip; - return true; - } - } - - info.keys.Add(key.ToArray()); - cursorRecordResult = CursorRecordResult.Accept; - return true; - } - - public bool OnStart(long beginAddress, long endAddress) => true; - public void OnStop(bool completed, long numberOfRecords) { } - public void OnException(Exception exception, long numberOfRecords) { } - } - - internal sealed class ObjectStoreGetDBKeys : IScanIteratorFunctions - { - private readonly GetDBKeysInfo info; - - internal ObjectStoreGetDBKeys() => info = new(); + internal UnifiedStoreGetDBKeys() => info = new(); internal void Initialize(List keys, byte* patternB, int length, Type matchType = null) => info.Initialize(keys, patternB, length, matchType); @@ -397,37 +327,13 @@ internal class GetDBSizeInfo internal void Initialize() => count = 0; } - internal sealed class MainStoreGetDBSize : IScanIteratorFunctions - { - private readonly GetDBSizeInfo info; - - internal int Count => info.count; - - internal MainStoreGetDBSize() => info = new(); - - internal void Initialize() => info.Initialize(); - - public bool Reader(in TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - where TSourceLogRecord : ISourceLogRecord - { - cursorRecordResult = CursorRecordResult.Skip; - if (!CheckExpiry(in logRecord)) - ++info.count; - return true; - } - - public bool OnStart(long beginAddress, long endAddress) => true; - public void OnStop(bool completed, long numberOfRecords) { } - public void OnException(Exception exception, long numberOfRecords) { } - } - - internal sealed class ObjectStoreGetDBSize : IScanIteratorFunctions + internal sealed class UnifiedStoreGetDBSize : IScanIteratorFunctions { private readonly GetDBSizeInfo info; internal int Count => info.count; - internal ObjectStoreGetDBSize() => info = new(); + internal UnifiedStoreGetDBSize() => info = new(); internal void Initialize() => info.Initialize(); diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index c35bb0bcbfd..27c7dac1cba 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -937,51 +937,6 @@ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } - public unsafe GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - GarnetStatus status = GarnetStatus.NOTFOUND; - - var inputHeader = new RawStringInput(RespCommand.PERSIST); - - var pbOutput = stackalloc byte[8]; - var o = SpanByteAndMemory.FromPinnedPointer(pbOutput, 8); - - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var _status = context.RMW(key.ReadOnlySpan, ref inputHeader, ref o); - - if (_status.IsPending) - CompletePendingForSession(ref _status, ref o, ref context); - - Debug.Assert(o.IsSpanByte); - if (o.SpanByte.ReadOnlySpan[0] == 1) - status = GarnetStatus.OK; - } - - if (status == GarnetStatus.NOTFOUND && (storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) - { - // Retry on object store - var header = new RespInputHeader(GarnetObjectType.Persist); - var objInput = new ObjectInput(header); - - var objO = new GarnetObjectStoreOutput(o); - var _key = key.ToArray(); - var _status = objectStoreContext.RMW(key.ReadOnlySpan, ref objInput, ref objO); - - if (_status.IsPending) - CompletePendingForObjectStoreSession(ref _status, ref objO, ref objectStoreContext); - - Debug.Assert(o.IsSpanByte); - if (o.SpanByte.ReadOnlySpan.Slice(0, CmdStrings.RESP_RETURN_VAL_1.Length) - .SequenceEqual(CmdStrings.RESP_RETURN_VAL_1)) - status = GarnetStatus.OK; - } - - return status; - } - /// /// For existing keys - overwrites part of the value at a specified offset (in-place if possible) /// For non-existing keys - creates a new string with the value at a specified offset (padded with '\0's) diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index 9c91a771fe6..6162397eec2 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -1500,36 +1500,6 @@ public void SingleDeleteWithObjectStoreDisable_LTM() } } - [Test] - public void GarnetObjectStoreDisabledError() - { - TearDown(); - TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); - server.Start(); - - using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); - var db = redis.GetDatabase(0); - - var iter = 100; - var mykey = "mykey"; - for (var i = 0; i < iter; i++) - { - var exception = Assert.Throws(() => _ = db.ListLength(mykey)); - ClassicAssert.AreEqual("ERR Garnet Exception: Object store is disabled", exception.Message); - } - - // Ensure connection is still healthy - for (var i = 0; i < iter; i++) - { - var myvalue = "myvalue" + i; - var result = db.StringSet(mykey, myvalue); - ClassicAssert.IsTrue(result); - var returned = (string)db.StringGet(mykey); - ClassicAssert.AreEqual(myvalue, returned); - } - } - [Test] public void MultiKeyDelete([Values] bool withoutObjectStore) { @@ -3388,21 +3358,36 @@ public void MainObjectKey() // Do StringSet ClassicAssert.IsTrue(db.StringSet(key, "v1")); - // Do SetAdd using the same key - ClassicAssert.IsTrue(db.SetAdd(key, "v2")); + // Do SetAdd using the same key, expected error + Assert.Throws(() => db.SetAdd(key, "v2"), + Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE)); - // Two keys "test:1" - this is expected as of now - // because Garnet has a separate main and object store + // One key "test:1" with a string value is expected var keys = server.Keys(db.Database, key).ToList(); - ClassicAssert.AreEqual(2, keys.Count); + ClassicAssert.AreEqual(1, keys.Count); ClassicAssert.AreEqual(key, (string)keys[0]); - ClassicAssert.AreEqual(key, (string)keys[1]); + var value = db.StringGet(key); + ClassicAssert.AreEqual("v1", (string)value); // do ListRightPush using the same key, expected error - var ex = Assert.Throws(() => db.ListRightPush(key, "v3")); - var expectedError = Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE); - ClassicAssert.IsNotNull(ex); - ClassicAssert.AreEqual(expectedError, ex.Message); + Assert.Throws(() => db.ListRightPush(key, "v3"), Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE)); + + // Delete the key + ClassicAssert.IsTrue(db.KeyDelete(key)); + + // Do SetAdd using the same key + ClassicAssert.IsTrue(db.SetAdd(key, "v2")); + + // Do StringIncrement using the same key, expected error + //Assert.Throws(() => db.StringIncrement(key), Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE)); + + // One key "test:1" with a set value is expected + keys = server.Keys(db.Database, key).ToList(); + ClassicAssert.AreEqual(1, keys.Count); + ClassicAssert.AreEqual(key, (string)keys[0]); + var members = db.SetMembers(key); + ClassicAssert.AreEqual(1, members.Length); + ClassicAssert.AreEqual("v2", (string)members[0]); } [Test] From bf2618a557a39ba545a4b3cee6a3db70e2f6288f Mon Sep 17 00:00:00 2001 From: Vasileios Zois <96085550+vazois@users.noreply.github.com> Date: Tue, 23 Sep 2025 18:45:14 -0700 Subject: [PATCH 18/28] Ensure Streaming Checkpoint Cancellation on Timeout (#1385) * cancel streaming checkpoint if timeout occurred * propagate cancellation token to stateMachineDriver when initiating a checkpoint --- .../ReplicationSyncManager.cs | 53 ++++++++++++------- .../Index/Checkpointing/StateMachineDriver.cs | 6 ++- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 19 +++---- 3 files changed, 48 insertions(+), 30 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index e57e0f37b33..f89213eac2b 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -280,47 +280,60 @@ async Task TakeStreamingCheckpoint() // Iterate through main store var mainStoreCheckpointTask = ClusterProvider.storeWrapper.store. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, cancellationToken: cts.Token, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); var result = await WaitOrDie(checkpointTask: mainStoreCheckpointTask, iteratorManager: manager); if (!result.success) - throw new InvalidOperationException("Main store checkpoint stream failed!"); + throw new GarnetException("Main store checkpoint stream failed!"); if (!ClusterProvider.serverOptions.DisableObjects) { // Iterate through object store var objectStoreCheckpointTask = ClusterProvider.storeWrapper.objectStore. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, cancellationToken: cts.Token, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); result = await WaitOrDie(checkpointTask: objectStoreCheckpointTask, iteratorManager: manager); if (!result.success) - throw new InvalidOperationException("Object store checkpoint stream failed!"); + throw new GarnetException("Object store checkpoint stream failed!"); } // Note: We do not truncate the AOF here as this was just a "virtual" checkpoint - + // WaitOrDie is needed here to check if streaming checkpoint is making progress. + // We cannot use a timeout on the cancellationToken because we don't know in total how long the streaming checkpoint will take async ValueTask<(bool success, Guid token)> WaitOrDie(ValueTask<(bool success, Guid token)> checkpointTask, SnapshotIteratorManager iteratorManager) { - var timeout = replicaSyncTimeout; - var delay = TimeSpan.FromSeconds(1); - while (true) + try { - // Check if cancellation requested - cts.Token.ThrowIfCancellationRequested(); + var timeout = replicaSyncTimeout; + var delay = TimeSpan.FromSeconds(1); + while (true) + { + // Check if cancellation requested + cts.Token.ThrowIfCancellationRequested(); - // Wait for stream sync to make some progress - await Task.Delay(delay); + // Wait for stream sync to make some progress + await Task.Delay(delay); - // Check if checkpoint has completed - if (checkpointTask.IsCompleted) - return await checkpointTask; + // Check if checkpoint has completed + if (checkpointTask.IsCompleted) + return await checkpointTask; - // Check if we made some progress - timeout = !manager.IsProgressing() ? timeout.Subtract(delay) : replicaSyncTimeout; + // Check if we made some progress + timeout = !manager.IsProgressing() ? timeout.Subtract(delay) : replicaSyncTimeout; - // Throw timeout equals to zero - if (timeout.TotalSeconds <= 0) - throw new TimeoutException("Streaming snapshot checkpoint timed out"); + // Throw timeout equals to zero + if (timeout.TotalSeconds <= 0) + { + cts.Cancel(); + throw new TimeoutException("Streaming snapshot checkpoint timed out"); + } + } + } + catch (Exception ex) + { + logger?.LogError(ex, "{method} faulted", nameof(WaitOrDie)); + cts.Cancel(); } + return (false, default); } } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 85f909bbe3c..e9fceba76cf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -299,7 +299,11 @@ async Task RunStateMachine(CancellationToken token = default) _ = Interlocked.Exchange(ref stateMachine, null); if (ex != null) { - _ = _stateMachineCompleted.TrySetException(ex); + // If the state machine stopped due to cancellation, propagate cancellation to the completion TCS + if (ex is OperationCanceledException || ex is TaskCanceledException) + _ = _stateMachineCompleted.TrySetCanceled(); + else + _ = _stateMachineCompleted.TrySetException(ex); } else { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 23849f07013..1010d8faac6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -188,12 +188,13 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun /// Checkpoint token /// Checkpoint type /// Iterator for streaming snapshot records + /// Caller's cancellation token /// /// Whether we successfully initiated the checkpoint (initiation may /// fail if we are already taking a checkpoint or performing some other /// operation such as growing the index). Use CompleteCheckpointAsync to wait completion. /// - public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null, CancellationToken cancellationToken = default) { IStateMachine stateMachine; @@ -208,7 +209,7 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT { stateMachine = Checkpoint.Full(this, checkpointType, out token); } - return stateMachineDriver.Register(stateMachine); + return stateMachineDriver.Register(stateMachine, cancellationToken); } /// @@ -228,7 +229,7 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT public async ValueTask<(bool success, Guid token)> TakeFullCheckpointAsync(CheckpointType checkpointType, CancellationToken cancellationToken = default, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { - var success = TryInitiateFullCheckpoint(out Guid token, checkpointType, streamingSnapshotIteratorFunctions); + var success = TryInitiateFullCheckpoint(out Guid token, checkpointType, streamingSnapshotIteratorFunctions, cancellationToken); if (success) await CompleteCheckpointAsync(cancellationToken).ConfigureAwait(false); @@ -241,10 +242,10 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// /// Checkpoint token /// Whether we could initiate the checkpoint. Use CompleteCheckpointAsync to wait completion. - public bool TryInitiateIndexCheckpoint(out Guid token) + public bool TryInitiateIndexCheckpoint(out Guid token, CancellationToken cancellationToken = default) { var stateMachine = Checkpoint.IndexOnly(this, out token); - return stateMachineDriver.Register(stateMachine); + return stateMachineDriver.Register(stateMachine, cancellationToken); } /// @@ -261,7 +262,7 @@ public bool TryInitiateIndexCheckpoint(out Guid token) /// public async ValueTask<(bool success, Guid token)> TakeIndexCheckpointAsync(CancellationToken cancellationToken = default) { - var success = TryInitiateIndexCheckpoint(out Guid token); + var success = TryInitiateIndexCheckpoint(out Guid token, cancellationToken); if (success) await CompleteCheckpointAsync(cancellationToken).ConfigureAwait(false); @@ -277,7 +278,7 @@ public bool TryInitiateIndexCheckpoint(out Guid token) /// For snapshot, try to store as incremental delta over last snapshot /// Whether we could initiate the checkpoint. Use CompleteCheckpointAsync to wait completion. public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkpointType, bool tryIncremental = false, - IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null, CancellationToken cancellationToken = default) { IStateMachine stateMachine; @@ -305,7 +306,7 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, out token); } } - return stateMachineDriver.Register(stateMachine); + return stateMachineDriver.Register(stateMachine, cancellationToken); } /// @@ -340,7 +341,7 @@ public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid public async ValueTask<(bool success, Guid token)> TakeHybridLogCheckpointAsync(CheckpointType checkpointType, bool tryIncremental = false, CancellationToken cancellationToken = default) { - var success = TryInitiateHybridLogCheckpoint(out Guid token, checkpointType, tryIncremental); + var success = TryInitiateHybridLogCheckpoint(out Guid token, checkpointType, tryIncremental, cancellationToken: cancellationToken); if (success) await CompleteCheckpointAsync(cancellationToken).ConfigureAwait(false); From d2d6944b5daa90fcb4a4bffa52af0065eb68cbdb Mon Sep 17 00:00:00 2001 From: Tejas Kulkarni Date: Thu, 25 Sep 2025 00:09:39 +0530 Subject: [PATCH 19/28] Resetting replication recovery when cluster reset is issued (#1319) * Resetting replication recovery when cluster reset is issued * validating using clusterendpoint during reset * missed flag change * using reset cancellation token * Merge fix * ADded other recovery statuses during reset recovery. * using storeWrapper.serverOptions.ReplicaAttachTimeout for replica diskless sync * ran dotnet formatter * added reset during replication tests * ran dotnet format * removed if debug * if debug encompasses using statements * version bump --------- Co-authored-by: Tejas Kulkarni Co-authored-by: Vasileios Zois <96085550+vazois@users.noreply.github.com> --- Version.props | 2 +- .../Server/ClusterManagerWorkerState.cs | 9 +- .../ReplicaOps/ReplicaDisklessSync.cs | 15 +- .../ReplicaOps/ReplicaReceiveCheckpoint.cs | 14 +- .../Server/Replication/ReplicationManager.cs | 17 ++ libs/common/ExceptionInjectionType.cs | 8 + .../ClusterResetDuringReplicationTests.cs | 154 ++++++++++++++++++ 7 files changed, 214 insertions(+), 5 deletions(-) create mode 100644 test/Garnet.test.cluster/ReplicationTests/ClusterResetDuringReplicationTests.cs diff --git a/Version.props b/Version.props index 20e5abf6635..45fcee605a6 100644 --- a/Version.props +++ b/Version.props @@ -1,6 +1,6 @@ - 1.0.83 + 1.0.84 diff --git a/libs/cluster/Server/ClusterManagerWorkerState.cs b/libs/cluster/Server/ClusterManagerWorkerState.cs index 8d83c355ca5..15f4f308a94 100644 --- a/libs/cluster/Server/ClusterManagerWorkerState.cs +++ b/libs/cluster/Server/ClusterManagerWorkerState.cs @@ -100,6 +100,10 @@ public ReadOnlySpan TryReset(bool soft, int expirySeconds = 60) try { SuspendConfigMerge(); + + // Reset recovery operations before proceeding with reset + clusterProvider.replicationManager.ResetRecovery(); + var resp = CmdStrings.RESP_OK; while (true) { @@ -113,8 +117,9 @@ public ReadOnlySpan TryReset(bool soft, int expirySeconds = 60) this.clusterConnectionStore.CloseAll(); var newNodeId = soft ? current.LocalNodeId : Generator.CreateHexId(); - var address = current.LocalNodeIp; - var port = current.LocalNodePort; + var endpoint = clusterProvider.storeWrapper.GetClusterEndpoint(); + var address = endpoint.Address.ToString(); + var port = endpoint.Port; var configEpoch = soft ? current.LocalNodeConfigEpoch : 0; var expiry = DateTimeOffset.UtcNow.Ticks + TimeSpan.FromSeconds(expirySeconds).Ticks; diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 01b577e5027..7bde92ff39b 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -4,9 +4,11 @@ using System; using System.Net; using System.Text; +using System.Threading; using System.Threading.Tasks; using Garnet.client; using Garnet.cluster.Server.Replication; +using Garnet.common; using Microsoft.Extensions.Logging; namespace Garnet.cluster @@ -58,6 +60,7 @@ async Task TryBeginReplicaSync(bool downgradeLock) var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync; var disableObjects = clusterProvider.serverOptions.DisableObjects; GarnetClientSession gcs = null; + resetHandler ??= new CancellationTokenSource(); try { if (!clusterProvider.serverOptions.EnableFastCommit) @@ -119,7 +122,12 @@ async Task TryBeginReplicaSync(bool downgradeLock) currentReplicationOffset: ReplicationOffset, checkpointEntry: checkpointEntry); - var resp = await gcs.ExecuteAttachSync(syncMetadata.ToByteArray()).ConfigureAwait(false); + using var linkedCts = CancellationTokenSource.CreateLinkedTokenSource(ctsRepManager.Token, resetHandler.Token); + + // Exception injection point for testing cluster reset during diskless replication + await ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Replication_InProgress_During_Diskless_Replica_Attach_Sync).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false); + + var resp = await gcs.ExecuteAttachSync(syncMetadata.ToByteArray()).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false); } catch (Exception ex) { @@ -144,6 +152,11 @@ async Task TryBeginReplicaSync(bool downgradeLock) } gcs?.Dispose(); recvCheckpointHandler?.Dispose(); + if (!resetHandler.TryReset()) + { + resetHandler.Dispose(); + resetHandler = new CancellationTokenSource(); + } } return null; } diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs index 36ef01d4520..1f6e697dd30 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs @@ -6,9 +6,11 @@ using System.IO; using System.Net; using System.Text; +using System.Threading; using System.Threading.Tasks; using Garnet.client; using Garnet.cluster.Server.Replication; +using Garnet.common; using Garnet.server; using Microsoft.Extensions.Logging; using Tsavorite.core; @@ -72,6 +74,7 @@ async Task ReplicaSyncAttachTask(bool downgradeLock) { Debug.Assert(IsRecovering); GarnetClientSession gcs = null; + resetHandler ??= new CancellationTokenSource(); try { // Immediately try to connect to a primary, so we FAIL @@ -139,12 +142,16 @@ async Task ReplicaSyncAttachTask(bool downgradeLock) // 4. Replica responds with aofStartAddress sync // 5. Primary will initiate aof sync task // 6. Primary releases checkpoint + using var linkedCts = CancellationTokenSource.CreateLinkedTokenSource(ctsRepManager.Token, resetHandler.Token); + + // Exception injection point for testing cluster reset during disk-based replication + await ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Replication_InProgress_During_DiskBased_Replica_Attach_Sync).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false); var resp = await gcs.ExecuteReplicaSync( nodeId, PrimaryReplId, cEntry.ToByteArray(), storeWrapper.appendOnlyFile.BeginAddress, - storeWrapper.appendOnlyFile.TailAddress).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, ctsRepManager.Token).ConfigureAwait(false); + storeWrapper.appendOnlyFile.TailAddress).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false); } catch (Exception ex) { @@ -167,6 +174,11 @@ async Task ReplicaSyncAttachTask(bool downgradeLock) } recvCheckpointHandler?.Dispose(); gcs?.Dispose(); + if (!resetHandler.TryReset()) + { + resetHandler.Dispose(); + resetHandler = new CancellationTokenSource(); + } } return null; } diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index 7eecb80878e..63d85866fcc 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -25,6 +25,7 @@ internal sealed partial class ReplicationManager : IDisposable readonly CheckpointStore checkpointStore; readonly ReplicationSyncManager replicationSyncManager; readonly CancellationTokenSource ctsRepManager = new(); + CancellationTokenSource resetHandler = new(); readonly int pageSizeBits; @@ -454,6 +455,20 @@ public void EndRecovery(RecoveryStatus nextRecoveryStatus, bool downgradeLock) } } + public void ResetRecovery() + { + switch (currentRecoveryStatus) + { + case RecoveryStatus.ClusterReplicate: + case RecoveryStatus.ClusterFailover: + case RecoveryStatus.ReplicaOfNoOne: + case RecoveryStatus.CheckpointRecoveredAtReplica: + case RecoveryStatus.InitializeRecover: + resetHandler.Cancel(); + break; + } + } + public void Dispose() { _disposed = true; @@ -470,6 +485,8 @@ public void Dispose() replicaReplayTaskCts.Dispose(); ctsRepManager.Cancel(); ctsRepManager.Dispose(); + resetHandler.Cancel(); + resetHandler.Dispose(); aofTaskStore.Dispose(); aofProcessor?.Dispose(); networkPool?.Dispose(); diff --git a/libs/common/ExceptionInjectionType.cs b/libs/common/ExceptionInjectionType.cs index dd122b0cbfd..8dc62305fd3 100644 --- a/libs/common/ExceptionInjectionType.cs +++ b/libs/common/ExceptionInjectionType.cs @@ -53,5 +53,13 @@ public enum ExceptionInjectionType /// Delay response on receive checkpoint to trigger timeout /// Replication_Timeout_On_Receive_Checkpoint, + /// + /// Replication InProgress during disk-based replica attach sync operation + /// + Replication_InProgress_During_DiskBased_Replica_Attach_Sync, + /// + /// Replication InProgress during diskless replica attach sync operation + /// + Replication_InProgress_During_Diskless_Replica_Attach_Sync, } } \ No newline at end of file diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterResetDuringReplicationTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterResetDuringReplicationTests.cs new file mode 100644 index 00000000000..cb18e0ae80c --- /dev/null +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterResetDuringReplicationTests.cs @@ -0,0 +1,154 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +#if DEBUG +using System.Collections.Generic; +using System.Threading; +using System.Threading.Tasks; +using Garnet.common; +using Microsoft.Extensions.Logging; +using NUnit.Framework; +using NUnit.Framework.Legacy; + +namespace Garnet.test.cluster.ReplicationTests +{ + /// + /// These tests simulate scenarios where a replica gets stuck or is in replication attach and verify that + /// CLUSTER RESET HARD can properly cancel ongoing operations and allow the replica to be reused. + /// + [NonParallelizable] + public class ClusterResetDuringReplicationTests + { + ClusterTestContext context; + + readonly int createInstanceTimeout = (int)System.TimeSpan.FromSeconds(30).TotalSeconds; + const int testTimeout = 120_000; + + readonly Dictionary monitorTests = []; + + [SetUp] + public void Setup() + { + context = new ClusterTestContext(); + context.Setup(monitorTests, testTimeoutSeconds: testTimeout); + } + + [TearDown] + public void TearDown() + { + context?.TearDown(); + } + + /// + /// Test CLUSTER RESET HARD functionality during diskless replication attach. + /// This test simulates a scenario where a replica gets stuck while attaching to a primary + /// and verifies that CLUSTER RESET HARD can properly cancel the operation and reset the node. + /// + [Test, Order(1), CancelAfter(testTimeout)] + [Category("REPLICATION")] + public async Task ClusterResetHardDuringDisklessReplicationAttach(CancellationToken cancellationToken) + { + var primaryIndex = 0; + var replicaIndex = 1; + var nodes_count = 2; + + // Create instances with diskless sync enabled + context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, enableDisklessSync: true, timeout: createInstanceTimeout); + context.CreateConnection(); + + // Setup primary + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaIndex, replicaIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaIndex, logger: context.logger); + + // Ensure nodes know each other + context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); + + try + { + ExceptionInjectionHelper.EnableException(ExceptionInjectionType.Replication_InProgress_During_Diskless_Replica_Attach_Sync); + + var resp = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, failEx: false, async: true, logger: context.logger); + + await Task.Delay(1000, cancellationToken); + + // Verify that the replica is in a replicating state + var replicationInfo = context.clusterTestUtils.GetReplicationInfo(replicaIndex, [ReplicationInfoItem.RECOVER_STATUS], logger: context.logger); + ClassicAssert.AreEqual("ClusterReplicate", replicationInfo[0].Item2); + + // Issuing CLUSTER RESET HARD while replication is ongoing/stuck. + var resetResp = context.clusterTestUtils.ClusterReset(replicaIndex, soft: false, expiry: 60, logger: context.logger); + ClassicAssert.AreEqual("OK", resetResp); + + // Verify that the node is no longer in recovery state + replicationInfo = context.clusterTestUtils.GetReplicationInfo(replicaIndex, [ReplicationInfoItem.RECOVER_STATUS], logger: context.logger); + ClassicAssert.AreEqual("NoRecovery", replicationInfo[0].Item2); + + // Verify the node role is back to PRIMARY (default after reset) + var role = context.clusterTestUtils.RoleCommand(replicaIndex, logger: context.logger); + ClassicAssert.AreEqual("master", role.Value); + } + finally + { + ExceptionInjectionHelper.DisableException(ExceptionInjectionType.Replication_InProgress_During_Diskless_Replica_Attach_Sync); + } + } + + /// + /// Test CLUSTER RESET HARD functionality during diskbased replication attach. + /// This test simulates a scenario where a replica gets stuck while attaching to a primary + /// and verifies that CLUSTER RESET HARD can properly cancel the operation and reset the node. + /// + [Test, Order(2), CancelAfter(testTimeout)] + [Category("REPLICATION")] + public async Task ClusterResetHardDuringDiskBasedReplicationAttach(CancellationToken cancellationToken) + { + var primaryIndex = 0; + var replicaIndex = 1; + var nodes_count = 2; + + // (diskless sync is false) + context.CreateInstances(nodes_count, disableObjects: false, enableAOF: true, enableDisklessSync: false, timeout: createInstanceTimeout); + context.CreateConnection(); + + // Setup primary + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaIndex, replicaIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaIndex, logger: context.logger); + + context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); + + try + { + ExceptionInjectionHelper.EnableException(ExceptionInjectionType.Replication_InProgress_During_DiskBased_Replica_Attach_Sync); + + var resp = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, failEx: false, async: true, logger: context.logger); + + await Task.Delay(1000, cancellationToken); + + // Verify that the replica is in a replicating state + var replicationInfo = context.clusterTestUtils.GetReplicationInfo(replicaIndex, [ReplicationInfoItem.RECOVER_STATUS], logger: context.logger); + ClassicAssert.AreEqual("ClusterReplicate", replicationInfo[0].Item2); + + // Issueing CLUSTER RESET HARD while replication is ongoing/stuck. + var resetResp = context.clusterTestUtils.ClusterReset(replicaIndex, soft: false, expiry: 60, logger: context.logger); + ClassicAssert.AreEqual("OK", resetResp); + + // Verify that the node is no longer in recovery state + replicationInfo = context.clusterTestUtils.GetReplicationInfo(replicaIndex, [ReplicationInfoItem.RECOVER_STATUS], logger: context.logger); + ClassicAssert.AreEqual("NoRecovery", replicationInfo[0].Item2); + + // Verify the node role is back to PRIMARY (default after reset) + var role = context.clusterTestUtils.RoleCommand(replicaIndex, logger: context.logger); + ClassicAssert.AreEqual("master", role.Value); + } + finally + { + ExceptionInjectionHelper.DisableException(ExceptionInjectionType.Replication_InProgress_During_DiskBased_Replica_Attach_Sync); + } + } + } +} +#endif \ No newline at end of file From 3773b269af931d11f74ef421705a317ed9e4bbbb Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Wed, 24 Sep 2025 15:08:58 -0700 Subject: [PATCH 20/28] Fix transaction key locking --- .../BDN.benchmark/Custom/CustomTxnSet.cs | 8 +- .../Custom/CustomTransactionProcedure.cs | 6 +- libs/server/Lua/LuaRunner.Functions.cs | 4 +- libs/server/Lua/LuaRunner.cs | 8 +- .../ItemBroker/CollectionItemBroker.cs | 4 +- .../Storage/Session/MainStore/BitmapOps.cs | 4 +- .../Session/MainStore/HyperLogLogOps.cs | 8 +- .../Storage/Session/MainStore/MainStoreOps.cs | 19 ++- .../Storage/Session/ObjectStore/ListOps.cs | 4 +- .../Storage/Session/ObjectStore/SetOps.cs | 24 ++-- .../Session/ObjectStore/SortedSetGeoOps.cs | 6 +- .../Session/ObjectStore/SortedSetOps.cs | 24 ++-- libs/server/Transaction/TransactionManager.cs | 114 ++++++++++-------- libs/server/Transaction/TxnKeyEntry.cs | 73 ++++++++--- .../Transaction/TxnKeyEntryComparison.cs | 21 ++-- libs/server/Transaction/TxnKeyManager.cs | 13 +- .../Transaction/TxnWatchedKeysContainer.cs | 5 +- main/GarnetServer/Extensions/ReadWriteTxn.cs | 4 +- .../Extensions/SampleDeleteTxn.cs | 6 +- .../Extensions/SampleUpdateTxn.cs | 6 +- .../CustomProcs/ClusterDelRmw.cs | 2 +- .../RedirectTests/TestClusterProc.cs | 12 +- .../Garnet.test/AofFinalizeDoubleReplayTxn.cs | 2 +- test/Garnet.test/DeleteTxn.cs | 2 +- test/Garnet.test/Extensions/RateLimiterTxn.cs | 2 +- .../Extensions/SortedSetCountTxn.cs | 2 +- test/Garnet.test/Extensions/TxnCustomCmd.cs | 4 +- test/Garnet.test/ObjectExpiryTxn.cs | 2 +- test/Garnet.test/RespCustomCommandTests.cs | 6 +- test/Garnet.test/SortedSetRemoveTxn.cs | 2 +- test/Garnet.test/TestProcedureBitmap.cs | 6 +- test/Garnet.test/TestProcedureHLL.cs | 2 +- test/Garnet.test/TestProcedureHash.cs | 2 +- test/Garnet.test/TestProcedureLists.cs | 6 +- test/Garnet.test/TestProcedureSet.cs | 2 +- test/Garnet.test/TestProcedureSortedSets.cs | 2 +- test/Garnet.test/WriteWithExpiryTxn.cs | 2 +- 37 files changed, 235 insertions(+), 184 deletions(-) diff --git a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs index de849013dab..835e966ba6c 100644 --- a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs +++ b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs @@ -52,10 +52,10 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce valueC = GetNextArg(ref procInput, ref offset); valueD = GetNextArg(ref procInput, ref offset); - AddKey(setA, LockType.Exclusive, isObject: false); - AddKey(setB, LockType.Exclusive, isObject: false); - AddKey(setC, LockType.Exclusive, isObject: false); - AddKey(setD, LockType.Exclusive, isObject: false); + AddKey(setA, LockType.Exclusive, StoreType.Main); + AddKey(setB, LockType.Exclusive, StoreType.Main); + AddKey(setC, LockType.Exclusive, StoreType.Main); + AddKey(setD, LockType.Exclusive, StoreType.Main); return true; } diff --git a/libs/server/Custom/CustomTransactionProcedure.cs b/libs/server/Custom/CustomTransactionProcedure.cs index 3b2d2372f08..f0eb426a9fe 100644 --- a/libs/server/Custom/CustomTransactionProcedure.cs +++ b/libs/server/Custom/CustomTransactionProcedure.cs @@ -36,10 +36,10 @@ public abstract class CustomTransactionProcedure : CustomProcedureBase /// /// /// - /// - protected void AddKey(PinnedSpanByte key, LockType type, bool isObject) + /// + protected void AddKey(PinnedSpanByte key, LockType type, StoreType storeType) { - txnManager.SaveKeyEntryToLock(key, isObject, type); + txnManager.SaveKeyEntryToLock(key, storeType, type); txnManager.VerifyKeyOwnership(key, type); } diff --git a/libs/server/Lua/LuaRunner.Functions.cs b/libs/server/Lua/LuaRunner.Functions.cs index 03d9717113c..8fdf814aea6 100644 --- a/libs/server/Lua/LuaRunner.Functions.cs +++ b/libs/server/Lua/LuaRunner.Functions.cs @@ -227,9 +227,7 @@ internal int UnsafeRunPreambleForSession(nint luaStatePtr) if (txnMode) { - txnKeyEntries.AddKey(key, false, Tsavorite.core.LockType.Exclusive); - if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) - txnKeyEntries.AddKey(key, true, Tsavorite.core.LockType.Exclusive); + txnKeyEntries.AddKey(key, StoreType.All, LockType.Exclusive); } // Equivalent to KEYS[i+1] = key diff --git a/libs/server/Lua/LuaRunner.cs b/libs/server/Lua/LuaRunner.cs index 310f2ba6fe0..c84c83cd73e 100644 --- a/libs/server/Lua/LuaRunner.cs +++ b/libs/server/Lua/LuaRunner.cs @@ -263,7 +263,9 @@ public unsafe LuaRunner( delegate* unmanaged[Cdecl] garnetCall; if (txnMode) { - txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.transactionalContext, respServerSession.storageSession.objectStoreTransactionalContext); + txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.transactionalContext, + respServerSession.storageSession.objectStoreTransactionalContext, + respServerSession.storageSession.unifiedStoreTransactionalContext); garnetCall = &LuaRunnerTrampolines.GarnetCallWithTransaction; } @@ -1237,9 +1239,7 @@ public unsafe object RunForRunner(string[] keys = null, string[] argv = null) foreach (var key in keys) { var _key = scratchBufferBuilder.CreateArgSlice(key); - txnKeyEntries.AddKey(_key, false, Tsavorite.core.LockType.Exclusive); - if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) - txnKeyEntries.AddKey(_key, true, Tsavorite.core.LockType.Exclusive); + txnKeyEntries.AddKey(_key, StoreType.All, Tsavorite.core.LockType.Exclusive); } adapter = new(scratchBufferBuilder); diff --git a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs index b7fbae5a2f5..ba93e605b4e 100644 --- a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs +++ b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs @@ -529,10 +529,10 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp { Debug.Assert(storageSession.txnManager.state == TxnState.None); createTransaction = true; - storageSession.txnManager.SaveKeyEntryToLock(asKey, true, LockType.Exclusive); + storageSession.txnManager.SaveKeyEntryToLock(asKey, StoreType.Object, LockType.Exclusive); if (command == RespCommand.BLMOVE) - storageSession.txnManager.SaveKeyEntryToLock(dstKey, true, LockType.Exclusive); + storageSession.txnManager.SaveKeyEntryToLock(dstKey, StoreType.Object, LockType.Exclusive); _ = storageSession.txnManager.Run(true); } diff --git a/libs/server/Storage/Session/MainStore/BitmapOps.cs b/libs/server/Storage/Session/MainStore/BitmapOps.cs index 23d73fec7ad..286cad20a8e 100644 --- a/libs/server/Storage/Session/MainStore/BitmapOps.cs +++ b/libs/server/Storage/Session/MainStore/BitmapOps.cs @@ -88,9 +88,9 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp { createTransaction = true; Debug.Assert(txnManager.state == TxnState.None); - txnManager.SaveKeyEntryToLock(keys[0], false, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(keys[0], StoreType.Main, LockType.Exclusive); for (var i = 1; i < keys.Length; i++) - txnManager.SaveKeyEntryToLock(keys[i], false, LockType.Shared); + txnManager.SaveKeyEntryToLock(keys[i], StoreType.Main, LockType.Shared); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs index dedce5df223..3c195df29cb 100644 --- a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs +++ b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs @@ -102,11 +102,11 @@ public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; var dstKey = input.parseState.GetArgSliceByRef(0); - txnManager.SaveKeyEntryToLock(dstKey, false, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(dstKey, StoreType.Main, LockType.Exclusive); for (var i = 1; i < input.parseState.Count; i++) { var currSrcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(currSrcKey, false, LockType.Shared); + txnManager.SaveKeyEntryToLock(currSrcKey, StoreType.Main, LockType.Shared); } _ = txnManager.Run(true); } @@ -195,11 +195,11 @@ public unsafe GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool e Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; var dstKey = input.parseState.GetArgSliceByRef(0); - txnManager.SaveKeyEntryToLock(dstKey, false, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(dstKey, StoreType.Main, LockType.Exclusive); for (var i = 1; i < input.parseState.Count; i++) { var currSrcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(currSrcKey, false, LockType.Shared); + txnManager.SaveKeyEntryToLock(currSrcKey, StoreType.Main, LockType.Shared); } _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 27c7dac1cba..763e0403468 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -370,21 +370,20 @@ internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref T for (var i = 0; i < count; i += 2) { var srcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(srcKey, false, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(srcKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(srcKey, StoreType.All, LockType.Exclusive); } txnManager.Run(true); } var context = txnManager.TransactionalContext; - var objContext = txnManager.ObjectStoreTransactionalContext; + var unifiedContext = txnManager.UnifiedStoreTransactionalContext; try { for (var i = 0; i < count; i += 2) { var srcKey = input.parseState.GetArgSliceByRef(i); - var status = EXISTS(srcKey, StoreType.All, ref context, ref objContext); + var status = EXISTS(srcKey, ref unifiedContext); if (status != GarnetStatus.NOTFOUND) { count = 0; @@ -538,8 +537,8 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(oldKeySlice, false, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(newKeySlice, false, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(oldKeySlice, StoreType.Main, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(newKeySlice, StoreType.Main, LockType.Exclusive); _ = txnManager.Run(true); } @@ -662,8 +661,8 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne createTransaction = false; if (txnManager.state != TxnState.Running) { - txnManager.SaveKeyEntryToLock(oldKeySlice, true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(newKeySlice, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(oldKeySlice, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(newKeySlice, StoreType.Object, LockType.Exclusive); txnManager.Run(true); createTransaction = true; } @@ -1022,8 +1021,8 @@ public unsafe GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref Spa var createTransaction = false; if (txnManager.state != TxnState.Running) { - txnManager.SaveKeyEntryToLock(key1, false, LockType.Shared); - txnManager.SaveKeyEntryToLock(key2, false, LockType.Shared); + txnManager.SaveKeyEntryToLock(key1, StoreType.Main, LockType.Shared); + txnManager.SaveKeyEntryToLock(key2, StoreType.Main, LockType.Shared); txnManager.Run(true); createTransaction = true; } diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index 76b247c1921..98257cae3d3 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -218,8 +218,8 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(sourceKey, true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(sourceKey, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index ae06623ad39..05b8ded1843 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -267,8 +267,8 @@ internal unsafe GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte de if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(sourceKey, true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(sourceKey, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); _ = txnManager.Run(true); } @@ -362,7 +362,7 @@ public GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet outp Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -403,9 +403,9 @@ public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -533,7 +533,7 @@ public GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output) Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -572,9 +572,9 @@ public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -793,7 +793,7 @@ public GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet members) Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -832,9 +832,9 @@ public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -951,7 +951,7 @@ public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? l Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index 2c2da0f5797..f631e1816e7 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs @@ -69,7 +69,7 @@ public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref Ge { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Shared); txnManager.Run(true); } @@ -126,8 +126,8 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destination, true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(key, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(destination, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index 5558d730bf6..bd3fec1d58a 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -531,7 +531,7 @@ public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); txnManager.Run(true); } @@ -578,9 +578,9 @@ public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, Read { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -712,8 +712,8 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(dstKey, isObject: true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(srcKey, isObject: true, LockType.Shared); + txnManager.SaveKeyEntryToLock(dstKey, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(srcKey, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -1037,7 +1037,7 @@ public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] w Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); txnManager.Run(true); } @@ -1078,9 +1078,9 @@ public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlyS { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -1269,7 +1269,7 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var key in keys) - txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); txnManager.Run(true); } @@ -1334,9 +1334,9 @@ public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadO { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } @@ -1396,7 +1396,7 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); txnManager.Run(true); } diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 397951dcb43..d6ad748ce20 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -56,6 +56,16 @@ public sealed unsafe partial class TransactionManager /// readonly TransactionalContext objectStoreTransactionalContext; + /// + /// Basic context for unified store + /// + readonly BasicContext unifiedStoreBasicContext; + + /// + /// Transactional context for unified store + /// + readonly TransactionalContext unifiedStoreTransactionalContext; + // Not readonly to avoid defensive copy GarnetWatchApi garnetTxPrepareApi; @@ -80,9 +90,11 @@ public sealed unsafe partial class TransactionManager public TxnState state; private const int initialSliceBufferSize = 1 << 10; private const int initialKeyBufferSize = 1 << 10; - StoreType transactionStoreType; readonly ILogger logger; long txnVersion; + private bool isMainTxn = false; + private bool isObjectTxn = false; + private bool isUnifiedTxn = false; internal TransactionalContext TransactionalContext => transactionalContext; @@ -90,6 +102,8 @@ internal TransactionalUnsafeContext basicContext.Session.TransactionalUnsafeContext; internal TransactionalContext ObjectStoreTransactionalContext => objectStoreTransactionalContext; + internal TransactionalContext UnifiedStoreTransactionalContext + => unifiedStoreTransactionalContext; /// /// Array to keep pointer keys in keyBuffer @@ -111,11 +125,12 @@ internal TransactionManager(StoreWrapper storeWrapper, transactionalContext = session.TransactionalContext; var objectStoreSession = storageSession.objectStoreBasicContext.Session; - if (objectStoreSession != null) - { - objectStoreBasicContext = objectStoreSession.BasicContext; - objectStoreTransactionalContext = objectStoreSession.TransactionalContext; - } + objectStoreBasicContext = objectStoreSession.BasicContext; + objectStoreTransactionalContext = objectStoreSession.TransactionalContext; + + var unifiedStoreSession = storageSession.unifiedStoreBasicContext.Session; + unifiedStoreBasicContext = unifiedStoreSession.BasicContext; + unifiedStoreTransactionalContext = unifiedStoreSession.TransactionalContext; this.functionsState = storageSession.functionsState; this.appendOnlyFile = functionsState.appendOnlyFile; @@ -124,7 +139,7 @@ internal TransactionManager(StoreWrapper storeWrapper, this.respSession = respSession; watchContainer = new WatchedKeysContainer(initialSliceBufferSize, functionsState.watchVersionMap); - keyEntries = new TxnKeyEntries(initialSliceBufferSize, transactionalContext, objectStoreTransactionalContext); + keyEntries = new TxnKeyEntries(initialSliceBufferSize, transactionalContext, objectStoreTransactionalContext, unifiedStoreTransactionalContext); this.scratchBufferAllocator = scratchBufferAllocator; var dbFound = storeWrapper.TryGetDatabase(dbId, out var db); @@ -151,14 +166,12 @@ internal void Reset(bool isRunning) keyEntries.UnlockAllKeys(); // Release context - if (transactionStoreType == StoreType.Main || transactionStoreType == StoreType.All) + if (isMainTxn) transactionalContext.EndTransaction(); - if (transactionStoreType == StoreType.Object || transactionStoreType == StoreType.All) - { - if (objectStoreBasicContext.IsNull) - throw new Exception("Trying to perform object store transaction with object store disabled"); + if (isObjectTxn) objectStoreTransactionalContext.EndTransaction(); - } + if (isUnifiedTxn) + unifiedStoreTransactionalContext.EndTransaction(); } finally { @@ -169,7 +182,9 @@ internal void Reset(bool isRunning) this.txnStartHead = 0; this.operationCntTxn = 0; this.state = TxnState.None; - this.transactionStoreType = 0; + this.isMainTxn = false; + this.isObjectTxn = false; + this.isUnifiedTxn = false; functionsState.StoredProcMode = false; // Reset cluster variables used for slot verification @@ -282,30 +297,37 @@ internal void Commit(bool internal_txn = false) internal void Watch(PinnedSpanByte key, StoreType type) { - // Update watch type if object store is disabled - if (type == StoreType.All && objectStoreBasicContext.IsNull) - type = StoreType.Main; - UpdateTransactionStoreType(type); watchContainer.AddWatch(key, type); - if (type == StoreType.Main || type == StoreType.All) - basicContext.ResetModified(key.ReadOnlySpan); - if ((type == StoreType.Object || type == StoreType.All) && !objectStoreBasicContext.IsNull) - objectStoreBasicContext.ResetModified(key.ReadOnlySpan); + // Release context + switch (type) + { + case StoreType.Main: + basicContext.ResetModified(key.ReadOnlySpan); + break; + case StoreType.Object: + objectStoreBasicContext.ResetModified(key.ReadOnlySpan); + break; + case StoreType.All: + unifiedStoreBasicContext.ResetModified(key.ReadOnlySpan); + break; + } } void UpdateTransactionStoreType(StoreType type) { - if (transactionStoreType != StoreType.All) + switch (type) { - if (transactionStoreType == 0) - transactionStoreType = type; - else - { - if (transactionStoreType != type) - transactionStoreType = StoreType.All; - } + case StoreType.Main: + isMainTxn = true; + break; + case StoreType.Object: + isObjectTxn = true; + break; + case StoreType.All: + isUnifiedTxn = true; + break; } } @@ -320,32 +342,24 @@ internal void GetKeysForValidation(byte* recvBufferPtr, out PinnedSpanByte[] key readOnly = keyEntries.IsReadOnly; } - void BeginTransaction(StoreType transactionStoreType) + void BeginTransaction() { - if (transactionStoreType is StoreType.All or StoreType.Main) - { + if (isMainTxn) transactionalContext.BeginTransaction(); - } - if (transactionStoreType is StoreType.All or StoreType.Object) - { - if (objectStoreBasicContext.IsNull) - throw new Exception("Trying to perform object store transaction with object store disabled"); + if (isObjectTxn) objectStoreTransactionalContext.BeginTransaction(); - } + if (isUnifiedTxn) + unifiedStoreTransactionalContext.BeginTransaction(); } - void LocksAcquired(StoreType transactionStoreType, long txnVersion) + void LocksAcquired(long txnVersion) { - if (transactionStoreType is StoreType.All or StoreType.Main) - { + if (isMainTxn) transactionalContext.LocksAcquired(txnVersion); - } - if (transactionStoreType is StoreType.All or StoreType.Object) - { - if (objectStoreBasicContext.IsNull) - throw new Exception("Trying to perform object store transaction with object store disabled"); + if (isObjectTxn) objectStoreTransactionalContext.LocksAcquired(txnVersion); - } + if (isUnifiedTxn) + unifiedStoreTransactionalContext.LocksAcquired(txnVersion); } internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, TimeSpan lock_timeout = default) @@ -358,7 +372,7 @@ internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, Tim txnVersion = stateMachineDriver.AcquireTransactionVersion(); // Acquire lock sessions - BeginTransaction(transactionStoreType); + BeginTransaction(); bool lockSuccess; if (fail_fast_on_lock) @@ -388,7 +402,7 @@ internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, Tim txnVersion = stateMachineDriver.VerifyTransactionVersion(txnVersion); // Update sessions with transaction version - LocksAcquired(transactionStoreType, txnVersion); + LocksAcquired(txnVersion); if (appendOnlyFile != null && !functionsState.StoredProcMode) { diff --git a/libs/server/Transaction/TxnKeyEntry.cs b/libs/server/Transaction/TxnKeyEntry.cs index ca384a8b97a..5311ebbeaa3 100644 --- a/libs/server/Transaction/TxnKeyEntry.cs +++ b/libs/server/Transaction/TxnKeyEntry.cs @@ -21,7 +21,7 @@ struct TxnKeyEntry : ITransactionalKey internal long keyHash; [FieldOffset(8)] - internal bool isObject; + internal StoreType storeType; [FieldOffset(9)] internal LockType lockType; @@ -40,7 +40,14 @@ public override readonly string ToString() // The debugger often can't call the Globalization NegativeSign property so ToString() would just display the class name var keyHashSign = keyHash < 0 ? "-" : string.Empty; var absKeyHash = keyHash >= 0 ? keyHash : -keyHash; - return $"{keyHashSign}{absKeyHash}:{(isObject ? "obj" : "raw")}:{(lockType == LockType.None ? "-" : (lockType == LockType.Shared ? "s" : "x"))}"; + var storeTypeStr = storeType switch + { + StoreType.Object => "obj", + StoreType.Main => "raw", + StoreType.All => "uni", + _ => "unk" + }; + return $"{keyHashSign}{absKeyHash}:{storeTypeStr}:{(lockType == LockType.None ? "-" : (lockType == LockType.Shared ? "s" : "x"))}"; } } @@ -49,21 +56,24 @@ internal sealed class TxnKeyEntries // Basic keys int keyCount; int mainKeyCount; + int objectKeyCount; TxnKeyEntry[] keys; bool mainStoreKeyLocked; bool objectStoreKeyLocked; + bool unifiedStoreKeyLocked; readonly TxnKeyComparison comparison; public int phase; internal TxnKeyEntries(int initialCount, TransactionalContext transactionalContext, - TransactionalContext objectStoreTransactionalContext) + TransactionalContext objectStoreTransactionalContext, + TransactionalContext unifiedStoreTransactionalContext) { keys = GC.AllocateArray(initialCount, pinned: true); // We sort a single array for speed, and the sessions use the same sorting logic, - comparison = new(transactionalContext, objectStoreTransactionalContext); + comparison = new(transactionalContext, objectStoreTransactionalContext, unifiedStoreTransactionalContext); } public bool IsReadOnly @@ -71,7 +81,7 @@ public bool IsReadOnly get { var readOnly = true; - for (int i = 0; i < keyCount; i++) + for (var i = 0; i < keyCount; i++) { if (keys[i].lockType == LockType.Exclusive) { @@ -83,11 +93,15 @@ public bool IsReadOnly } } - public void AddKey(PinnedSpanByte keyArgSlice, bool isObject, LockType type) + public void AddKey(PinnedSpanByte keyArgSlice, StoreType storeType, LockType type) { - var keyHash = !isObject - ? comparison.transactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan) - : comparison.objectStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan); + var keyHash = storeType switch + { + StoreType.Main => comparison.transactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), + StoreType.Object => comparison.objectStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), + StoreType.All => comparison.unifiedStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), + _ => throw new ArgumentOutOfRangeException(nameof(storeType), "Invalid store type") + }; // Grow the buffer if needed if (keyCount >= keys.Length) @@ -99,11 +113,13 @@ public void AddKey(PinnedSpanByte keyArgSlice, bool isObject, LockType type) // Populate the new key slot. keys[keyCount].keyHash = keyHash; - keys[keyCount].isObject = isObject; + keys[keyCount].storeType = storeType; keys[keyCount].lockType = type; ++keyCount; - if (!isObject) + if (storeType == StoreType.Main) ++mainKeyCount; + else if (storeType == StoreType.Object) + ++objectKeyCount; } internal void LockAllKeys() @@ -124,12 +140,19 @@ internal void LockAllKeys() } // Issue object store locks - if (mainKeyCount < keyCount) + if (objectKeyCount > 0) { - comparison.objectStoreTransactionalContext.Lock(keys.AsSpan().Slice(mainKeyCount, keyCount - mainKeyCount)); + comparison.objectStoreTransactionalContext.Lock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount)); objectStoreKeyLocked = true; } + // Issue unified store locks + if (mainKeyCount + objectKeyCount < keyCount) + { + comparison.unifiedStoreTransactionalContext.Lock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount)); + unifiedStoreKeyLocked = true; + } + phase = 0; } @@ -157,9 +180,9 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) // Issue object store locks // TryLock will unlock automatically in case of partial failure - if (mainKeyCount < keyCount) + if (objectKeyCount > 0) { - objectStoreKeyLocked = comparison.objectStoreTransactionalContext.TryLock(keys.AsSpan().Slice(mainKeyCount, keyCount - mainKeyCount), lock_timeout); + objectStoreKeyLocked = comparison.objectStoreTransactionalContext.TryLock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount), lock_timeout); if (!objectStoreKeyLocked) { phase = 0; @@ -167,6 +190,18 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) } } + // Issue unified store locks + // TryLock will unlock automatically in case of partial failure + if (mainKeyCount + objectKeyCount < keyCount) + { + unifiedStoreKeyLocked = comparison.unifiedStoreTransactionalContext.TryLock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount), lock_timeout); + if (!unifiedStoreKeyLocked) + { + phase = 0; + return false; + } + } + phase = 0; return true; } @@ -176,12 +211,16 @@ internal void UnlockAllKeys() phase = 2; if (mainStoreKeyLocked && mainKeyCount > 0) comparison.transactionalContext.Unlock(keys.AsSpan()[..mainKeyCount]); - if (objectStoreKeyLocked && mainKeyCount < keyCount) - comparison.objectStoreTransactionalContext.Unlock(keys.AsSpan().Slice(mainKeyCount, keyCount - mainKeyCount)); + if (objectStoreKeyLocked && objectKeyCount > 0) + comparison.objectStoreTransactionalContext.Unlock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount)); + if (unifiedStoreKeyLocked && mainKeyCount + objectKeyCount < keyCount) + comparison.unifiedStoreTransactionalContext.Unlock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount)); mainKeyCount = 0; + objectKeyCount = 0; keyCount = 0; mainStoreKeyLocked = false; objectStoreKeyLocked = false; + unifiedStoreKeyLocked = false; phase = 0; } diff --git a/libs/server/Transaction/TxnKeyEntryComparison.cs b/libs/server/Transaction/TxnKeyEntryComparison.cs index cd83979d773..12c8824eef9 100644 --- a/libs/server/Transaction/TxnKeyEntryComparison.cs +++ b/libs/server/Transaction/TxnKeyEntryComparison.cs @@ -13,28 +13,35 @@ internal sealed class TxnKeyComparison { public TransactionalContext transactionalContext; public TransactionalContext objectStoreTransactionalContext; + public TransactionalContext unifiedStoreTransactionalContext; public readonly Comparison comparisonDelegate; internal TxnKeyComparison(TransactionalContext transactionalContext, - TransactionalContext objectStoreTransactionalContext) + TransactionalContext objectStoreTransactionalContext, + TransactionalContext unifiedStoreTransactionalContext) { this.transactionalContext = transactionalContext; this.objectStoreTransactionalContext = objectStoreTransactionalContext; + this.unifiedStoreTransactionalContext = unifiedStoreTransactionalContext; comparisonDelegate = Compare; } /// public int Compare(TxnKeyEntry key1, TxnKeyEntry key2) { - // This sorts by isObject, then calls Tsavorite to sort by lock code and then by lockType. - var cmp = key1.isObject.CompareTo(key2.isObject); + // This sorts by storeType, then calls Tsavorite to sort by lock code and then by lockType. + var cmp = key1.storeType.CompareTo(key2.storeType); if (cmp != 0) return cmp; - if (key1.isObject) - return objectStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2); - else - return transactionalContext.CompareKeyHashes(ref key1, ref key2); + + return key1.storeType switch + { + StoreType.Main => transactionalContext.CompareKeyHashes(ref key1, ref key2), + StoreType.Object => objectStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2), + StoreType.All => unifiedStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2), + _ => throw new InvalidOperationException($"Unknown store type"), + }; } } } \ No newline at end of file diff --git a/libs/server/Transaction/TxnKeyManager.cs b/libs/server/Transaction/TxnKeyManager.cs index b87413035a2..7f0f4318853 100644 --- a/libs/server/Transaction/TxnKeyManager.cs +++ b/libs/server/Transaction/TxnKeyManager.cs @@ -12,12 +12,12 @@ sealed partial class TransactionManager /// Save key entry /// /// - /// + /// /// - public void SaveKeyEntryToLock(PinnedSpanByte key, bool isObject, LockType type) + public void SaveKeyEntryToLock(PinnedSpanByte key, StoreType storeType, LockType type) { - UpdateTransactionStoreType(isObject ? StoreType.Object : StoreType.Main); - keyEntries.AddKey(key, isObject, type); + UpdateTransactionStoreType(storeType); + keyEntries.AddKey(key, storeType, type); } /// @@ -74,10 +74,7 @@ internal void LockKeys(SimpleRespCommandInfo cmdInfo) for (var currIdx = searchArgs.firstIdx; currIdx <= searchArgs.lastIdx; currIdx += searchArgs.step) { var key = respSession.parseState.GetArgSliceByRef(currIdx); - if (cmdInfo.StoreType is StoreType.Main or StoreType.All) - SaveKeyEntryToLock(key, false, lockType); - if (cmdInfo.StoreType is StoreType.Object or StoreType.All && !objectStoreBasicContext.IsNull) - SaveKeyEntryToLock(key, true, lockType); + SaveKeyEntryToLock(key, cmdInfo.StoreType, lockType); SaveKeyArgSlice(key); } } diff --git a/libs/server/Transaction/TxnWatchedKeysContainer.cs b/libs/server/Transaction/TxnWatchedKeysContainer.cs index 0887e3a9339..8edee6030a3 100644 --- a/libs/server/Transaction/TxnWatchedKeysContainer.cs +++ b/libs/server/Transaction/TxnWatchedKeysContainer.cs @@ -129,10 +129,7 @@ public bool SaveKeysToLock(TransactionManager txnManager) if (watchedKeySlice.type == 0) continue; var slice = keySlices[i].slice; - if (watchedKeySlice.type == StoreType.Main || watchedKeySlice.type == StoreType.All) - txnManager.SaveKeyEntryToLock(slice, false, LockType.Shared); - if (watchedKeySlice.type == StoreType.Object || watchedKeySlice.type == StoreType.All) - txnManager.SaveKeyEntryToLock(slice, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(slice, watchedKeySlice.type, LockType.Shared); } return true; } diff --git a/main/GarnetServer/Extensions/ReadWriteTxn.cs b/main/GarnetServer/Extensions/ReadWriteTxn.cs index 16778b83735..445a9ce4412 100644 --- a/main/GarnetServer/Extensions/ReadWriteTxn.cs +++ b/main/GarnetServer/Extensions/ReadWriteTxn.cs @@ -24,8 +24,8 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce api.GET(GetNextArg(ref procInput, ref offset), out PinnedSpanByte key1); if (key1.ReadOnlySpan.SequenceEqual("wrong_string"u8)) return false; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); return true; } diff --git a/main/GarnetServer/Extensions/SampleDeleteTxn.cs b/main/GarnetServer/Extensions/SampleDeleteTxn.cs index 5e1c47998d3..adffaad40fa 100644 --- a/main/GarnetServer/Extensions/SampleDeleteTxn.cs +++ b/main/GarnetServer/Extensions/SampleDeleteTxn.cs @@ -32,12 +32,12 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var offset = 0; var mainStoreKey = GetNextArg(ref procInput, ref offset); - AddKey(mainStoreKey, LockType.Exclusive, false); + AddKey(mainStoreKey, LockType.Exclusive, StoreType.Main); var sortedSet1Key = GetNextArg(ref procInput, ref offset); if (sortedSet1Key.Length > 0) { - AddKey(sortedSet1Key, LockType.Exclusive, true); + AddKey(sortedSet1Key, LockType.Exclusive, StoreType.Object); } GetNextArg(ref procInput, ref offset); // sortedSet1Entry @@ -45,7 +45,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var sortedSet2Key = GetNextArg(ref procInput, ref offset); if (sortedSet2Key.Length > 0) { - AddKey(sortedSet2Key, LockType.Exclusive, true); + AddKey(sortedSet2Key, LockType.Exclusive, StoreType.Object); } return true; diff --git a/main/GarnetServer/Extensions/SampleUpdateTxn.cs b/main/GarnetServer/Extensions/SampleUpdateTxn.cs index ac132ef507b..f7762836c9c 100644 --- a/main/GarnetServer/Extensions/SampleUpdateTxn.cs +++ b/main/GarnetServer/Extensions/SampleUpdateTxn.cs @@ -34,12 +34,12 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var mainStoreKey = GetNextArg(ref procInput, ref offset); GetNextArg(ref procInput, ref offset); // mainStoreValue - AddKey(mainStoreKey, LockType.Exclusive, false); + AddKey(mainStoreKey, LockType.Exclusive, StoreType.Main); var sortedSet1Key = GetNextArg(ref procInput, ref offset); if (sortedSet1Key.Length > 0) { - AddKey(sortedSet1Key, LockType.Exclusive, true); + AddKey(sortedSet1Key, LockType.Exclusive, StoreType.Object); } GetNextArg(ref procInput, ref offset); // sortedSet1Entry @@ -48,7 +48,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var sortedSet2Key = GetNextArg(ref procInput, ref offset); if (sortedSet2Key.Length > 0) { - AddKey(sortedSet2Key, LockType.Exclusive, true); + AddKey(sortedSet2Key, LockType.Exclusive, StoreType.Object); } return true; diff --git a/test/Garnet.test.cluster/CustomProcs/ClusterDelRmw.cs b/test/Garnet.test.cluster/CustomProcs/ClusterDelRmw.cs index 5c2d97d0e72..d782a57f6f6 100644 --- a/test/Garnet.test.cluster/CustomProcs/ClusterDelRmw.cs +++ b/test/Garnet.test.cluster/CustomProcs/ClusterDelRmw.cs @@ -14,7 +14,7 @@ sealed class ClusterDelRmw : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { var offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, isObject: false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); return true; } diff --git a/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs b/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs index 90a8c7bfd0e..3115b2e2252 100644 --- a/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs +++ b/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs @@ -30,9 +30,9 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var getB = GetNextArg(ref procInput, ref offset); var getC = GetNextArg(ref procInput, ref offset); - AddKey(getA, LockType.Shared, isObject: false); - AddKey(getB, LockType.Shared, isObject: false); - AddKey(getC, LockType.Shared, isObject: false); + AddKey(getA, LockType.Shared, StoreType.Main); + AddKey(getB, LockType.Shared, StoreType.Main); + AddKey(getC, LockType.Shared, StoreType.Main); return true; } @@ -103,9 +103,9 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var setB = GetNextArg(ref procInput, ref offset); var setC = GetNextArg(ref procInput, ref offset); - AddKey(getA, LockType.Shared, isObject: false); - AddKey(setB, LockType.Exclusive, isObject: false); - AddKey(setC, LockType.Exclusive, isObject: false); + AddKey(getA, LockType.Shared, StoreType.Main); + AddKey(setB, LockType.Exclusive, StoreType.Main); + AddKey(setC, LockType.Exclusive, StoreType.Main); return true; } diff --git a/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs b/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs index ce33ffa35c7..2c88df77f1d 100644 --- a/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs +++ b/test/Garnet.test/AofFinalizeDoubleReplayTxn.cs @@ -11,7 +11,7 @@ public class AofFinalizeDoubleReplayTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); return true; } public override void Main(TGarnetApi api, ref CustomProcedureInput procInput, ref MemoryResult output) diff --git a/test/Garnet.test/DeleteTxn.cs b/test/Garnet.test/DeleteTxn.cs index cc522e471e3..a77cb1216f6 100644 --- a/test/Garnet.test/DeleteTxn.cs +++ b/test/Garnet.test/DeleteTxn.cs @@ -19,7 +19,7 @@ sealed class DeleteTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { var offset = 0; - AddKey(GetNextArg(ref procInput.parseState, ref offset), LockType.Exclusive, false); + AddKey(GetNextArg(ref procInput.parseState, ref offset), LockType.Exclusive, StoreType.All); return true; } diff --git a/test/Garnet.test/Extensions/RateLimiterTxn.cs b/test/Garnet.test/Extensions/RateLimiterTxn.cs index f9b45ac9ee1..ce45bda1391 100644 --- a/test/Garnet.test/Extensions/RateLimiterTxn.cs +++ b/test/Garnet.test/Extensions/RateLimiterTxn.cs @@ -14,7 +14,7 @@ sealed class RateLimiterTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, true); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/Extensions/SortedSetCountTxn.cs b/test/Garnet.test/Extensions/SortedSetCountTxn.cs index 023a9653640..199d922b84e 100644 --- a/test/Garnet.test/Extensions/SortedSetCountTxn.cs +++ b/test/Garnet.test/Extensions/SortedSetCountTxn.cs @@ -12,7 +12,7 @@ sealed class SortedSetCountTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput input) { int offset = 0; - AddKey(GetNextArg(ref input, ref offset), LockType.Shared, true); + AddKey(GetNextArg(ref input, ref offset), LockType.Shared, StoreType.Object); return true; } diff --git a/test/Garnet.test/Extensions/TxnCustomCmd.cs b/test/Garnet.test/Extensions/TxnCustomCmd.cs index 48646f46f7d..ce0ad587348 100644 --- a/test/Garnet.test/Extensions/TxnCustomCmd.cs +++ b/test/Garnet.test/Extensions/TxnCustomCmd.cs @@ -18,10 +18,10 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var mainStoreKey = GetNextArg(ref procInput, ref offset); _ = GetNextArg(ref procInput, ref offset); // mainStoreValue - AddKey(mainStoreKey, LockType.Exclusive, false); + AddKey(mainStoreKey, LockType.Exclusive, StoreType.Main); var myDictKey = GetNextArg(ref procInput, ref offset); - AddKey(myDictKey, LockType.Exclusive, true); + AddKey(myDictKey, LockType.Exclusive, StoreType.Object); if (!ParseCustomObjectCommand("MYDICTSET", out customObjectCommand)) return false; diff --git a/test/Garnet.test/ObjectExpiryTxn.cs b/test/Garnet.test/ObjectExpiryTxn.cs index 090321f0964..9b7aed88d58 100644 --- a/test/Garnet.test/ObjectExpiryTxn.cs +++ b/test/Garnet.test/ObjectExpiryTxn.cs @@ -19,7 +19,7 @@ sealed class ObjectExpiryTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { var offset = 0; - AddKey(GetNextArg(ref procInput.parseState, ref offset), LockType.Exclusive, true); + AddKey(GetNextArg(ref procInput.parseState, ref offset), LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/RespCustomCommandTests.cs b/test/Garnet.test/RespCustomCommandTests.cs index a247f34ec17..c441de0859b 100644 --- a/test/Garnet.test/RespCustomCommandTests.cs +++ b/test/Garnet.test/RespCustomCommandTests.cs @@ -68,7 +68,7 @@ public class LargeGetTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Shared, false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Shared, StoreType.Main); return true; } @@ -164,8 +164,8 @@ public class RandomSubstituteOrExpandValForKeyTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); return true; } diff --git a/test/Garnet.test/SortedSetRemoveTxn.cs b/test/Garnet.test/SortedSetRemoveTxn.cs index 1a2d341dd64..4cb442f94d1 100644 --- a/test/Garnet.test/SortedSetRemoveTxn.cs +++ b/test/Garnet.test/SortedSetRemoveTxn.cs @@ -21,7 +21,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce var offset = 0; var subscriptionContainerKey = GetNextArg(ref procInput.parseState, ref offset); - AddKey(subscriptionContainerKey, LockType.Exclusive, true); + AddKey(subscriptionContainerKey, LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/TestProcedureBitmap.cs b/test/Garnet.test/TestProcedureBitmap.cs index e8f77cdca03..379e80241f3 100644 --- a/test/Garnet.test/TestProcedureBitmap.cs +++ b/test/Garnet.test/TestProcedureBitmap.cs @@ -35,9 +35,9 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (bitmapB.Length == 0) return false; - AddKey(bitmapA, LockType.Exclusive, false); - AddKey(destinationKey, LockType.Exclusive, false); - AddKey(bitmapB, LockType.Exclusive, false); + AddKey(bitmapA, LockType.Exclusive, StoreType.Main); + AddKey(destinationKey, LockType.Exclusive, StoreType.Main); + AddKey(bitmapB, LockType.Exclusive, StoreType.Main); return true; } diff --git a/test/Garnet.test/TestProcedureHLL.cs b/test/Garnet.test/TestProcedureHLL.cs index a47f4b74ff4..8d01e0a1e59 100644 --- a/test/Garnet.test/TestProcedureHLL.cs +++ b/test/Garnet.test/TestProcedureHLL.cs @@ -26,7 +26,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (hll.Length == 0) return false; - AddKey(hll, LockType.Exclusive, false); + AddKey(hll, LockType.Exclusive, StoreType.Main); return true; } diff --git a/test/Garnet.test/TestProcedureHash.cs b/test/Garnet.test/TestProcedureHash.cs index 28aa2f4ac18..ae8cdddf2c6 100644 --- a/test/Garnet.test/TestProcedureHash.cs +++ b/test/Garnet.test/TestProcedureHash.cs @@ -27,7 +27,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (setA.Length == 0) return false; - AddKey(setA, LockType.Exclusive, true); + AddKey(setA, LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/TestProcedureLists.cs b/test/Garnet.test/TestProcedureLists.cs index f6348e0edc8..9cf678d10aa 100644 --- a/test/Garnet.test/TestProcedureLists.cs +++ b/test/Garnet.test/TestProcedureLists.cs @@ -29,9 +29,9 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (lstKey.Length == 0 || lstKeyB.Length == 0 || lstKeyC.Length == 0) return false; - AddKey(lstKey, LockType.Exclusive, true); - AddKey(lstKeyB, LockType.Exclusive, true); - AddKey(lstKeyC, LockType.Exclusive, true); + AddKey(lstKey, LockType.Exclusive, StoreType.Object); + AddKey(lstKeyB, LockType.Exclusive, StoreType.Object); + AddKey(lstKeyC, LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/TestProcedureSet.cs b/test/Garnet.test/TestProcedureSet.cs index 06d42c57e7a..a31a4e983c0 100644 --- a/test/Garnet.test/TestProcedureSet.cs +++ b/test/Garnet.test/TestProcedureSet.cs @@ -26,7 +26,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (setA.Length == 0) return false; - AddKey(setA, LockType.Exclusive, true); + AddKey(setA, LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/TestProcedureSortedSets.cs b/test/Garnet.test/TestProcedureSortedSets.cs index a3e35334c9f..ff3b04f2e48 100644 --- a/test/Garnet.test/TestProcedureSortedSets.cs +++ b/test/Garnet.test/TestProcedureSortedSets.cs @@ -27,7 +27,7 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce if (ssA.Length == 0) return false; - AddKey(ssA, LockType.Exclusive, true); + AddKey(ssA, LockType.Exclusive, StoreType.Object); return true; } diff --git a/test/Garnet.test/WriteWithExpiryTxn.cs b/test/Garnet.test/WriteWithExpiryTxn.cs index 2f2993444cc..4faacf2cc93 100644 --- a/test/Garnet.test/WriteWithExpiryTxn.cs +++ b/test/Garnet.test/WriteWithExpiryTxn.cs @@ -19,7 +19,7 @@ sealed class WriteWithExpiryTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, false); + AddKey(GetNextArg(ref procInput, ref offset), LockType.Exclusive, StoreType.Main); return true; } From feec549b17b2986d74387036137f0b19d23c9a51 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Wed, 24 Sep 2025 15:11:03 -0700 Subject: [PATCH 21/28] format --- libs/cluster/Server/Replication/CheckpointStore.cs | 2 +- .../Server/Replication/ReplicaOps/ReplicaDisklessSync.cs | 2 +- libs/server/API/GarnetApiUnifiedCommands.cs | 2 +- libs/server/OutputHeader.cs | 2 +- .../Storage/Functions/UnifiedStore/CallbackMethods.cs | 2 +- .../Storage/Functions/UnifiedStore/DeleteMethods.cs | 4 ++-- .../Storage/Functions/UnifiedStore/PrivateMethods.cs | 2 +- libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs | 4 ++-- libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs | 8 ++++---- .../Functions/UnifiedStore/UnifiedSessionFunctions.cs | 2 +- .../Storage/Functions/UnifiedStore/UpsertMethods.cs | 2 +- .../Storage/Functions/UnifiedStore/VarLenInputMethods.cs | 2 +- libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs | 2 +- .../Storage/Session/UnifiedStore/CompletePending.cs | 2 +- .../Storage/Session/UnifiedStore/UnifiedStoreOps.cs | 2 +- test/Garnet.test/RespConfigTests.cs | 2 +- test/Garnet.test/RespTests.cs | 2 +- test/Garnet.test/TestUtils.cs | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/libs/cluster/Server/Replication/CheckpointStore.cs b/libs/cluster/Server/Replication/CheckpointStore.cs index 4bc55205f83..8616f5e5be5 100644 --- a/libs/cluster/Server/Replication/CheckpointStore.cs +++ b/libs/cluster/Server/Replication/CheckpointStore.cs @@ -81,7 +81,7 @@ public void PurgeAllCheckpointsExceptEntry(CheckpointEntry entry = null) if (entry == null) return; logger?.LogCheckpointEntry(LogLevel.Trace, nameof(PurgeAllCheckpointsExceptEntry), entry); PurgeAllCheckpointsExceptTokens(entry.metadata.storeHlogToken, entry.metadata.storeIndexToken); - + void PurgeAllCheckpointsExceptTokens(Guid logToken, Guid indexToken) { var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(); diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index d41746c8143..b5c50462e6e 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -182,7 +182,7 @@ public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata, out ReadOnl // Set DB version storeWrapper.store.SetVersion(primarySyncMetadata.currentStoreVersion); - + // Update replicationId to mark any subsequent checkpoints as part of this history logger?.LogInformation("Updating ReplicationId"); TryUpdateMyPrimaryReplId(primarySyncMetadata.currentPrimaryReplId); diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index 8c7d738d5bf..f4f82a4e11a 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -84,4 +84,4 @@ public unsafe GarnetStatus PERSIST(PinnedSpanByte key, ref UnifiedStoreInput inp #endregion } -} +} \ No newline at end of file diff --git a/libs/server/OutputHeader.cs b/libs/server/OutputHeader.cs index 05c1d4ee95d..bb45baf3ba8 100644 --- a/libs/server/OutputHeader.cs +++ b/libs/server/OutputHeader.cs @@ -45,4 +45,4 @@ public struct OutputHeader [FieldOffset(0)] public int result1; } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs b/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs index 6873ef68670..6938cebc2d7 100644 --- a/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/CallbackMethods.cs @@ -20,4 +20,4 @@ public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref UnifiedSt { } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs index fd62eba8460..d3e5943e349 100644 --- a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs @@ -38,8 +38,8 @@ public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); logRecord.ClearValueObject(obj => { }); } - + return true; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs index 2e4cab431e1..956917efd22 100644 --- a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -161,4 +161,4 @@ bool TrySetRecordExpiration(ref LogRecord logRecord, ExpireOption optionType, lo } } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs index bfec4d53294..f2777c2d984 100644 --- a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -99,7 +99,7 @@ public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref _ => throw new NotImplementedException() }; - if (!result) + if (!result) return false; if (shouldUpdateEtag) @@ -315,4 +315,4 @@ private bool HandlePersist(in TSourceLogRecord srcLogRecord, r return true; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs index abc50db04e9..2ee711b8914 100644 --- a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs @@ -30,9 +30,9 @@ public bool Reader(in TSourceLogRecord srcLogRecord, ref Unifi RespCommand.EXISTS => true, RespCommand.MEMORY_USAGE => HandleMemoryUsage(in srcLogRecord, ref input, ref output, ref readInfo), RespCommand.TYPE => HandleType(in srcLogRecord, ref input, ref output, ref readInfo), - RespCommand.TTL or + RespCommand.TTL or RespCommand.PTTL => HandleTtl(in srcLogRecord, ref input, ref output, ref readInfo, cmd == RespCommand.PTTL), - RespCommand.EXPIRETIME or + RespCommand.EXPIRETIME or RespCommand.PEXPIRETIME => HandleExpireTime(in srcLogRecord, ref input, ref output, ref readInfo, cmd == RespCommand.PEXPIRETIME), _ => throw new NotImplementedException(), }; @@ -101,7 +101,7 @@ private bool HandleTtl(in TSourceLogRecord srcLogRecord, ref U var ttlValue = milliseconds ? ConvertUtils.MillisecondsFromDiffUtcNowTicks(expiration) : ConvertUtils.SecondsFromDiffUtcNowTicks(expiration); - + writer.WriteInt64(ttlValue); return true; } @@ -120,4 +120,4 @@ private bool HandleExpireTime(in TSourceLogRecord srcLogRecord return true; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs b/libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs index d45faf69447..733c037cb65 100644 --- a/libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs +++ b/libs/server/Storage/Functions/UnifiedStore/UnifiedSessionFunctions.cs @@ -26,4 +26,4 @@ public void ConvertOutputToHeap(ref UnifiedStoreInput input, ref GarnetUnifiedSt //output.ConvertToHeap(); } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs index 934760c5bac..355bad98e17 100644 --- a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs @@ -211,4 +211,4 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSi return true; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs index 79b60cb5976..6c4c4a40696 100644 --- a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs @@ -130,4 +130,4 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan k }; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs index 672608985d3..605be41209d 100644 --- a/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/AdvancedOps.cs @@ -35,4 +35,4 @@ public GarnetStatus RMW_UnifiedStore(ReadOnlySpan key, re : GarnetStatus.NOTFOUND; } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Session/UnifiedStore/CompletePending.cs b/libs/server/Storage/Session/UnifiedStore/CompletePending.cs index 5810fdca203..9e4a1d153d5 100644 --- a/libs/server/Storage/Session/UnifiedStore/CompletePending.cs +++ b/libs/server/Storage/Session/UnifiedStore/CompletePending.cs @@ -29,4 +29,4 @@ static void CompletePendingForUnifiedStoreSession(ref Status st completedOutputs.Dispose(); } } -} +} \ No newline at end of file diff --git a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs index d1307d7de77..e6b81340bb4 100644 --- a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs @@ -47,4 +47,4 @@ public GarnetStatus DELETE(PinnedSpanByte key, ref TUnifiedCont return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } } -} +} \ No newline at end of file diff --git a/test/Garnet.test/RespConfigTests.cs b/test/Garnet.test/RespConfigTests.cs index 2a2e61adc91..19b7f7a1bad 100644 --- a/test/Garnet.test/RespConfigTests.cs +++ b/test/Garnet.test/RespConfigTests.cs @@ -320,7 +320,7 @@ public void TearDown() [Test] [TestCase("1m", "4m")] [TestCase("1024k", "4000k")] - [TestCase( "1024", "4000")] + [TestCase("1024", "4000")] [TestCase("1024", "4096")] public void ConfigSetMemorySizeUtilizationTest(string smallerSize, string largerSize) { diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index 6162397eec2..1b85d6674b0 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -1800,7 +1800,7 @@ public void PExpiretimeWithUnknownKey() var db = redis.GetDatabase(0); var expireTime = (long)db.Execute("PEXPIRETIME", "keyZ"); - + ClassicAssert.AreEqual(-2, expireTime); } diff --git a/test/Garnet.test/TestUtils.cs b/test/Garnet.test/TestUtils.cs index 9170456fcc6..2e4d51f614a 100644 --- a/test/Garnet.test/TestUtils.cs +++ b/test/Garnet.test/TestUtils.cs @@ -373,7 +373,7 @@ public static GarnetServer CreateGarnetServer( if (lowMemory) { opts.MemorySize = memorySize == default ? "1024" : memorySize; - opts.PageSize = pageSize == default ? "512" : pageSize; + opts.PageSize = pageSize == default ? "512" : pageSize; if (enableReadCache) { opts.ReadCacheMemorySize = opts.MemorySize; From 0db196789f5ae202475d90cc25c7d471a6b97099 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Wed, 24 Sep 2025 15:38:01 -0700 Subject: [PATCH 22/28] Some test fixes --- test/Garnet.test/GarnetServerConfigTests.cs | 6 +++--- test/Garnet.test/MultiDatabaseTests.cs | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/test/Garnet.test/GarnetServerConfigTests.cs b/test/Garnet.test/GarnetServerConfigTests.cs index 953a5d2c4e0..288fb52ac1c 100644 --- a/test/Garnet.test/GarnetServerConfigTests.cs +++ b/test/Garnet.test/GarnetServerConfigTests.cs @@ -142,7 +142,7 @@ public void ImportExportConfigLocal() // Validate non-default configuration options nonDefaultOptions = JsonSerializer.Deserialize>(optionsJson); - ClassicAssert.AreEqual(10, nonDefaultOptions.Count); + ClassicAssert.AreEqual(9, nonDefaultOptions.Count); ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.PageSize))); ClassicAssert.AreEqual("4m", ((JsonElement)nonDefaultOptions[nameof(Options.PageSize)]).GetString()); ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.Port))); @@ -174,7 +174,7 @@ public void ImportExportConfigLocal() // Validate non-default configuration options nonDefaultOptions = JsonSerializer.Deserialize>(optionsJson); - ClassicAssert.AreEqual(10, nonDefaultOptions.Count); + ClassicAssert.AreEqual(9, nonDefaultOptions.Count); ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.PageSize))); ClassicAssert.AreEqual("4m", ((JsonElement)nonDefaultOptions[nameof(Options.PageSize)]).GetString()); @@ -196,7 +196,7 @@ public void ImportExportConfigLocal() // Validate non-default configuration options nonDefaultOptions = JsonSerializer.Deserialize>(optionsJson); - ClassicAssert.AreEqual(11, nonDefaultOptions.Count); + ClassicAssert.AreEqual(9, nonDefaultOptions.Count); ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.PageSize))); ClassicAssert.AreEqual("12m", ((JsonElement)nonDefaultOptions[nameof(Options.PageSize)]).GetString()); ClassicAssert.IsTrue(nonDefaultOptions.ContainsKey(nameof(Options.Port))); diff --git a/test/Garnet.test/MultiDatabaseTests.cs b/test/Garnet.test/MultiDatabaseTests.cs index d1775e64e19..661f514eb07 100644 --- a/test/Garnet.test/MultiDatabaseTests.cs +++ b/test/Garnet.test/MultiDatabaseTests.cs @@ -32,7 +32,7 @@ public void MultiDatabaseBasicSelectTestSE() var db2Key1 = "db2:key1"; var db2Key2 = "db2:key2"; var db12Key1 = "db12:key1"; - var db12Key2 = "db12:key1"; + var db12Key2 = "db12:key2"; using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db1 = redis.GetDatabase(0); @@ -57,7 +57,7 @@ public void MultiDatabaseBasicSelectTestSE() ClassicAssert.IsFalse(db12.KeyExists(db1Key1)); ClassicAssert.IsFalse(db12.KeyExists(db1Key2)); - db2.StringSet(db12Key2, "db12:value2"); + db2.StringSet(db12Key1, "db12:value2"); db2.SetAdd(db12Key2, [new RedisValue("db12:val2"), new RedisValue("db12:val2")]); ClassicAssert.IsFalse(db12.KeyExists(db12Key1)); @@ -438,7 +438,7 @@ public void MultiDatabaseBasicSelectTestLC() var db1Key1 = "db1:key1"; var db1Key2 = "db1:key2"; var db2Key1 = "db2:key1"; - var db2Key2 = "db2:key1"; + var db2Key2 = "db2:key2"; using var lightClientRequest = TestUtils.CreateRequest(); From 360c7e63bf410b0e7842f4867ca0e83e9daedb06 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 25 Sep 2025 13:07:26 -0700 Subject: [PATCH 23/28] Fixing tests --- libs/server/AOF/AofEntryType.cs | 12 ++- libs/server/AOF/AofProcessor.cs | 80 ++++++++++++++++ libs/server/Lua/LuaRunner.Functions.cs | 2 +- libs/server/Lua/LuaRunner.cs | 2 +- .../Functions/UnifiedStore/PrivateMethods.cs | 5 +- .../Common/ArrayKeyIterationFunctions.cs | 8 +- libs/server/Transaction/TransactionManager.cs | 16 +--- libs/server/Transaction/TxnKeyEntry.cs | 91 ++----------------- .../Transaction/TxnKeyEntryComparison.cs | 15 +-- libs/server/Transaction/TxnKeyManager.cs | 2 +- test/Garnet.test/RespCustomCommandTests.cs | 23 ++--- test/Garnet.test/RespEtagTests.cs | 18 ++-- 12 files changed, 137 insertions(+), 137 deletions(-) diff --git a/libs/server/AOF/AofEntryType.cs b/libs/server/AOF/AofEntryType.cs index 315fc765bfe..1a3369166d9 100644 --- a/libs/server/AOF/AofEntryType.cs +++ b/libs/server/AOF/AofEntryType.cs @@ -80,17 +80,21 @@ public enum AofEntryType : byte FlushDb = 0x61, /// - /// Unified store upsert + /// Unified store upsert sting /// - UnifiedStoreUpsert = 0x70, + UnifiedStoreStringUpsert = 0x70, + /// + /// Unified store upsert object + /// + UnifiedStoreObjectUpsert = 0x71, /// /// Unified store RMW /// - UnifiedStoreRMW = 0x71, + UnifiedStoreRMW = 0x72, /// /// Unified store delete /// - UnifiedStoreDelete = 0x72, + UnifiedStoreDelete = 0x73, #region Deprecated markers /// diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index f041524a2b9..73f7468d0ff 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -26,6 +26,7 @@ public sealed unsafe partial class AofProcessor private readonly RawStringInput storeInput; private readonly ObjectInput objectStoreInput; + private readonly UnifiedStoreInput unifiedStoreInput; private readonly CustomProcedureInput customProcInput; private readonly SessionParseState parseState; @@ -82,6 +83,7 @@ public AofProcessor( parseState.Initialize(); storeInput.parseState = parseState; objectStoreInput.parseState = parseState; + unifiedStoreInput.parseState = parseState; customProcInput.parseState = parseState; inflightTxns = []; @@ -357,6 +359,18 @@ private unsafe bool ReplayOp(byte* entryPtr, int length, bool replayAsReplica) case AofEntryType.ObjectStoreDelete: ObjectStoreDelete(objectStoreBasicContext, entryPtr); break; + case AofEntryType.UnifiedStoreRMW: + UnifiedStoreRMW(unifiedStoreBasicContext, unifiedStoreInput, entryPtr, bufferPtr, buffer.Length); + break; + case AofEntryType.UnifiedStoreStringUpsert: + UnifiedStoreStringUpsert(unifiedStoreBasicContext, unifiedStoreInput, entryPtr, bufferPtr, buffer.Length); + break; + case AofEntryType.UnifiedStoreObjectUpsert: + UnifiedStoreObjectUpsert(unifiedStoreBasicContext, storeWrapper.GarnetObjectSerializer, entryPtr, bufferPtr, buffer.Length); + break; + case AofEntryType.UnifiedStoreDelete: + UnifiedStoreDelete(unifiedStoreBasicContext, entryPtr); + break; case AofEntryType.StoredProcedure: RunStoredProc(header.procedureId, customProcInput, entryPtr); break; @@ -487,6 +501,72 @@ static void ObjectStoreDelete(BasicContext basicContext, + UnifiedStoreInput storeInput, byte* ptr, byte* outputPtr, int outputLength) + { + var curr = ptr + sizeof(AofHeader); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + curr += key.TotalSize; + + var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + curr += value.TotalSize; + + // Reconstructing UnifiedStoreInput + + // input + _ = storeInput.DeserializeFrom(curr); + + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength); + basicContext.Upsert(key.ReadOnlySpan, ref storeInput, value.ReadOnlySpan, ref output); + if (!output.SpanByteAndMemory.IsSpanByte) + output.SpanByteAndMemory.Memory.Dispose(); + } + + static void UnifiedStoreObjectUpsert(BasicContext basicContext, + GarnetObjectSerializer garnetObjectSerializer, byte* ptr, byte* outputPtr, int outputLength) + { + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); + + var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader) + key.TotalSize); + var valB = garnetObjectSerializer.Deserialize(value.ToArray()); + + // input + // TODOMigrate: _ = unifiedStoreInput.DeserializeFrom(curr); // TODO - need to serialize this as well + + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength); + basicContext.Upsert(key.ReadOnlySpan, valB); + if (!output.SpanByteAndMemory.IsSpanByte) + output.SpanByteAndMemory.Memory.Dispose(); + } + + static void UnifiedStoreRMW(BasicContext basicContext, + UnifiedStoreInput unifiedStoreInput, byte* ptr, byte* outputPtr, int outputLength) + { + var curr = ptr + sizeof(AofHeader); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + curr += key.TotalSize; + + // Reconstructing UnifiedStoreInput + + // input + _ = unifiedStoreInput.DeserializeFrom(curr); + + // Call RMW with the reconstructed key & UnifiedStoreInput + var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength); + if (basicContext.RMW(key.ReadOnlySpan, ref unifiedStoreInput, ref output).IsPending) + basicContext.CompletePending(true); + + if (!output.SpanByteAndMemory.IsSpanByte) + output.SpanByteAndMemory.Memory.Dispose(); + } + + static void UnifiedStoreDelete( + BasicContext basicContext, byte* ptr) + { + var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); + basicContext.Delete(key); + } + /// /// On recovery apply records with header.version greater than CurrentVersion. /// diff --git a/libs/server/Lua/LuaRunner.Functions.cs b/libs/server/Lua/LuaRunner.Functions.cs index 8fdf814aea6..56246344555 100644 --- a/libs/server/Lua/LuaRunner.Functions.cs +++ b/libs/server/Lua/LuaRunner.Functions.cs @@ -227,7 +227,7 @@ internal int UnsafeRunPreambleForSession(nint luaStatePtr) if (txnMode) { - txnKeyEntries.AddKey(key, StoreType.All, LockType.Exclusive); + txnKeyEntries.AddKey(key, LockType.Exclusive); } // Equivalent to KEYS[i+1] = key diff --git a/libs/server/Lua/LuaRunner.cs b/libs/server/Lua/LuaRunner.cs index c84c83cd73e..0a6c6ccbd31 100644 --- a/libs/server/Lua/LuaRunner.cs +++ b/libs/server/Lua/LuaRunner.cs @@ -1239,7 +1239,7 @@ public unsafe object RunForRunner(string[] keys = null, string[] argv = null) foreach (var key in keys) { var _key = scratchBufferBuilder.CreateArgSlice(key); - txnKeyEntries.AddKey(_key, StoreType.All, Tsavorite.core.LockType.Exclusive); + txnKeyEntries.AddKey(_key, Tsavorite.core.LockType.Exclusive); } adapter = new(scratchBufferBuilder); diff --git a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs index 956917efd22..1d2a155d832 100644 --- a/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/PrivateMethods.cs @@ -27,7 +27,7 @@ void WriteLogUpsert(ReadOnlySpan key, ref UnifiedStoreInput input, ReadOnl input.header.flags |= RespInputFlags.Deterministic; functionsState.appendOnlyFile.Enqueue( - new AofHeader { opType = AofEntryType.UnifiedStoreUpsert, storeVersion = version, sessionID = sessionID }, + new AofHeader { opType = AofEntryType.UnifiedStoreStringUpsert, storeVersion = version, sessionID = sessionID }, key, value, out _); } @@ -40,13 +40,14 @@ void WriteLogUpsert(ReadOnlySpan key, ref UnifiedStoreInput input, IGarnet { if (functionsState.StoredProcMode) return; + input.header.flags |= RespInputFlags.Deterministic; GarnetObjectSerializer.Serialize(value, out var valueBytes); fixed (byte* valPtr = valueBytes) { functionsState.appendOnlyFile.Enqueue( - new AofHeader { opType = AofEntryType.UnifiedStoreUpsert, storeVersion = version, sessionID = sessionID }, + new AofHeader { opType = AofEntryType.UnifiedStoreObjectUpsert, storeVersion = version, sessionID = sessionID }, key, new ReadOnlySpan(valPtr, valueBytes.Length), out _); } } diff --git a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs index 62396525862..f4920b91824 100644 --- a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs +++ b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs @@ -65,6 +65,10 @@ internal unsafe bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, { matchType = typeof(HashObject); } + else if (typeObject.SequenceEqual(CmdStrings.STRING) || typeObject.SequenceEqual(CmdStrings.stringt)) + { + matchType = typeof(string); + } else if (!typeObject.SequenceEqual(CmdStrings.STRING) && !typeObject.SequenceEqual(CmdStrings.stringt)) { // Unexpected typeObject type @@ -303,7 +307,9 @@ public bool Reader(in TSourceLogRecord logRecord, RecordMetada } } - if (info.matchType != null && logRecord.ValueObject.GetType() != info.matchType) + if (info.matchType != null && + ((logRecord.Info.ValueIsObject && (info.matchType == typeof(string) || info.matchType != logRecord.ValueObject.GetType())) || + (!logRecord.Info.ValueIsObject && info.matchType != typeof(string)))) { cursorRecordResult = CursorRecordResult.Skip; return true; diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index d6ad748ce20..0a151a765b4 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -94,7 +94,6 @@ public sealed unsafe partial class TransactionManager long txnVersion; private bool isMainTxn = false; private bool isObjectTxn = false; - private bool isUnifiedTxn = false; internal TransactionalContext TransactionalContext => transactionalContext; @@ -170,8 +169,7 @@ internal void Reset(bool isRunning) transactionalContext.EndTransaction(); if (isObjectTxn) objectStoreTransactionalContext.EndTransaction(); - if (isUnifiedTxn) - unifiedStoreTransactionalContext.EndTransaction(); + unifiedStoreTransactionalContext.EndTransaction(); } finally { @@ -184,7 +182,6 @@ internal void Reset(bool isRunning) this.state = TxnState.None; this.isMainTxn = false; this.isObjectTxn = false; - this.isUnifiedTxn = false; functionsState.StoredProcMode = false; // Reset cluster variables used for slot verification @@ -325,9 +322,6 @@ void UpdateTransactionStoreType(StoreType type) case StoreType.Object: isObjectTxn = true; break; - case StoreType.All: - isUnifiedTxn = true; - break; } } @@ -348,8 +342,8 @@ void BeginTransaction() transactionalContext.BeginTransaction(); if (isObjectTxn) objectStoreTransactionalContext.BeginTransaction(); - if (isUnifiedTxn) - unifiedStoreTransactionalContext.BeginTransaction(); + + unifiedStoreTransactionalContext.BeginTransaction(); } void LocksAcquired(long txnVersion) @@ -358,8 +352,8 @@ void LocksAcquired(long txnVersion) transactionalContext.LocksAcquired(txnVersion); if (isObjectTxn) objectStoreTransactionalContext.LocksAcquired(txnVersion); - if (isUnifiedTxn) - unifiedStoreTransactionalContext.LocksAcquired(txnVersion); + + unifiedStoreTransactionalContext.LocksAcquired(txnVersion); } internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, TimeSpan lock_timeout = default) diff --git a/libs/server/Transaction/TxnKeyEntry.cs b/libs/server/Transaction/TxnKeyEntry.cs index 5311ebbeaa3..e6ec46826c1 100644 --- a/libs/server/Transaction/TxnKeyEntry.cs +++ b/libs/server/Transaction/TxnKeyEntry.cs @@ -14,16 +14,13 @@ namespace Garnet.server /// /// Entry for a key to lock and unlock in transactions /// - [StructLayout(LayoutKind.Explicit, Size = 10)] + [StructLayout(LayoutKind.Explicit, Size = 9)] struct TxnKeyEntry : ITransactionalKey { [FieldOffset(0)] internal long keyHash; [FieldOffset(8)] - internal StoreType storeType; - - [FieldOffset(9)] internal LockType lockType; #region ITransactionalKey @@ -40,14 +37,7 @@ public override readonly string ToString() // The debugger often can't call the Globalization NegativeSign property so ToString() would just display the class name var keyHashSign = keyHash < 0 ? "-" : string.Empty; var absKeyHash = keyHash >= 0 ? keyHash : -keyHash; - var storeTypeStr = storeType switch - { - StoreType.Object => "obj", - StoreType.Main => "raw", - StoreType.All => "uni", - _ => "unk" - }; - return $"{keyHashSign}{absKeyHash}:{storeTypeStr}:{(lockType == LockType.None ? "-" : (lockType == LockType.Shared ? "s" : "x"))}"; + return $"{keyHashSign}{absKeyHash}:{(lockType == LockType.None ? "-" : (lockType == LockType.Shared ? "s" : "x"))}"; } } @@ -55,12 +45,8 @@ internal sealed class TxnKeyEntries { // Basic keys int keyCount; - int mainKeyCount; - int objectKeyCount; TxnKeyEntry[] keys; - bool mainStoreKeyLocked; - bool objectStoreKeyLocked; bool unifiedStoreKeyLocked; readonly TxnKeyComparison comparison; @@ -93,15 +79,9 @@ public bool IsReadOnly } } - public void AddKey(PinnedSpanByte keyArgSlice, StoreType storeType, LockType type) + public void AddKey(PinnedSpanByte keyArgSlice, LockType type) { - var keyHash = storeType switch - { - StoreType.Main => comparison.transactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), - StoreType.Object => comparison.objectStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), - StoreType.All => comparison.unifiedStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan), - _ => throw new ArgumentOutOfRangeException(nameof(storeType), "Invalid store type") - }; + var keyHash = comparison.unifiedStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan); // Grow the buffer if needed if (keyCount >= keys.Length) @@ -113,13 +93,8 @@ public void AddKey(PinnedSpanByte keyArgSlice, StoreType storeType, LockType typ // Populate the new key slot. keys[keyCount].keyHash = keyHash; - keys[keyCount].storeType = storeType; keys[keyCount].lockType = type; ++keyCount; - if (storeType == StoreType.Main) - ++mainKeyCount; - else if (storeType == StoreType.Object) - ++objectKeyCount; } internal void LockAllKeys() @@ -132,24 +107,10 @@ internal void LockAllKeys() // This does not call Tsavorite's SortKeyHashes because we need to consider isObject as well. MemoryExtensions.Sort(keys.AsSpan().Slice(0, keyCount), comparison.comparisonDelegate); - // Issue main store locks - if (mainKeyCount > 0) - { - comparison.transactionalContext.Lock(keys.AsSpan()[..mainKeyCount]); - mainStoreKeyLocked = true; - } - - // Issue object store locks - if (objectKeyCount > 0) - { - comparison.objectStoreTransactionalContext.Lock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount)); - objectStoreKeyLocked = true; - } - // Issue unified store locks - if (mainKeyCount + objectKeyCount < keyCount) + if (keyCount > 0) { - comparison.unifiedStoreTransactionalContext.Lock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount)); + comparison.unifiedStoreTransactionalContext.Lock(keys.AsSpan().Slice(0,keyCount)); unifiedStoreKeyLocked = true; } @@ -166,35 +127,11 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) // This does not call Tsavorite's SortKeyHashes because we need to consider isObject as well. MemoryExtensions.Sort(keys.AsSpan().Slice(0, keyCount), comparison.comparisonDelegate); - // Issue main store locks - // TryLock will unlock automatically in case of partial failure - if (mainKeyCount > 0) - { - mainStoreKeyLocked = comparison.transactionalContext.TryLock(keys.AsSpan()[..mainKeyCount], lock_timeout); - if (!mainStoreKeyLocked) - { - phase = 0; - return false; - } - } - - // Issue object store locks - // TryLock will unlock automatically in case of partial failure - if (objectKeyCount > 0) - { - objectStoreKeyLocked = comparison.objectStoreTransactionalContext.TryLock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount), lock_timeout); - if (!objectStoreKeyLocked) - { - phase = 0; - return false; - } - } - // Issue unified store locks // TryLock will unlock automatically in case of partial failure - if (mainKeyCount + objectKeyCount < keyCount) + if (keyCount > 0) { - unifiedStoreKeyLocked = comparison.unifiedStoreTransactionalContext.TryLock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount), lock_timeout); + unifiedStoreKeyLocked = comparison.unifiedStoreTransactionalContext.TryLock(keys.AsSpan().Slice(0, keyCount), lock_timeout); if (!unifiedStoreKeyLocked) { phase = 0; @@ -209,17 +146,9 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) internal void UnlockAllKeys() { phase = 2; - if (mainStoreKeyLocked && mainKeyCount > 0) - comparison.transactionalContext.Unlock(keys.AsSpan()[..mainKeyCount]); - if (objectStoreKeyLocked && objectKeyCount > 0) - comparison.objectStoreTransactionalContext.Unlock(keys.AsSpan().Slice(mainKeyCount, objectKeyCount)); - if (unifiedStoreKeyLocked && mainKeyCount + objectKeyCount < keyCount) - comparison.unifiedStoreTransactionalContext.Unlock(keys.AsSpan().Slice(objectKeyCount + mainKeyCount, keyCount - mainKeyCount - objectKeyCount)); - mainKeyCount = 0; - objectKeyCount = 0; + if (unifiedStoreKeyLocked && keyCount > 0) + comparison.unifiedStoreTransactionalContext.Unlock(keys.AsSpan().Slice(0, keyCount)); keyCount = 0; - mainStoreKeyLocked = false; - objectStoreKeyLocked = false; unifiedStoreKeyLocked = false; phase = 0; } diff --git a/libs/server/Transaction/TxnKeyEntryComparison.cs b/libs/server/Transaction/TxnKeyEntryComparison.cs index 12c8824eef9..c64a24a8167 100644 --- a/libs/server/Transaction/TxnKeyEntryComparison.cs +++ b/libs/server/Transaction/TxnKeyEntryComparison.cs @@ -29,19 +29,6 @@ internal TxnKeyComparison(TransactionalContext public int Compare(TxnKeyEntry key1, TxnKeyEntry key2) - { - // This sorts by storeType, then calls Tsavorite to sort by lock code and then by lockType. - var cmp = key1.storeType.CompareTo(key2.storeType); - if (cmp != 0) - return cmp; - - return key1.storeType switch - { - StoreType.Main => transactionalContext.CompareKeyHashes(ref key1, ref key2), - StoreType.Object => objectStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2), - StoreType.All => unifiedStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2), - _ => throw new InvalidOperationException($"Unknown store type"), - }; - } + => unifiedStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2); } } \ No newline at end of file diff --git a/libs/server/Transaction/TxnKeyManager.cs b/libs/server/Transaction/TxnKeyManager.cs index 7f0f4318853..587549626f8 100644 --- a/libs/server/Transaction/TxnKeyManager.cs +++ b/libs/server/Transaction/TxnKeyManager.cs @@ -17,7 +17,7 @@ sealed partial class TransactionManager public void SaveKeyEntryToLock(PinnedSpanByte key, StoreType storeType, LockType type) { UpdateTransactionStoreType(storeType); - keyEntries.AddKey(key, storeType, type); + keyEntries.AddKey(key, type); } /// diff --git a/test/Garnet.test/RespCustomCommandTests.cs b/test/Garnet.test/RespCustomCommandTests.cs index c441de0859b..ff01793cf88 100644 --- a/test/Garnet.test/RespCustomCommandTests.cs +++ b/test/Garnet.test/RespCustomCommandTests.cs @@ -974,14 +974,15 @@ public void RegisterCustomCommandTest() [.. args]); // Test READWRITETX - string key = "readkey"; + string key1 = "readkey1"; + string key2 = "readkey2"; string value = "foovalue0"; - db.StringSet(key, value); + db.StringSet(key1, value); string writekey1 = "writekey1"; string writekey2 = "writekey2"; - var result = db.Execute("READWRITETX", key, writekey1, writekey2); + var result = db.Execute("READWRITETX", key1, writekey1, writekey2); ClassicAssert.AreEqual("SUCCESS", (string)result); // Read keys to verify transaction succeeded @@ -997,32 +998,32 @@ public void RegisterCustomCommandTest() string newValue2 = "foovalue2"; // This conditional set should pass (prefix matches) - result = db.Execute("SETIFPM", key, newValue1, "foo"); + result = db.Execute("SETIFPM", key1, newValue1, "foo"); ClassicAssert.AreEqual("OK", (string)result); - retValue = db.StringGet(key); + retValue = db.StringGet(key1); ClassicAssert.AreEqual(newValue1, retValue); // This conditional set should fail (prefix does not match) - result = db.Execute("SETIFPM", key, newValue2, "bar"); + result = db.Execute("SETIFPM", key1, newValue2, "bar"); ClassicAssert.AreEqual("OK", (string)result); - retValue = db.StringGet(key); + retValue = db.StringGet(key1); ClassicAssert.AreEqual(newValue1, retValue); // Test MYDICTSET string newKey1 = "newkey1"; string newKey2 = "newkey2"; - db.Execute("MYDICTSET", key, newKey1, newValue1); + db.Execute("MYDICTSET", key2, newKey1, newValue1); - var dictVal = db.Execute("MYDICTGET", key, newKey1); + var dictVal = db.Execute("MYDICTGET", key2, newKey1); ClassicAssert.AreEqual(newValue1, (string)dictVal); - db.Execute("MYDICTSET", key, newKey2, newValue2); + db.Execute("MYDICTSET", key2, newKey2, newValue2); // Test MYDICTGET - dictVal = db.Execute("MYDICTGET", key, newKey2); + dictVal = db.Execute("MYDICTGET", key2, newKey2); ClassicAssert.AreEqual(newValue2, (string)dictVal); } diff --git a/test/Garnet.test/RespEtagTests.cs b/test/Garnet.test/RespEtagTests.cs index a4b434c146c..61b193e714f 100644 --- a/test/Garnet.test/RespEtagTests.cs +++ b/test/Garnet.test/RespEtagTests.cs @@ -1817,21 +1817,19 @@ public void MainObjectKeyForEtagSetData() ClassicAssert.AreEqual(1, long.Parse(db.Execute("SET", key, "v1", "WITHETAG").ToString())); - // Do SetAdd using the same key - ClassicAssert.IsTrue(db.SetAdd(key, "v2")); + // Do SetAdd using the same key, expected error + Assert.Throws(() => db.SetAdd(key, "v2"), + Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE)); - // Two keys "test:1" - this is expected as of now - // because Garnet has a separate main and object store + // One key "test:1" with a string value is expected var keys = server.Keys(db.Database, key).ToList(); - ClassicAssert.AreEqual(2, keys.Count); + ClassicAssert.AreEqual(1, keys.Count); ClassicAssert.AreEqual(key, (string)keys[0]); - ClassicAssert.AreEqual(key, (string)keys[1]); + var value = db.StringGet(key); + ClassicAssert.AreEqual("v1", (string)value); // do ListRightPush using the same key, expected error - var ex = Assert.Throws(() => db.ListRightPush(key, "v3")); - var expectedError = Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE); - ClassicAssert.IsNotNull(ex); - ClassicAssert.AreEqual(expectedError, ex.Message); + Assert.Throws(() => db.ListRightPush(key, "v3"), Encoding.ASCII.GetString(CmdStrings.RESP_ERR_WRONG_TYPE)); } [Test] From b68e53817fcfd54bb1bfa90171cb164229b6e3d5 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 25 Sep 2025 13:37:54 -0700 Subject: [PATCH 24/28] reverting a couple of unnecessary changes --- .../Server/Migration/MigrateOperation.cs | 34 ++++++++++++++----- .../Server/Migration/MigrateSessionSlots.cs | 21 ++++++------ 2 files changed, 37 insertions(+), 18 deletions(-) diff --git a/libs/cluster/Server/Migration/MigrateOperation.cs b/libs/cluster/Server/Migration/MigrateOperation.cs index 3287f7f73ad..dcdf0721bc7 100644 --- a/libs/cluster/Server/Migration/MigrateOperation.cs +++ b/libs/cluster/Server/Migration/MigrateOperation.cs @@ -55,17 +55,23 @@ public void Dispose() /// /// Perform scan to gather keys and build sketch /// + /// /// /// - public void Scan(ref long currentAddress, long endAddress) - => _ = localServerSession.BasicGarnetApi.IterateMainStore(ref mss, ref currentAddress, endAddress, - endAddress, includeTombstones: true); + public void Scan(StoreType storeType, ref long currentAddress, long endAddress) + { + if (storeType == StoreType.Main) + _ = localServerSession.BasicGarnetApi.IterateMainStore(ref mss, ref currentAddress, endAddress, endAddress, includeTombstones: true); + else if (storeType == StoreType.Object) + _ = localServerSession.BasicGarnetApi.IterateObjectStore(ref oss, ref currentAddress, endAddress, endAddress, includeTombstones: true); + } /// /// Transmit gathered keys /// + /// /// - public bool TransmitSlots() + public bool TransmitSlots(StoreType storeType) { // Use this for both stores; main store will just use the SpanByteAndMemory directly. We want it to be outside iterations // so we can reuse the SpanByteAndMemory.Memory across iterations. @@ -73,11 +79,23 @@ public bool TransmitSlots() try { - var input = new RawStringInput(RespCommandAccessor.MIGRATE); - foreach (var key in sketch.argSliceVector) + if (storeType == StoreType.Main) { - if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output.SpanByteAndMemory, out _)) - return false; + var input = new RawStringInput(RespCommandAccessor.MIGRATE); + foreach (var key in sketch.argSliceVector) + { + if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output.SpanByteAndMemory, out _)) + return false; + } + } + else + { + var input = new ObjectInput(new RespInputHeader(GarnetObjectType.Migrate)); + foreach (var key in sketch.argSliceVector) + { + if (!session.WriteOrSendObjectStoreKeyValuePair(gcs, localServerSession, key, ref input, ref output, out _)) + return false; + } } // Flush final data in client buffer diff --git a/libs/cluster/Server/Migration/MigrateSessionSlots.cs b/libs/cluster/Server/Migration/MigrateSessionSlots.cs index 5589d503680..5406c79141d 100644 --- a/libs/cluster/Server/Migration/MigrateSessionSlots.cs +++ b/libs/cluster/Server/Migration/MigrateSessionSlots.cs @@ -6,6 +6,7 @@ #if DEBUG using Garnet.common; #endif +using Garnet.server; using Microsoft.Extensions.Logging; namespace Garnet.cluster @@ -27,22 +28,22 @@ public async Task MigrateSlotsDriverInline() ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Migration_Slot_End_Scan_Range_Acquisition).GetAwaiter().GetResult(); #endif - // Send store + // Send main store logger?.LogWarning("Store migrate scan range [{storeBeginAddress}, {storeTailAddress}]", storeBeginAddress, storeTailAddress); - var success = await CreateAndRunMigrateTasks(storeBeginAddress, storeTailAddress, mainStorePageSize); + var success = await CreateAndRunMigrateTasks(StoreType.Main, storeBeginAddress, storeTailAddress, mainStorePageSize); if (!success) return false; return true; - async Task CreateAndRunMigrateTasks(long beginAddress, long tailAddress, int pageSize) + async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress, long tailAddress, int pageSize) { - logger?.LogTrace("{method} > Scan in range ({BeginAddress},{TailAddress})", nameof(CreateAndRunMigrateTasks), beginAddress, tailAddress); + logger?.LogTrace("{method} > [{storeType}] Scan in range ({BeginAddress},{TailAddress})", nameof(CreateAndRunMigrateTasks), storeType, beginAddress, tailAddress); var migrateOperationRunners = new Task[clusterProvider.serverOptions.ParallelMigrateTaskCount]; var i = 0; while (i < migrateOperationRunners.Length) { var idx = i; - migrateOperationRunners[idx] = Task.Run(() => ScanStoreTask(idx, beginAddress, tailAddress, pageSize)); + migrateOperationRunners[idx] = Task.Run(() => ScanStoreTask(idx, storeType, beginAddress, tailAddress, pageSize)); i++; } @@ -52,14 +53,14 @@ async Task CreateAndRunMigrateTasks(long beginAddress, long tailAddress, i } catch (Exception ex) { - logger?.LogError(ex, "{CreateAndRunMigrateTasks}: {beginAddress} {tailAddress} {pageSize}", nameof(CreateAndRunMigrateTasks), beginAddress, tailAddress, pageSize); + logger?.LogError(ex, "{CreateAndRunMigrateTasks}: {storeType} {beginAddress} {tailAddress} {pageSize}", nameof(CreateAndRunMigrateTasks), storeType, beginAddress, tailAddress, pageSize); _cts.Cancel(); return false; } return true; } - Task ScanStoreTask(int taskId, long beginAddress, long tailAddress, int pageSize) + Task ScanStoreTask(int taskId, StoreType storeType, long beginAddress, long tailAddress, int pageSize) { var migrateOperation = this.migrateOperation[taskId]; var range = (tailAddress - beginAddress) / clusterProvider.storeWrapper.serverOptions.ParallelMigrateTaskCount; @@ -72,13 +73,13 @@ Task ScanStoreTask(int taskId, long beginAddress, long tailAddress, int pa return Task.FromResult(false); var cursor = workerStartAddress; - logger?.LogWarning("<{taskId}> migrate scan range [{workerStartAddress}, {workerEndAddress}]", taskId, workerStartAddress, workerEndAddress); + logger?.LogWarning("<{StoreType}:{taskId}> migrate scan range [{workerStartAddress}, {workerEndAddress}]", storeType, taskId, workerStartAddress, workerEndAddress); while (true) { var current = cursor; // Build Sketch migrateOperation.sketch.SetStatus(SketchStatus.INITIALIZING); - migrateOperation.Scan(ref current, workerEndAddress); + migrateOperation.Scan(storeType, ref current, workerEndAddress); // Stop if no keys have been found if (migrateOperation.sketch.argSliceVector.IsEmpty) break; @@ -91,7 +92,7 @@ Task ScanStoreTask(int taskId, long beginAddress, long tailAddress, int pa WaitForConfigPropagation(); // Transmit all keys gathered - migrateOperation.TransmitSlots(); + migrateOperation.TransmitSlots(storeType); // Transition EPSM to DELETING migrateOperation.sketch.SetStatus(SketchStatus.DELETING); From 485dd03a1a54114ccd8911d29458324c733265d2 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Thu, 25 Sep 2025 17:32:06 -0700 Subject: [PATCH 25/28] Eliminating more multi-context methods from API --- .../Session/RespClusterMigrateCommands.cs | 4 +- .../Session/RespClusterReplicationCommands.cs | 4 +- libs/server/API/GarnetApi.cs | 33 +-- libs/server/API/GarnetApiUnifiedCommands.cs | 21 ++ libs/server/API/IGarnetApi.cs | 23 +- .../ItemBroker/CollectionItemBroker.cs | 9 +- .../Storage/Session/MainStore/MainStoreOps.cs | 243 +----------------- .../Storage/Session/ObjectStore/ListOps.cs | 10 +- .../Storage/Session/ObjectStore/SetOps.cs | 16 +- .../Session/ObjectStore/SortedSetGeoOps.cs | 11 +- .../Session/ObjectStore/SortedSetOps.cs | 23 +- .../Session/UnifiedStore/UnifiedStoreOps.cs | 89 +++++++ libs/server/Transaction/TransactionManager.cs | 6 + test/Garnet.test/Extensions/RateLimiterTxn.cs | 2 +- test/Garnet.test/ObjectExpiryTxn.cs | 2 +- 15 files changed, 188 insertions(+), 308 deletions(-) diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 1980f807e06..465a39de360 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -116,7 +116,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan, // Set if key replace flag is set or key does not exist var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); if (replaceOption || !Exists(keySlice)) - _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Main); + _ = basicGarnetApi.SET_Main(in diskLogRecord); i++; } } @@ -148,7 +148,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan, // Set if key replace flag is set or key does not exist var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); if (replaceOption || !Exists(keySlice)) - _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Object); + _ = basicGarnetApi.SET_Object(in diskLogRecord); i++; } } diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index 9d44d2b2d31..b9caf99708d 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -472,7 +472,7 @@ private bool NetworkClusterSync(out bool invalidParameters) return false; var diskLogRecord = new DiskLogRecord(startAddress, length); - _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Main); + _ = basicGarnetApi.SET_Main(in diskLogRecord); i++; } } @@ -486,7 +486,7 @@ private bool NetworkClusterSync(out bool invalidParameters) var diskLogRecord = new DiskLogRecord(startAddress, length); _ = diskLogRecord.DeserializeValueObject(clusterProvider.storeWrapper.GarnetObjectSerializer); - _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Object); + _ = basicGarnetApi.SET_Object(in diskLogRecord); i++; } } diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs index fcb493761ac..617eac186eb 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -117,9 +117,15 @@ public GarnetStatus SET(PinnedSpanByte key, IGarnetObject value) => storageSession.SET(key, value, ref objectContext); /// - public GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType) + public GarnetStatus SET_Main(in TSourceLogRecord srcLogRecord) where TSourceLogRecord : ISourceLogRecord - => storageSession.SET(in srcLogRecord, storeType, ref context, ref objectContext); + => storageSession.SET_Main(in srcLogRecord, ref context); + + /// + public GarnetStatus SET_Object(in TSourceLogRecord srcLogRecord) + where TSourceLogRecord : ISourceLogRecord + => storageSession.SET_Object(in srcLogRecord, ref objectContext); + #endregion #region SETEX @@ -169,29 +175,6 @@ public GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out i => storageSession.RENAMENX(oldKey, newKey, storeType, out result, withEtag); #endregion - #region EXPIRE - - /// - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) - => storageSession.EXPIRE(key, expiryMs, out timeoutSet, storeType, expireOption, ref context, ref objectContext); - - /// - public GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) - => storageSession.EXPIRE(key, expiry, out timeoutSet, storeType, expireOption, ref context, ref objectContext); - #endregion - - #region EXPIREAT - - /// - public GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) - => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectContext); - - /// - public GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) - => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectContext, milliseconds: true); - - #endregion - #region Increment (INCR, INCRBY, DECR, DECRBY) /// public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output) diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs index f4f82a4e11a..8f86389a373 100644 --- a/libs/server/API/GarnetApiUnifiedCommands.cs +++ b/libs/server/API/GarnetApiUnifiedCommands.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using Tsavorite.core; namespace Garnet.server @@ -74,6 +75,26 @@ public GarnetStatus DELETE(PinnedSpanByte key) public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output) => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext); + /// + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None) + => storageSession.EXPIRE(key, expiryMs, out timeoutSet, expireOption, ref unifiedContext); + + /// + public GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None) + => storageSession.EXPIRE(key, expiry, out timeoutSet, expireOption, ref unifiedContext); + + #endregion + + #region EXPIREAT + + /// + public GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None) + => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, expireOption, ref unifiedContext); + + /// + public GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None) + => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, expireOption, ref unifiedContext, milliseconds: true); + #endregion #region PERSIST diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 431ec266339..72726bb0d35 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -59,10 +59,15 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi GarnetStatus SET(PinnedSpanByte key, IGarnetObject value); /// - /// SET + /// SET in main store /// - GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType) - where TSourceLogRecord : ISourceLogRecord; + GarnetStatus SET_Main(in TSourceLogRecord srcLogRecord) where TSourceLogRecord : ISourceLogRecord; + + /// + /// SET in object store + /// + GarnetStatus SET_Object(in TSourceLogRecord srcLogRecord) where TSourceLogRecord : ISourceLogRecord; + #endregion #region SETEX @@ -175,10 +180,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// Key /// Expiry in milliseconds, formatted as ASCII digits /// Whether timeout was set by the call - /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None); /// /// Set a timeout on key using a timeSpan in seconds @@ -195,10 +199,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// Key /// Expiry in TimeSpan /// Whether timeout was set by the call - /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None); #endregion #region EXPIREAT @@ -209,10 +212,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// Key /// Absolute Unix timestamp in seconds /// Whether timeout was set by the call - /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None); /// /// Set a timeout on key using absolute Unix timestamp (seconds since January 1, 1970) in milliseconds @@ -220,10 +222,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s /// Key /// Absolute Unix timestamp in milliseconds /// Whether timeout was set by the call - /// Store type: main, object, or both /// Expire option /// - GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None); #endregion diff --git a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs index ba93e605b4e..b130cf907c2 100644 --- a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs +++ b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs @@ -539,6 +539,7 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp var transactionalContext = storageSession.txnManager.TransactionalContext; var objectTransactionalContext = storageSession.txnManager.ObjectStoreTransactionalContext; + var unifiedTransactionalContext = storageSession.txnManager.UnifiedStoreTransactionalContext; try { @@ -641,8 +642,8 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp if (isSuccessful && listObj.LnkList.Count == 0) { - _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectTransactionalContext); + _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, ExpireOption.None, + ref unifiedTransactionalContext); } return isSuccessful; @@ -655,8 +656,8 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp if (isSuccessful && sortedSetObj.Count() == 0) { - _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectTransactionalContext); + _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, ExpireOption.None, + ref unifiedTransactionalContext); } return isSuccessful; diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 763e0403468..d75c1d8e04e 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -425,15 +425,19 @@ public GarnetStatus SET(PinnedSpanByte key, Memory value, ref TC return GarnetStatus.OK; } - public GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType, ref TContext context, ref TObjectContext objectContext) + public GarnetStatus SET_Main(in TSourceLogRecord srcLogRecord, ref TContext context) where TContext : ITsavoriteContext + where TSourceLogRecord : ISourceLogRecord + { + context.Upsert(in srcLogRecord); + return GarnetStatus.OK; + } + + public GarnetStatus SET_Object(in TSourceLogRecord srcLogRecord, ref TObjectContext objectContext) where TObjectContext : ITsavoriteContext where TSourceLogRecord : ISourceLogRecord { - if (storeType == StoreType.Main) - context.Upsert(in srcLogRecord); - else - objectContext.Upsert(in srcLogRecord); + objectContext.Upsert(in srcLogRecord); return GarnetStatus.OK; } @@ -569,7 +573,7 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne // Find expiration time of the old key var expireSpan = new SpanByteAndMemory(); var ttlStatus = TTL(oldKey, storeType, ref expireSpan, ref context, ref objectContext, true); - + if (ttlStatus == GarnetStatus.OK && !expireSpan.IsSpanByte) { var newValSlice = PinnedSpanByte.FromPinnedPointer(ptrVal, headerLength); @@ -709,233 +713,6 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne return returnStatus; } - /// - /// Returns if key is an existing one in the store. - /// - /// - /// - /// The name of the key to use in the operation - /// The store to operate on. - /// Basic context for the main store. - /// Object context for the object store. - /// - public GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - var status = GarnetStatus.NOTFOUND; - RawStringInput input = default; - - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var _output = new SpanByteAndMemory { SpanByte = scratchBufferBuilder.ViewRemainingArgSlice() }; - status = GET(key, ref input, ref _output, ref context); - - if (status == GarnetStatus.OK) - { - if (!_output.IsSpanByte) - _output.Memory.Dispose(); - return status; - } - } - - if ((storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) - { - status = GET(key, out _, ref objectContext); - } - - return status; - } - - /// - /// Set a timeout on key - /// - /// - /// - /// The key to set the timeout on. - /// Milliseconds value for the timeout. - /// True when the timeout was properly set. - /// The store to operate on. - /// >Flags to use for the operation. - /// Basic context for the main store. - /// Object context for the object store. - /// - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - => EXPIRE(key, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ToPointer())), out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext); - - /// - /// Set a timeout on key. - /// - /// - /// - /// The key to set the timeout on. - /// Input for the main store - /// True when the timeout was properly set. - /// The store to operate on. - /// Basic context for the main store - /// Object context for the object store - /// - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - Span rmwOutput = stackalloc byte[OutputHeader.Size]; - var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); - timeoutSet = false; - - var found = false; - - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var status = context.RMW(key.ReadOnlySpan, ref input, ref output); - - if (status.IsPending) - CompletePendingForSession(ref status, ref output, ref context); - if (status.Found) found = true; - } - - if (!found && (storeType == StoreType.Object || storeType == StoreType.All) && - !objectStoreBasicContext.IsNull) - { - var header = new RespInputHeader(GarnetObjectType.Expire); - - // Re-encode expiration and expiration option as two integers instead of a long - var expirationWithOption = new ExpirationWithOption(input.arg1); - - var objInput = new ObjectInput(header, arg1: expirationWithOption.WordHead, arg2: expirationWithOption.WordTail); - - // Retry on object store - var objOutput = new GarnetObjectStoreOutput(output); - var status = objectStoreContext.RMW(key.ReadOnlySpan, ref objInput, ref objOutput); - - if (status.IsPending) - CompletePendingForObjectStoreSession(ref status, ref objOutput, ref objectStoreContext); - if (status.Found) found = true; - - output = objOutput.SpanByteAndMemory; - } - - Debug.Assert(output.IsSpanByte); - if (found) timeoutSet = ((OutputHeader*)output.SpanByte.ToPointer())->result1 == 1; - - return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; - } - - - - /// - /// Set a timeout on key using absolute Unix timestamp (seconds since January 1, 1970). - /// - /// - /// - /// The key to set the timeout on. - /// Absolute Unix timestamp - /// True when the timeout was properly set. - /// The store to operate on. - /// Flags to use for the operation. - /// Basic context for the main store - /// Object context for the object store - /// When true, is treated as milliseconds else seconds - /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND - public unsafe GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - return EXPIRE(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext, milliseconds ? RespCommand.PEXPIREAT : RespCommand.EXPIREAT); - } - - /// - /// Set a timeout on key. - /// - /// - /// - /// The key to set the timeout on. - /// The timespan value to set the expiration for. - /// True when the timeout was properly set. - /// The store to operate on. - /// Flags to use for the operation. - /// Basic context for the main store - /// Object context for the object store - /// When true the command executed is PEXPIRE, expire by default. - /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - return EXPIRE(key, (long)(milliseconds ? expiry.TotalMilliseconds : expiry.TotalSeconds), out timeoutSet, storeType, expireOption, - ref context, ref objectStoreContext, milliseconds ? RespCommand.PEXPIRE : RespCommand.EXPIRE); - } - - /// - /// Set a timeout on key. - /// - /// - /// - /// The key to set the timeout on. - /// The timespan value to set the expiration for. - /// True when the timeout was properly set. - /// The store to operate on. - /// Flags to use for the operation. - /// Basic context for the main store - /// Object context for the object store - /// The current RESP command - /// - public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, long expiration, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, RespCommand respCommand) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - Span rmwOutput = stackalloc byte[OutputHeader.Size]; - var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); - timeoutSet = false; - var found = false; - - // Convert to expiration time in ticks - var expirationTimeInTicks = respCommand switch - { - RespCommand.EXPIRE => DateTimeOffset.UtcNow.AddSeconds(expiration).UtcTicks, - RespCommand.PEXPIRE => DateTimeOffset.UtcNow.AddMilliseconds(expiration).UtcTicks, - RespCommand.EXPIREAT => ConvertUtils.UnixTimestampInSecondsToTicks(expiration), - _ => ConvertUtils.UnixTimestampInMillisecondsToTicks(expiration) - }; - - var expirationWithOption = new ExpirationWithOption(expirationTimeInTicks, expireOption); - - if (storeType == StoreType.Main || storeType == StoreType.All) - { - var input = new RawStringInput(RespCommand.EXPIRE, arg1: expirationWithOption.Word); - var status = context.RMW(key.ReadOnlySpan, ref input, ref output); - - if (status.IsPending) - CompletePendingForSession(ref status, ref output, ref context); - if (status.Found) found = true; - } - - if (!found && (storeType == StoreType.Object || storeType == StoreType.All) && - !objectStoreBasicContext.IsNull) - { - var header = new RespInputHeader(GarnetObjectType.Expire); - var objInput = new ObjectInput(header, arg1: expirationWithOption.WordHead, arg2: expirationWithOption.WordTail); - - // Retry on object store - var objOutput = new GarnetObjectStoreOutput(output); - var keyBytes = key.ToArray(); - var status = objectStoreContext.RMW(key.ReadOnlySpan, ref objInput, ref objOutput); - - if (status.IsPending) - CompletePendingForObjectStoreSession(ref status, ref objOutput, ref objectStoreContext); - if (status.Found) found = true; - - output = objOutput.SpanByteAndMemory; - } - - Debug.Assert(output.IsSpanByte); - if (found) timeoutSet = ((OutputHeader*)output.SpanByte.ToPointer())->result1 == 1; - - return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; - } - /// /// For existing keys - overwrites part of the value at a specified offset (in-place if possible) /// For non-existing keys - creates a new string with the value at a specified offset (padded with '\0's) diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index 98257cae3d3..4d526c73846 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -223,7 +223,8 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio _ = txnManager.Run(true); } - var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; + var unifiedContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -246,7 +247,7 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio if (!sameKey) { // Read destination key - statusOp = GET(destinationKey, out var destinationList, ref objectStoreTransactionalContext); + statusOp = GET(destinationKey, out var destinationList, ref objectContext); if (statusOp == GarnetStatus.NOTFOUND) { @@ -287,8 +288,7 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio { if (srcListObject.LnkList.Count == 0) { - _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectTransactionalContext); + _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, ExpireOption.None, ref unifiedContext); } // Left push (addfirst) to destination @@ -301,7 +301,7 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio newListValue = new ListObject(dstListObject.LnkList, dstListObject.sizes); // Upsert - _ = SET(destinationKey, newListValue, ref objectStoreTransactionalContext); + _ = SET(destinationKey, newListValue, ref objectContext); } else { diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index 05b8ded1843..3e54b60e8b4 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -273,6 +273,7 @@ internal unsafe GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte de } var objectTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var unifiedTransactionalContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -313,8 +314,7 @@ internal unsafe GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte de if (srcSetObject.Set.Count == 0) { - _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectTransactionalContext); + _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, ExpireOption.None, ref unifiedTransactionalContext); } _ = dstSetObject.Set.Add(arrMember); @@ -411,6 +411,7 @@ public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, // SetObject var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var setUnifiedStoreTransactionalContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -431,8 +432,7 @@ public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, } else { - _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref setObjectStoreTransactionalContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, ExpireOption.None, ref setUnifiedStoreTransactionalContext); } count = members.Count; @@ -580,6 +580,7 @@ public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out // SetObject var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var setUnifiedStoreTransactionalContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -600,8 +601,7 @@ public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out } else { - _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref setObjectStoreTransactionalContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, ExpireOption.None, ref setUnifiedStoreTransactionalContext); } count = members.Count; @@ -840,6 +840,7 @@ public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out // SetObject var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var setUnifiedStoreTransactionalContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -859,8 +860,7 @@ public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out } else { - _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref setObjectStoreTransactionalContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, ExpireOption.None, ref setUnifiedStoreTransactionalContext); } count = diffSet.Count; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index f631e1816e7..82d18aa7814 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs @@ -130,7 +130,8 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Shared); _ = txnManager.Run(true); } - var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var geoObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var geoUnifiedStoreTransactionalContext = txnManager.UnifiedStoreTransactionalContext; using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output); @@ -138,7 +139,7 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi { SpanByteAndMemory searchOutMem = default; - var status = GET(key, out var firstObj, ref objectStoreTransactionalContext); + var status = GET(key, out var firstObj, ref geoObjectStoreTransactionalContext); if (status == GarnetStatus.OK) { if (firstObj.GarnetObject is SortedSetObject firstSortedSet) @@ -155,7 +156,7 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi if (status == GarnetStatus.NOTFOUND) { // Expire/Delete the destination key if the source key is not found - _ = EXPIRE(destination, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, ref transactionalContext, ref objectStoreTransactionalContext); + _ = EXPIRE(destination, TimeSpan.Zero, out _, ExpireOption.None, ref geoUnifiedStoreTransactionalContext); writer.WriteInt32(0); return GarnetStatus.OK; } @@ -175,7 +176,7 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi return GarnetStatus.OK; } - _ = objectStoreTransactionalContext.Delete(destination.ReadOnlySpan); + _ = geoObjectStoreTransactionalContext.Delete(destination.ReadOnlySpan); _ = RespReadUtils.TryReadUnsignedArrayLength(out var foundItems, ref currOutPtr, endOutPtr); @@ -201,7 +202,7 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi }, ref parseState); var zAddOutput = new GarnetObjectStoreOutput(); - RMWObjectStoreOperationWithOutput(destination, ref zAddInput, ref objectStoreTransactionalContext, ref zAddOutput); + RMWObjectStoreOperationWithOutput(destination, ref zAddInput, ref geoObjectStoreTransactionalContext, ref zAddOutput); writer.WriteInt32(foundItems); } diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index bd3fec1d58a..0ebcc5c3e61 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -585,6 +585,7 @@ public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, Read } var objectContext = txnManager.ObjectStoreTransactionalContext; + var unifiedContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -609,8 +610,7 @@ public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, Read } else { - _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectContext); + _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, ExpireOption.None, ref unifiedContext); } return status; @@ -718,13 +718,14 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds } // SetObject - var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var ssObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; + var ssUnifiedStoreTransactionalContext = txnManager.UnifiedStoreTransactionalContext; try { SpanByteAndMemory rangeOutputMem = default; var rangeOutput = new GarnetObjectStoreOutput(rangeOutputMem); - var status = SortedSetRange(srcKey, ref input, ref rangeOutput, ref objectStoreTransactionalContext); + var status = SortedSetRange(srcKey, ref input, ref rangeOutput, ref ssObjectStoreTransactionalContext); rangeOutputMem = rangeOutput.SpanByteAndMemory; if (status == GarnetStatus.WRONGTYPE) @@ -733,7 +734,7 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds if (status == GarnetStatus.NOTFOUND) { // Expire/Delete the destination key if the source key is not found - _ = EXPIRE(dstKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, ref transactionalContext, ref objectStoreTransactionalContext); + _ = EXPIRE(dstKey, TimeSpan.Zero, out _, ExpireOption.None, ref ssUnifiedStoreTransactionalContext); return GarnetStatus.OK; } @@ -747,7 +748,7 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds var endOutPtr = rangeOutPtr + rangeOutputMem.Length; var destinationKey = dstKey.ReadOnlySpan; - objectStoreTransactionalContext.Delete(destinationKey); + ssObjectStoreTransactionalContext.Delete(destinationKey); RespReadUtils.TryReadUnsignedArrayLength(out var arrayLen, ref currOutPtr, endOutPtr); Debug.Assert(arrayLen % 2 == 0, "Should always contain element and its score"); @@ -772,7 +773,7 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds }, ref parseState); var zAddOutput = new GarnetObjectStoreOutput(); - RMWObjectStoreOperationWithOutput(destinationKey, ref zAddInput, ref objectStoreTransactionalContext, ref zAddOutput); + RMWObjectStoreOperationWithOutput(destinationKey, ref zAddInput, ref ssObjectStoreTransactionalContext, ref zAddOutput); itemBroker.HandleCollectionUpdate(destinationKey.ToArray()); } } @@ -1085,6 +1086,7 @@ public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlyS } var objectContext = txnManager.ObjectStoreTransactionalContext; + var unifiedContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -1110,8 +1112,7 @@ public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlyS } else { - _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectContext); + _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, ExpireOption.None, ref unifiedContext); } return status; @@ -1341,6 +1342,7 @@ public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadO } var objectContext = txnManager.ObjectStoreTransactionalContext; + var unifiedContext = txnManager.UnifiedStoreTransactionalContext; try { @@ -1366,8 +1368,7 @@ public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadO } else { - _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref transactionalContext, ref objectContext); + _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, ExpireOption.None, ref unifiedContext); } return status; diff --git a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs index e6b81340bb4..588e3651872 100644 --- a/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs +++ b/libs/server/Storage/Session/UnifiedStore/UnifiedStoreOps.cs @@ -3,6 +3,7 @@ using System; using System.Diagnostics; +using Garnet.common; using Tsavorite.core; namespace Garnet.server @@ -46,5 +47,93 @@ public GarnetStatus DELETE(PinnedSpanByte key, ref TUnifiedCont Debug.Assert(!status.IsPending); return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } + + /// + /// Set a timeout on key + /// + /// + /// The key to set the timeout on. + /// Milliseconds value for the timeout. + /// True when the timeout was properly set. + /// >Flags to use for the operation. + /// Basic context for the unified store. + /// + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, ExpireOption expireOption, ref TUnifiedContext unifiedContext) + where TUnifiedContext : ITsavoriteContext + => EXPIRE(key, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ToPointer())), out timeoutSet, expireOption, ref unifiedContext); + + /// + /// Set a timeout on key using absolute Unix timestamp (seconds since January 1, 1970). + /// + /// + /// The key to set the timeout on. + /// Absolute Unix timestamp + /// True when the timeout was properly set. + /// Flags to use for the operation. + /// Basic context for the unified store. + /// When true, is treated as milliseconds else seconds + /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND + public unsafe GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption, ref TUnifiedContext unifiedContext, bool milliseconds = false) + where TUnifiedContext : ITsavoriteContext + { + return EXPIRE(key, expiryTimestamp, out timeoutSet, expireOption, ref unifiedContext, milliseconds ? RespCommand.PEXPIREAT : RespCommand.EXPIREAT); + } + + /// + /// Set a timeout on key. + /// + /// + /// The key to set the timeout on. + /// The timespan value to set the expiration for. + /// True when the timeout was properly set. + /// Flags to use for the operation. + /// Basic context for the unified store. + /// When true the command executed is PEXPIRE, expire by default. + /// Return GarnetStatus.OK when key found, else GarnetStatus.NOTFOUND + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, ExpireOption expireOption, ref TUnifiedContext unifiedContext, bool milliseconds = false) + where TUnifiedContext : ITsavoriteContext + { + return EXPIRE(key, (long)(milliseconds ? expiry.TotalMilliseconds : expiry.TotalSeconds), out timeoutSet, expireOption, + ref unifiedContext, milliseconds ? RespCommand.PEXPIRE : RespCommand.EXPIRE); + } + + /// + /// Set a timeout on key. + /// + /// + /// The key to set the timeout on. + /// The timespan value to set the expiration for. + /// True when the timeout was properly set. + /// Flags to use for the operation. + /// Basic context for the main store + /// The current RESP command + /// + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, long expiration, out bool timeoutSet, ExpireOption expireOption, ref TUnifiedContext unifiedContext, RespCommand respCommand) + where TUnifiedContext : ITsavoriteContext + { + Span rmwOutput = stackalloc byte[OutputHeader.Size]; + var unifiedOutput = new GarnetUnifiedStoreOutput(SpanByteAndMemory.FromPinnedSpan(rmwOutput)); + + // Convert to expiration time in ticks + var expirationTimeInTicks = respCommand switch + { + RespCommand.EXPIRE => DateTimeOffset.UtcNow.AddSeconds(expiration).UtcTicks, + RespCommand.PEXPIRE => DateTimeOffset.UtcNow.AddMilliseconds(expiration).UtcTicks, + RespCommand.EXPIREAT => ConvertUtils.UnixTimestampInSecondsToTicks(expiration), + _ => ConvertUtils.UnixTimestampInMillisecondsToTicks(expiration) + }; + + var expirationWithOption = new ExpirationWithOption(expirationTimeInTicks, expireOption); + + var input = new UnifiedStoreInput(RespCommand.EXPIRE, arg1: expirationWithOption.Word); + var status = unifiedContext.RMW(key.ReadOnlySpan, ref input, ref unifiedOutput); + + if (status.IsPending) + CompletePendingForUnifiedStoreSession(ref status, ref unifiedOutput, ref unifiedContext); + + timeoutSet = status.Found && ((OutputHeader*)unifiedOutput.SpanByteAndMemory.SpanByte.ToPointer())->result1 == 1; + + return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; + } } } \ No newline at end of file diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 0a151a765b4..829473e8733 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -307,6 +307,8 @@ internal void Watch(PinnedSpanByte key, StoreType type) objectStoreBasicContext.ResetModified(key.ReadOnlySpan); break; case StoreType.All: + basicContext.ResetModified(key.ReadOnlySpan); + objectStoreBasicContext.ResetModified(key.ReadOnlySpan); unifiedStoreBasicContext.ResetModified(key.ReadOnlySpan); break; } @@ -322,6 +324,10 @@ void UpdateTransactionStoreType(StoreType type) case StoreType.Object: isObjectTxn = true; break; + case StoreType.All: + isMainTxn = true; + isObjectTxn = true; + break; } } diff --git a/test/Garnet.test/Extensions/RateLimiterTxn.cs b/test/Garnet.test/Extensions/RateLimiterTxn.cs index ce45bda1391..757fc4a84d9 100644 --- a/test/Garnet.test/Extensions/RateLimiterTxn.cs +++ b/test/Garnet.test/Extensions/RateLimiterTxn.cs @@ -54,7 +54,7 @@ public override unsafe void Main(TGarnetApi api, ref CustomProcedure fixed (byte* timeInMicroSecondBytesPtr = timeInMicroSecondBytes) { api.SortedSetAdd(key, PinnedSpanByte.FromPinnedPointer(unixTimeInMilliSecondPtr, unixTimeInMilliSecondBytes.Length), PinnedSpanByte.FromPinnedPointer(timeInMicroSecondBytesPtr, timeInMicroSecondBytes.Length), out var _); - api.EXPIRE(key, TimeSpan.FromMilliseconds(slidingWindowInMilliSeconds), out var _, StoreType.Object); + api.EXPIRE(key, TimeSpan.FromMilliseconds(slidingWindowInMilliSeconds), out _); } } diff --git a/test/Garnet.test/ObjectExpiryTxn.cs b/test/Garnet.test/ObjectExpiryTxn.cs index 9b7aed88d58..7a69716028f 100644 --- a/test/Garnet.test/ObjectExpiryTxn.cs +++ b/test/Garnet.test/ObjectExpiryTxn.cs @@ -29,7 +29,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p var key = GetNextArg(ref procInput.parseState, ref offset); var expiryMs = GetNextArg(ref procInput.parseState, ref offset); - api.EXPIRE(key, expiryMs, out _, StoreType.Object); + api.EXPIRE(key, expiryMs, out _); WriteSimpleString(ref output, "SUCCESS"); } } From f3708a57b277158eadc642c427e2cfea9f2514e6 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Fri, 26 Sep 2025 12:52:05 -0700 Subject: [PATCH 26/28] Removed some unnecessary stuff --- libs/server/Objects/Types/GarnetObject.cs | 2 - libs/server/Objects/Types/GarnetObjectType.cs | 25 ----------- .../Functions/ObjectStore/RMWMethods.cs | 41 ------------------- .../Functions/ObjectStore/ReadMethods.cs | 9 ---- .../ObjectStore/VarLenInputMethods.cs | 17 +------- 5 files changed, 1 insertion(+), 93 deletions(-) diff --git a/libs/server/Objects/Types/GarnetObject.cs b/libs/server/Objects/Types/GarnetObject.cs index b9fc4abf680..c99f91aa957 100644 --- a/libs/server/Objects/Types/GarnetObject.cs +++ b/libs/server/Objects/Types/GarnetObject.cs @@ -74,8 +74,6 @@ internal static bool NeedToCreate(RespInputHeader header) HashOperation.HCOLLECT => false, _ => true, }, - GarnetObjectType.Expire => false, - GarnetObjectType.Persist => false, GarnetObjectType.Migrate => false, _ => true, }; diff --git a/libs/server/Objects/Types/GarnetObjectType.cs b/libs/server/Objects/Types/GarnetObjectType.cs index 2960e75a5d3..df5d77bcb4a 100644 --- a/libs/server/Objects/Types/GarnetObjectType.cs +++ b/libs/server/Objects/Types/GarnetObjectType.cs @@ -43,21 +43,6 @@ public enum GarnetObjectType : byte /// DelIfExpIm = 0xf7, - /// - /// Special type indicating PEXPIRE command - /// - PExpire = 0xf8, - - /// - /// Special type indicating EXPIRETIME command - /// - ExpireTime = 0xf9, - - /// - /// Special type indicating PEXPIRETIME command - /// - PExpireTime = 0xfa, - /// /// Indicating a Custom Object command /// @@ -68,20 +53,10 @@ public enum GarnetObjectType : byte /// PTtl = 0xfc, - /// - /// Special type indicating PERSIST command - /// - Persist = 0xfd, - /// /// Special type indicating TTL command /// Ttl = 0xfe, - - /// - /// Special type indicating EXPIRE command - /// - Expire = 0xff, } public static class GarnetObjectTypeExtensions diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index dd2b71a7a81..aeebca56df0 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -21,8 +21,6 @@ public bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref switch (type) { - case GarnetObjectType.Expire: - case GarnetObjectType.Persist: case GarnetObjectType.DelIfExpIm: return false; default: @@ -60,10 +58,6 @@ public bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, _ = logRecord.TrySetValueObject(value, in sizeInfo); return true; } - else - Debug.Assert(type != GarnetObjectType.Expire && type != GarnetObjectType.Persist, "Expire and Persist commands should have been handled already by NeedInitialUpdate."); - - Debug.Assert(type is not GarnetObjectType.Expire and not GarnetObjectType.PExpire and not GarnetObjectType.Persist, "Expire and Persist commands should have returned false from NeedInitialUpdate."); var customObjectCommand = GetCustomObjectCommand(ref input, type); value = functionsState.GetCustomObjectFactory((byte)type).Create((byte)type); @@ -135,20 +129,6 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r switch (input.header.type) { - case GarnetObjectType.Expire: - var expirationWithOption = new ExpirationWithOption(input.arg1, input.arg2); - if (!EvaluateObjectExpireInPlace(ref logRecord, expirationWithOption.ExpireOption, expirationWithOption.ExpirationTimeInTicks, ref output)) - return false; - return true; // The options may or may not produce a result that matches up with what sizeInfo has, so return rather than drop down to AssertOptionals - case GarnetObjectType.Persist: - if (logRecord.Info.HasExpiration) - { - logRecord.RemoveExpiration(); - functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); - } - else - functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); - return true; case GarnetObjectType.DelIfExpIm: return true; default: @@ -243,27 +223,6 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, switch (input.header.type) { - case GarnetObjectType.Expire: - var expirationWithOption = new ExpirationWithOption(input.arg1, input.arg2); - - // Expire will have allocated space for the expiration, so copy it over and do the "in-place" logic to replace it in the new record - if (srcLogRecord.Info.HasExpiration) - dstLogRecord.TrySetExpiration(srcLogRecord.Expiration); - if (!EvaluateObjectExpireInPlace(ref dstLogRecord, expirationWithOption.ExpireOption, expirationWithOption.ExpirationTimeInTicks, ref output)) - return false; - break; - - case GarnetObjectType.Persist: - if (!dstLogRecord.TryCopyFrom(in srcLogRecord, in sizeInfo)) - return false; - if (srcLogRecord.Info.HasExpiration) - { - dstLogRecord.RemoveExpiration(); - functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); - } - else - functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); - break; case GarnetObjectType.DelIfExpIm: break; default: diff --git a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs index 4ed1fca6c85..6e03893779e 100644 --- a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs @@ -40,15 +40,6 @@ public bool Reader(in TSourceLogRecord srcLogRecord, ref Objec functionsState.CopyRespNumber(ttlValue, ref output.SpanByteAndMemory); return true; - case GarnetObjectType.ExpireTime: - var expireTime = ConvertUtils.UnixTimeInSecondsFromTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); - functionsState.CopyRespNumber(expireTime, ref output.SpanByteAndMemory); - return true; - case GarnetObjectType.PExpireTime: - expireTime = ConvertUtils.UnixTimeInMillisecondsFromTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); - functionsState.CopyRespNumber(expireTime, ref output.SpanByteAndMemory); - return true; - default: if ((byte)input.header.type < CustomCommandManager.CustomTypeIdStartOffset) { diff --git a/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs index 719782e4364..e25362d6e5e 100644 --- a/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs @@ -28,7 +28,7 @@ public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref Object public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref ObjectInput input) where TSourceLogRecord : ISourceLogRecord { - var fieldInfo = new RecordFieldInfo() + return new RecordFieldInfo() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, @@ -36,21 +36,6 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe HasETag = false, // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag(), HasExpiration = srcLogRecord.Info.HasExpiration }; - - switch (input.header.type) - { - case GarnetObjectType.Expire: - case GarnetObjectType.PExpire: - fieldInfo.HasExpiration = true; - return fieldInfo; - - case GarnetObjectType.Persist: - fieldInfo.HasExpiration = false; - return fieldInfo; - - default: - return fieldInfo; - } } public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref ObjectInput input) From 6a3acd01aef44c9222030a997bd72e5a82c63792 Mon Sep 17 00:00:00 2001 From: Tal Zaccai Date: Mon, 29 Sep 2025 12:07:11 -0700 Subject: [PATCH 27/28] Some more cleanup to TransactionManager --- .../Custom/CustomTransactionProcedure.cs | 3 +- .../ItemBroker/CollectionItemBroker.cs | 6 +- .../Storage/Session/MainStore/BitmapOps.cs | 5 +- .../Session/MainStore/HyperLogLogOps.cs | 10 ++- .../Storage/Session/MainStore/MainStoreOps.cs | 24 ++++-- .../Storage/Session/ObjectStore/ListOps.cs | 5 +- .../Storage/Session/ObjectStore/SetOps.cs | 32 +++++--- .../Session/ObjectStore/SortedSetGeoOps.cs | 8 +- .../Session/ObjectStore/SortedSetOps.cs | 32 +++++--- libs/server/Transaction/TransactionManager.cs | 82 +++++++++---------- libs/server/Transaction/TxnKeyManager.cs | 8 +- libs/server/Transaction/TxnRespCommands.cs | 4 +- .../Transaction/TxnWatchedKeysContainer.cs | 12 +-- libs/server/Transaction/WatchedKeySlice.cs | 2 +- 14 files changed, 131 insertions(+), 102 deletions(-) diff --git a/libs/server/Custom/CustomTransactionProcedure.cs b/libs/server/Custom/CustomTransactionProcedure.cs index f0eb426a9fe..13bf8037ba3 100644 --- a/libs/server/Custom/CustomTransactionProcedure.cs +++ b/libs/server/Custom/CustomTransactionProcedure.cs @@ -39,7 +39,8 @@ public abstract class CustomTransactionProcedure : CustomProcedureBase /// protected void AddKey(PinnedSpanByte key, LockType type, StoreType storeType) { - txnManager.SaveKeyEntryToLock(key, storeType, type); + txnManager.AddTransactionStoreType(storeType); + txnManager.SaveKeyEntryToLock(key, type); txnManager.VerifyKeyOwnership(key, type); } diff --git a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs index b130cf907c2..78a8f0011e7 100644 --- a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs +++ b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs @@ -529,15 +529,15 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp { Debug.Assert(storageSession.txnManager.state == TxnState.None); createTransaction = true; - storageSession.txnManager.SaveKeyEntryToLock(asKey, StoreType.Object, LockType.Exclusive); + storageSession.txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + storageSession.txnManager.SaveKeyEntryToLock(asKey, LockType.Exclusive); if (command == RespCommand.BLMOVE) - storageSession.txnManager.SaveKeyEntryToLock(dstKey, StoreType.Object, LockType.Exclusive); + storageSession.txnManager.SaveKeyEntryToLock(dstKey, LockType.Exclusive); _ = storageSession.txnManager.Run(true); } - var transactionalContext = storageSession.txnManager.TransactionalContext; var objectTransactionalContext = storageSession.txnManager.ObjectStoreTransactionalContext; var unifiedTransactionalContext = storageSession.txnManager.UnifiedStoreTransactionalContext; diff --git a/libs/server/Storage/Session/MainStore/BitmapOps.cs b/libs/server/Storage/Session/MainStore/BitmapOps.cs index 286cad20a8e..8dc5f86c6f7 100644 --- a/libs/server/Storage/Session/MainStore/BitmapOps.cs +++ b/libs/server/Storage/Session/MainStore/BitmapOps.cs @@ -88,9 +88,10 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp { createTransaction = true; Debug.Assert(txnManager.state == TxnState.None); - txnManager.SaveKeyEntryToLock(keys[0], StoreType.Main, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main); + txnManager.SaveKeyEntryToLock(keys[0], LockType.Exclusive); for (var i = 1; i < keys.Length; i++) - txnManager.SaveKeyEntryToLock(keys[i], StoreType.Main, LockType.Shared); + txnManager.SaveKeyEntryToLock(keys[i], LockType.Shared); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs index 3c195df29cb..27d1eb6d11c 100644 --- a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs +++ b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs @@ -102,11 +102,12 @@ public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; var dstKey = input.parseState.GetArgSliceByRef(0); - txnManager.SaveKeyEntryToLock(dstKey, StoreType.Main, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main); + txnManager.SaveKeyEntryToLock(dstKey, LockType.Exclusive); for (var i = 1; i < input.parseState.Count; i++) { var currSrcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(currSrcKey, StoreType.Main, LockType.Shared); + txnManager.SaveKeyEntryToLock(currSrcKey, LockType.Shared); } _ = txnManager.Run(true); } @@ -194,12 +195,13 @@ public unsafe GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool e { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main); var dstKey = input.parseState.GetArgSliceByRef(0); - txnManager.SaveKeyEntryToLock(dstKey, StoreType.Main, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(dstKey, LockType.Exclusive); for (var i = 1; i < input.parseState.Count; i++) { var currSrcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(currSrcKey, StoreType.Main, LockType.Shared); + txnManager.SaveKeyEntryToLock(currSrcKey, LockType.Shared); } _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index d75c1d8e04e..97c6711fc88 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -370,7 +370,8 @@ internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref T for (var i = 0; i < count; i += 2) { var srcKey = input.parseState.GetArgSliceByRef(i); - txnManager.SaveKeyEntryToLock(srcKey, StoreType.All, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(srcKey, LockType.Exclusive); } txnManager.Run(true); } @@ -541,8 +542,9 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(oldKeySlice, StoreType.Main, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(newKeySlice, StoreType.Main, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main | TransactionStoreTypes.Object); + txnManager.SaveKeyEntryToLock(oldKeySlice, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(newKeySlice, LockType.Exclusive); _ = txnManager.Run(true); } @@ -665,8 +667,9 @@ private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte ne createTransaction = false; if (txnManager.state != TxnState.Running) { - txnManager.SaveKeyEntryToLock(oldKeySlice, StoreType.Object, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(newKeySlice, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); + txnManager.SaveKeyEntryToLock(oldKeySlice, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(newKeySlice, LockType.Exclusive); txnManager.Run(true); createTransaction = true; } @@ -778,7 +781,11 @@ public unsafe GarnetStatus Increment(PinnedSpanByte key, out long outp return GarnetStatus.OK; } - public void WATCH(PinnedSpanByte key, StoreType type) => txnManager.Watch(key, type); + public void WATCH(PinnedSpanByte key, StoreType type) + { + txnManager.AddTransactionStoreType(type); + txnManager.Watch(key); + } public unsafe GarnetStatus SCAN(long cursor, PinnedSpanByte match, long count, ref TContext context) => GarnetStatus.OK; @@ -798,8 +805,9 @@ public unsafe GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref Spa var createTransaction = false; if (txnManager.state != TxnState.Running) { - txnManager.SaveKeyEntryToLock(key1, StoreType.Main, LockType.Shared); - txnManager.SaveKeyEntryToLock(key2, StoreType.Main, LockType.Shared); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Main); + txnManager.SaveKeyEntryToLock(key1, LockType.Shared); + txnManager.SaveKeyEntryToLock(key2, LockType.Shared); txnManager.Run(true); createTransaction = true; } diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index 4d526c73846..865c2122a99 100644 --- a/libs/server/Storage/Session/ObjectStore/ListOps.cs +++ b/libs/server/Storage/Session/ObjectStore/ListOps.cs @@ -218,8 +218,9 @@ public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinatio if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(sourceKey, StoreType.Object, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(sourceKey, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, LockType.Exclusive); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/ObjectStore/SetOps.cs b/libs/server/Storage/Session/ObjectStore/SetOps.cs index 3e54b60e8b4..7e1f52e0225 100644 --- a/libs/server/Storage/Session/ObjectStore/SetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SetOps.cs @@ -267,8 +267,9 @@ internal unsafe GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte de if (txnManager.state != TxnState.Running) { createTransaction = true; - txnManager.SaveKeyEntryToLock(sourceKey, StoreType.Object, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(sourceKey, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(destinationKey, LockType.Exclusive); _ = txnManager.Run(true); } @@ -361,8 +362,9 @@ public GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet outp { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -403,9 +405,10 @@ public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(key, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -532,8 +535,9 @@ public GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output) { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -572,9 +576,10 @@ public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(key, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -792,8 +797,9 @@ public GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet members) { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -832,9 +838,10 @@ public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(key, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -950,8 +957,9 @@ public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? l { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index 82d18aa7814..f38395a0bb9 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs @@ -69,7 +69,8 @@ public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref Ge { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Shared); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); + txnManager.SaveKeyEntryToLock(key, LockType.Shared); txnManager.Run(true); } @@ -126,8 +127,9 @@ public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, Pi { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destination, StoreType.Object, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Shared); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(destination, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, LockType.Shared); _ = txnManager.Run(true); } var geoObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index 0ebcc5c3e61..6262aab7bbf 100644 --- a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs +++ b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs @@ -530,8 +530,9 @@ public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); txnManager.Run(true); } @@ -578,9 +579,10 @@ public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, Read { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(destinationKey, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -712,8 +714,9 @@ public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte ds { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(dstKey, StoreType.Object, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(srcKey, StoreType.Object, LockType.Shared); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(dstKey, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(srcKey, LockType.Shared); _ = txnManager.Run(true); } @@ -1037,8 +1040,9 @@ public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] w { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); txnManager.Run(true); } @@ -1079,9 +1083,10 @@ public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlyS { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(destinationKey, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -1269,8 +1274,9 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var key in keys) - txnManager.SaveKeyEntryToLock(key, StoreType.Object, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, LockType.Exclusive); txnManager.Run(true); } @@ -1335,9 +1341,10 @@ public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadO { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destinationKey, StoreType.Object, LockType.Exclusive); + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified); + txnManager.SaveKeyEntryToLock(destinationKey, LockType.Exclusive); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); _ = txnManager.Run(true); } @@ -1396,8 +1403,9 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; + txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object); foreach (var item in keys) - txnManager.SaveKeyEntryToLock(item, StoreType.Object, LockType.Shared); + txnManager.SaveKeyEntryToLock(item, LockType.Shared); txnManager.Run(true); } diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 829473e8733..5240bd4a05d 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -31,6 +31,15 @@ namespace Garnet.server /* UnifiedStoreFunctions */ StoreFunctions, ObjectAllocator>>>; + [Flags] + public enum TransactionStoreTypes : byte + { + None = 0, + Main = 1, + Object = 1 << 1, + Unified = 1 << 2, + } + /// /// Transaction manager /// @@ -92,8 +101,7 @@ public sealed unsafe partial class TransactionManager private const int initialKeyBufferSize = 1 << 10; readonly ILogger logger; long txnVersion; - private bool isMainTxn = false; - private bool isObjectTxn = false; + private TransactionStoreTypes storeTypes; internal TransactionalContext TransactionalContext => transactionalContext; @@ -164,10 +172,10 @@ internal void Reset(bool isRunning) { keyEntries.UnlockAllKeys(); - // Release context - if (isMainTxn) + // Release contexts + if ((storeTypes & TransactionStoreTypes.Main) == TransactionStoreTypes.Main) transactionalContext.EndTransaction(); - if (isObjectTxn) + if ((storeTypes & TransactionStoreTypes.Object) == TransactionStoreTypes.Object) objectStoreTransactionalContext.EndTransaction(); unifiedStoreTransactionalContext.EndTransaction(); } @@ -180,8 +188,7 @@ internal void Reset(bool isRunning) this.txnStartHead = 0; this.operationCntTxn = 0; this.state = TxnState.None; - this.isMainTxn = false; - this.isObjectTxn = false; + this.storeTypes = TransactionStoreTypes.None; functionsState.StoredProcMode = false; // Reset cluster variables used for slot verification @@ -292,43 +299,34 @@ internal void Commit(bool internal_txn = false) Reset(true); } - internal void Watch(PinnedSpanByte key, StoreType type) + internal void Watch(PinnedSpanByte key) { - UpdateTransactionStoreType(type); - watchContainer.AddWatch(key, type); + watchContainer.AddWatch(key); // Release context - switch (type) - { - case StoreType.Main: - basicContext.ResetModified(key.ReadOnlySpan); - break; - case StoreType.Object: - objectStoreBasicContext.ResetModified(key.ReadOnlySpan); - break; - case StoreType.All: - basicContext.ResetModified(key.ReadOnlySpan); - objectStoreBasicContext.ResetModified(key.ReadOnlySpan); - unifiedStoreBasicContext.ResetModified(key.ReadOnlySpan); - break; - } + if ((storeTypes & TransactionStoreTypes.Main) == TransactionStoreTypes.Main) + transactionalContext.ResetModified(key.ReadOnlySpan); + if ((storeTypes & TransactionStoreTypes.Object) == TransactionStoreTypes.Object) + objectStoreTransactionalContext.ResetModified(key.ReadOnlySpan); + unifiedStoreTransactionalContext.ResetModified(key.ReadOnlySpan); + } + + internal void AddTransactionStoreTypes(TransactionStoreTypes transactionStoreTypes) + { + this.storeTypes |= transactionStoreTypes; } - void UpdateTransactionStoreType(StoreType type) + internal void AddTransactionStoreType(StoreType storeType) { - switch (type) + var transactionStoreTypes = storeType switch { - case StoreType.Main: - isMainTxn = true; - break; - case StoreType.Object: - isObjectTxn = true; - break; - case StoreType.All: - isMainTxn = true; - isObjectTxn = true; - break; - } + StoreType.Main => TransactionStoreTypes.Main, + StoreType.Object => TransactionStoreTypes.Object, + StoreType.All => TransactionStoreTypes.Unified, + _ => TransactionStoreTypes.None + }; + + this.storeTypes |= transactionStoreTypes; } internal string GetLockset() => keyEntries.GetLockset(); @@ -344,21 +342,19 @@ internal void GetKeysForValidation(byte* recvBufferPtr, out PinnedSpanByte[] key void BeginTransaction() { - if (isMainTxn) + if ((storeTypes & TransactionStoreTypes.Main) == TransactionStoreTypes.Main) transactionalContext.BeginTransaction(); - if (isObjectTxn) + if ((storeTypes & TransactionStoreTypes.Object) == TransactionStoreTypes.Object) objectStoreTransactionalContext.BeginTransaction(); - unifiedStoreTransactionalContext.BeginTransaction(); } void LocksAcquired(long txnVersion) { - if (isMainTxn) + if ((storeTypes & TransactionStoreTypes.Main) == TransactionStoreTypes.Main) transactionalContext.LocksAcquired(txnVersion); - if (isObjectTxn) + if ((storeTypes & TransactionStoreTypes.Object) == TransactionStoreTypes.Object) objectStoreTransactionalContext.LocksAcquired(txnVersion); - unifiedStoreTransactionalContext.LocksAcquired(txnVersion); } diff --git a/libs/server/Transaction/TxnKeyManager.cs b/libs/server/Transaction/TxnKeyManager.cs index 587549626f8..178f44f0948 100644 --- a/libs/server/Transaction/TxnKeyManager.cs +++ b/libs/server/Transaction/TxnKeyManager.cs @@ -12,11 +12,9 @@ sealed partial class TransactionManager /// Save key entry /// /// - /// /// - public void SaveKeyEntryToLock(PinnedSpanByte key, StoreType storeType, LockType type) + public void SaveKeyEntryToLock(PinnedSpanByte key, LockType type) { - UpdateTransactionStoreType(storeType); keyEntries.AddKey(key, type); } @@ -63,6 +61,8 @@ internal void LockKeys(SimpleRespCommandInfo cmdInfo) if (cmdInfo.KeySpecs == null || cmdInfo.KeySpecs.Length == 0) return; + AddTransactionStoreType(cmdInfo.StoreType); + foreach (var keySpec in cmdInfo.KeySpecs) { if (!respSession.parseState.TryGetKeySearchArgsFromSimpleKeySpec(keySpec, cmdInfo.IsSubCommand, out var searchArgs)) @@ -74,7 +74,7 @@ internal void LockKeys(SimpleRespCommandInfo cmdInfo) for (var currIdx = searchArgs.firstIdx; currIdx <= searchArgs.lastIdx; currIdx += searchArgs.step) { var key = respSession.parseState.GetArgSliceByRef(currIdx); - SaveKeyEntryToLock(key, cmdInfo.StoreType, lockType); + SaveKeyEntryToLock(key, lockType); SaveKeyArgSlice(key); } } diff --git a/libs/server/Transaction/TxnRespCommands.cs b/libs/server/Transaction/TxnRespCommands.cs index 9b74af99e45..6dd165336d3 100644 --- a/libs/server/Transaction/TxnRespCommands.cs +++ b/libs/server/Transaction/TxnRespCommands.cs @@ -223,9 +223,11 @@ private bool CommonWATCH(StoreType type) keys.Add(nextKey); } + txnManager.AddTransactionStoreType(type); + foreach (var toWatch in keys) { - txnManager.Watch(toWatch, type); + txnManager.Watch(toWatch); } while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) diff --git a/libs/server/Transaction/TxnWatchedKeysContainer.cs b/libs/server/Transaction/TxnWatchedKeysContainer.cs index 8edee6030a3..4d588c0e46f 100644 --- a/libs/server/Transaction/TxnWatchedKeysContainer.cs +++ b/libs/server/Transaction/TxnWatchedKeysContainer.cs @@ -55,14 +55,14 @@ public bool RemoveWatch(PinnedSpanByte key) { if (key.ReadOnlySpan.SequenceEqual(keySlices[i].slice.ReadOnlySpan)) { - keySlices[i].type = 0; + keySlices[i].isWatched = false; return true; } } return false; } - public void AddWatch(PinnedSpanByte key, StoreType type) + public void AddWatch(PinnedSpanByte key) { if (sliceCount >= sliceBufferSize) { @@ -96,7 +96,7 @@ public void AddWatch(PinnedSpanByte key, StoreType type) key.ReadOnlySpan.CopyTo(slice.Span); keySlices[sliceCount].slice = slice; - keySlices[sliceCount].type = type; + keySlices[sliceCount].isWatched = true; keySlices[sliceCount].hash = Utility.HashBytes(slice.ReadOnlySpan); keySlices[sliceCount].version = versionMap.ReadVersion(keySlices[sliceCount].hash); @@ -114,7 +114,7 @@ public bool ValidateWatchVersion() for (int i = 0; i < sliceCount; i++) { WatchedKeySlice key = keySlices[i]; - if (key.type == 0) continue; + if (!key.isWatched) continue; if (versionMap.ReadVersion(key.hash) != key.version) return false; } @@ -126,10 +126,10 @@ public bool SaveKeysToLock(TransactionManager txnManager) for (int i = 0; i < sliceCount; i++) { WatchedKeySlice watchedKeySlice = keySlices[i]; - if (watchedKeySlice.type == 0) continue; + if (!watchedKeySlice.isWatched) continue; var slice = keySlices[i].slice; - txnManager.SaveKeyEntryToLock(slice, watchedKeySlice.type, LockType.Shared); + txnManager.SaveKeyEntryToLock(slice, LockType.Shared); } return true; } diff --git a/libs/server/Transaction/WatchedKeySlice.cs b/libs/server/Transaction/WatchedKeySlice.cs index 42c393b2c96..640a191c4a1 100644 --- a/libs/server/Transaction/WatchedKeySlice.cs +++ b/libs/server/Transaction/WatchedKeySlice.cs @@ -19,6 +19,6 @@ struct WatchedKeySlice public long hash; [FieldOffset(28)] - public StoreType type; + public bool isWatched; } } \ No newline at end of file From bc9867a841a98b99bb4fc9992cc0abef7b176b40 Mon Sep 17 00:00:00 2001 From: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Date: Sun, 5 Oct 2025 08:05:57 -0700 Subject: [PATCH 28/28] merge tedhar/storage-v2 (ObjectAllocator serialization updates) --- .../ReplicationSnapshotIterator.cs | 4 +- .../Session/RespClusterMigrateCommands.cs | 109 +- .../Session/RespClusterReplicationCommands.cs | 54 +- libs/common/RespReadUtils.cs | 21 +- libs/server/Custom/CustomObjectBase.cs | 1 + libs/server/Databases/DatabaseManagerBase.cs | 2 +- libs/server/Objects/Hash/HashObject.cs | 57 +- libs/server/Objects/Hash/HashObjectImpl.cs | 17 +- libs/server/Objects/List/ListObject.cs | 20 +- libs/server/Objects/Set/SetObject.cs | 20 +- .../Objects/SortedSet/SortedSetObject.cs | 44 +- libs/server/Objects/Types/GarnetObjectBase.cs | 123 +- libs/server/Objects/Types/IGarnetObject.cs | 11 - .../Functions/MainStore/PrivateMethods.cs | 6 +- .../Storage/Functions/MainStore/RMWMethods.cs | 66 +- .../Functions/MainStore/ReadMethods.cs | 3 - .../Functions/MainStore/VarLenInputMethods.cs | 92 +- .../Functions/ObjectStore/DeleteMethods.cs | 4 +- .../Functions/ObjectStore/RMWMethods.cs | 28 +- .../Functions/ObjectStore/ReadMethods.cs | 2 +- .../Functions/ObjectStore/UpsertMethods.cs | 10 +- .../ObjectStore/VarLenInputMethods.cs | 20 +- .../Functions/UnifiedStore/DeleteMethods.cs | 2 +- .../Functions/UnifiedStore/RMWMethods.cs | 24 +- .../Functions/UnifiedStore/ReadMethods.cs | 10 +- .../Functions/UnifiedStore/UpsertMethods.cs | 10 +- .../UnifiedStore/VarLenInputMethods.cs | 26 +- .../Storage/Session/Common/MemoryUtils.cs | 2 +- .../Storage/Session/MainStore/MainStoreOps.cs | 2 +- .../Storage/SizeTracker/CacheSizeTracker.cs | 2 +- .../BDN-Tsavorite.Benchmark/InliningTests.cs | 2 +- .../BDN-Tsavorite.Benchmark/IterationTests.cs | 2 +- .../YCSB.benchmark/FixedLenYcsbBenchmark.cs | 22 +- .../YCSB.benchmark/FixedLengthKey.cs | 2 +- .../benchmark/YCSB.benchmark/KeySpanByte.cs | 27 +- .../benchmark/YCSB.benchmark/ObjectValue.cs | 19 +- .../YCSB.benchmark/ObjectYcsbBenchmark.cs | 4 +- .../cs/benchmark/YCSB.benchmark/Options.cs | 9 +- .../cs/benchmark/YCSB.benchmark/Program.cs | 26 +- .../SessionFixedLenFunctions.cs | 75 +- .../YCSB.benchmark/SpanByteYcsbBenchmark.cs | 43 +- .../cs/benchmark/YCSB.benchmark/TestLoader.cs | 23 +- .../cs/benchmark/YCSB.benchmark/TestStats.cs | 2 +- .../cs/src/core/Allocator/AllocatorBase.cs | 670 +++++------ .../cs/src/core/Allocator/AllocatorScan.cs | 9 +- .../cs/src/core/Allocator/AsyncIOContext.cs | 24 +- .../cs/src/core/Allocator/BlittableFrame.cs | 10 +- .../cs/src/core/Allocator/DiskLogRecord.cs | 975 +++++----------- .../cs/src/core/Allocator/HeapObjectBase.cs | 183 +++ .../cs/src/core/Allocator/IAllocator.cs | 23 +- .../src/core/Allocator/IAllocatorCallbacks.cs | 25 +- .../cs/src/core/Allocator/IHeapObject.cs | 48 +- .../cs/src/core/Allocator/ISourceLogRecord.cs | 42 +- .../cs/src/core/Allocator/LogField.cs | 294 ++--- .../cs/src/core/Allocator/LogRecord.cs | 1001 +++++++++++------ .../cs/src/core/Allocator/ObjectAllocator.cs | 44 +- .../src/core/Allocator/ObjectAllocatorImpl.cs | 929 +++++---------- .../cs/src/core/Allocator/ObjectIdMap.cs | 62 +- .../src/core/Allocator/ObjectScanIterator.cs | 418 +++++++ .../CircularDiskReadBuffer.cs | 208 ++++ .../CircularDiskWriteBuffer.cs | 274 +++++ .../ObjectSerialization/DiskReadBuffer.cs | 117 ++ .../ObjectSerialization/DiskWriteBuffer.cs | 126 +++ .../ObjectSerialization/IStreamBuffer.cs | 61 + .../ObjectLogFilePositionInfo.cs | 108 ++ .../ObjectSerialization/ObjectLogReader.cs | 230 ++++ .../ObjectSerialization/ObjectLogWriter.cs | 297 +++++ .../ObjectSerialization/PinnedMemoryStream.cs | 176 +++ .../VarbyteLengthUtility.cs | 361 ++++++ .../src/core/Allocator/OptionalFieldsShift.cs | 46 +- .../src/core/Allocator/OverflowByteArray.cs | 79 ++ .../cs/src/core/Allocator/PageHeader.cs | 73 ++ .../cs/src/core/Allocator/ScanIteratorBase.cs | 181 +-- .../src/core/Allocator/SerializationPhase.cs | 23 + .../src/core/Allocator/SpanByteAllocator.cs | 45 +- .../core/Allocator/SpanByteAllocatorImpl.cs | 136 +-- ...canIterator.cs => SpanByteScanIterator.cs} | 214 ++-- .../core/Allocator/TsavoriteLogAllocator.cs | 46 +- .../Allocator/TsavoriteLogAllocatorImpl.cs | 127 +-- .../src/core/ClientSession/ClientSession.cs | 6 +- .../ClientSession/ITransactionalContext.cs | 1 + .../ClientSession/NoOpSessionFunctions.cs | 4 +- .../ClientSession/SessionFunctionsWrapper.cs | 5 + .../Tsavorite/cs/src/core/Device/IDevice.cs | 16 +- .../cs/src/core/Device/LocalStorageDevice.cs | 7 + .../cs/src/core/Device/StorageDeviceBase.cs | 4 +- .../cs/src/core/Epochs/LightEpoch.cs | 33 +- .../HybridLogCheckpointSMTask.cs | 5 +- .../Checkpointing/SnapshotCheckpointSMTask.cs | 4 +- .../src/core/Index/Common/CompletedOutput.cs | 3 +- .../cs/src/core/Index/Common/LogAddress.cs | 64 +- .../cs/src/core/Index/Common/LogSettings.cs | 60 +- .../src/core/Index/Common/PendingContext.cs | 149 +-- .../cs/src/core/Index/Common/RecordInfo.cs | 89 +- .../core/Index/Interfaces/CallbackInfos.cs | 7 +- .../Index/Interfaces/SessionFunctionsBase.cs | 9 +- .../cs/src/core/Index/Recovery/DeltaLog.cs | 14 +- .../cs/src/core/Index/Recovery/Recovery.cs | 123 +- .../Index/StoreFunctions/DisposeReason.cs | 7 +- .../cs/src/core/Index/Tsavorite/Constants.cs | 1 + .../Tsavorite/Implementation/BlockAllocate.cs | 35 +- .../Implementation/ConditionalCopyToTail.cs | 4 +- .../Implementation/ContinuePending.cs | 33 +- .../Tsavorite/Implementation/FindRecord.cs | 4 +- .../Implementation/HandleOperationStatus.cs | 4 +- .../Index/Tsavorite/Implementation/Helpers.cs | 29 +- .../Implementation/InternalDelete.cs | 14 +- .../Tsavorite/Implementation/InternalRMW.cs | 65 +- .../Implementation/InternalUpsert.cs | 16 +- .../Implementation/ModifiedBitOperation.cs | 2 +- .../Tsavorite/Implementation/ReadCache.cs | 21 +- .../Tsavorite/Implementation/RecordSource.cs | 4 +- .../Revivification/FreeRecordPool.cs | 5 +- .../Tsavorite/Implementation/SplitIndex.cs | 4 +- .../Implementation/TryCopyToReadCache.cs | 28 +- .../Tsavorite/Implementation/TryCopyToTail.cs | 10 +- .../src/core/Index/Tsavorite/LogAccessor.cs | 11 +- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 41 +- .../core/Index/Tsavorite/TsavoriteIterator.cs | 38 +- .../core/Index/Tsavorite/TsavoriteThread.cs | 18 +- .../cs/src/core/TsavoriteLog/TsavoriteLog.cs | 16 +- .../core/TsavoriteLog/TsavoriteLogIterator.cs | 46 +- .../cs/src/core/Utilities/BufferPool.cs | 93 +- .../core/Utilities/PageAsyncResultTypes.cs | 318 +++++- .../Tsavorite/cs/src/core/Utilities/Status.cs | 19 +- .../cs/src/core/Utilities/StatusCode.cs | 9 +- .../cs/src/core/Utilities/Utility.cs | 84 +- .../cs/src/core/VarLen/RecordFieldInfo.cs | 22 +- .../cs/src/core/VarLen/RecordSizeInfo.cs | 59 +- .../cs/src/core/VarLen/SpanByteFunctions.cs | 8 +- .../src/core/VarLen/SpanByteHeapContainer.cs | 2 +- .../Tsavorite/cs/test/BasicLockTests.cs | 6 +- .../Tsavorite/cs/test/CancellationTests.cs | 8 +- .../Tsavorite/cs/test/LogRecordTests.cs | 329 +++--- .../storage/Tsavorite/cs/test/LogScanTests.cs | 2 +- .../Tsavorite/cs/test/ObjectIdMapTests.cs | 8 +- .../Tsavorite/cs/test/ObjectInlineTests.cs | 21 +- .../Tsavorite/cs/test/ObjectTestTypes.cs | 133 ++- libs/storage/Tsavorite/cs/test/ObjectTests.cs | 224 +++- .../Tsavorite/cs/test/ReadCacheChainTests.cs | 6 +- .../Tsavorite/cs/test/RevivificationTests.cs | 44 +- libs/storage/Tsavorite/cs/test/TestTypes.cs | 24 +- main/GarnetServer/Extensions/MyDictObject.cs | 14 +- modules/GarnetJSON/GarnetJsonObject.cs | 3 +- .../ClusterMigrateTests.cs | 2 +- test/Garnet.test/GarnetBitmapTests.cs | 8 +- test/Garnet.test/GarnetObjectTests.cs | 12 +- test/Garnet.test/RespEtagTests.cs | 2 +- test/Garnet.test/RespListTests.cs | 8 +- website/docs/dev/tsavorite/logrecord.md | 24 +- .../docs/dev/tsavorite/object-allocator.md | 22 +- 151 files changed, 7087 insertions(+), 4219 deletions(-) create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/HeapObjectBase.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectScanIterator.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskReadBuffer.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskWriteBuffer.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskReadBuffer.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskWriteBuffer.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/IStreamBuffer.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogFilePositionInfo.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogReader.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogWriter.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/PinnedMemoryStream.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/VarbyteLengthUtility.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/OverflowByteArray.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/PageHeader.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Allocator/SerializationPhase.cs rename libs/storage/Tsavorite/cs/src/core/Allocator/{RecordScanIterator.cs => SpanByteScanIterator.cs} (67%) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index ae96ddd972a..41f0e712887 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -109,7 +109,7 @@ public bool StringReader(in TSourceLogRecord srcLogRecord, Rec } // Note: We may be sending to multiple replicas, so cannot serialize LogRecords directly to the network buffer - DiskLogRecord.Serialize(in srcLogRecord, valueSerializer: null, ref serializationOutput, memoryPool); + DiskLogRecord.Serialize(in srcLogRecord, valueObjectSerializer: default, memoryPool, ref serializationOutput); var needToFlush = false; while (true) @@ -161,7 +161,7 @@ public bool ObjectReader(in TSourceLogRecord srcLogRecord, Rec } // Note: We may be sending to multiple replicas, so cannot serialize LogRecords directly to the network buffer - DiskLogRecord.Serialize(in srcLogRecord, valueObjectSerializer, ref serializationOutput, memoryPool); + DiskLogRecord.Serialize(in srcLogRecord, valueObjectSerializer, memoryPool, ref serializationOutput); var needToFlush = false; while (true) diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 465a39de360..7d1f9b63ce5 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -92,32 +92,44 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan, var i = 0; TrackImportProgress(keyCount, isMainStore: true, keyCount == 0); - while (i < keyCount) - { - if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) - return; - - // An error has occurred - if (migrateState > 0) - { - i++; - continue; - } + var storeWrapper = clusterProvider.storeWrapper; - var diskLogRecord = new DiskLogRecord(startAddress, length); - var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); - if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state + // Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord. + DiskLogRecord diskLogRecord = default; + try + { + while (i < keyCount) { - migrateState = 1; + if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr)) + return; + + // An error has occurred + if (migrateState > 0) + { + i++; + continue; + } + + diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.storeFunctions); + var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); + if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state + { + migrateState = 1; + i++; + continue; + } + + // Set if key replace flag is set or key does not exist + var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); + if (replaceOption || !Exists(keySlice)) + _ = basicGarnetApi.SET_Main(in diskLogRecord); + diskLogRecord.Dispose(); i++; - continue; } - - // Set if key replace flag is set or key does not exist - var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); - if (replaceOption || !Exists(keySlice)) - _ = basicGarnetApi.SET_Main(in diskLogRecord); - i++; + } + finally + { + diskLogRecord.Dispose(); } } else if (storeTypeSpan.Equals("OSTORE", StringComparison.OrdinalIgnoreCase)) @@ -126,30 +138,41 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan, payloadPtr += 4; var i = 0; TrackImportProgress(keyCount, isMainStore: false, keyCount == 0); - while (i < keyCount) - { - if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) - return; - - // An error has occurred - if (migrateState > 0) - continue; + var storeWrapper = clusterProvider.storeWrapper; + var transientObjectIdMap = storeWrapper.store.Log.TransientObjectIdMap; - var diskLogRecord = new DiskLogRecord(startAddress, length); - var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); - if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state + // Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord. + DiskLogRecord diskLogRecord = default; + try + { + while (i < keyCount) { - migrateState = 1; - continue; + if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr)) + return; + + // An error has occurred + if (migrateState > 0) + continue; + + diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.storeFunctions); + var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); + if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state + { + migrateState = 1; + continue; + } + + // Set if key replace flag is set or key does not exist + var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); + if (replaceOption || !Exists(keySlice)) + _ = basicGarnetApi.SET_Object(in diskLogRecord); + diskLogRecord.Dispose(); + i++; } - - _ = diskLogRecord.DeserializeValueObject(clusterProvider.storeWrapper.GarnetObjectSerializer); - - // Set if key replace flag is set or key does not exist - var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key); - if (replaceOption || !Exists(keySlice)) - _ = basicGarnetApi.SET_Object(in diskLogRecord); - i++; + } + finally + { + diskLogRecord.Dispose(); } } else diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index b9caf99708d..317e55f196a 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -466,28 +466,52 @@ private bool NetworkClusterSync(out bool invalidParameters) if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("SSTORE"u8)) { TrackImportProgress(recordCount, isMainStore: true, recordCount == 0); - while (i < recordCount) - { - if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) - return false; + var storeWrapper = clusterProvider.storeWrapper; - var diskLogRecord = new DiskLogRecord(startAddress, length); - _ = basicGarnetApi.SET_Main(in diskLogRecord); - i++; + // Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord. + DiskLogRecord diskLogRecord = default; + try + { + while (i < recordCount) + { + if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr)) + return false; + + diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.storeFunctions); + _ = basicGarnetApi.SET_Main(in diskLogRecord); + diskLogRecord.Dispose(); + i++; + } + } + finally + { + diskLogRecord.Dispose(); } } else if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("OSTORE"u8)) { TrackImportProgress(recordCount, isMainStore: false, recordCount == 0); - while (i < recordCount) - { - if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) - return false; + var storeWrapper = clusterProvider.storeWrapper; + var transientObjectIdMap = storeWrapper.store.Log.TransientObjectIdMap; - var diskLogRecord = new DiskLogRecord(startAddress, length); - _ = diskLogRecord.DeserializeValueObject(clusterProvider.storeWrapper.GarnetObjectSerializer); - _ = basicGarnetApi.SET_Object(in diskLogRecord); - i++; + // Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord. + DiskLogRecord diskLogRecord = default; + try + { + while (i < recordCount) + { + if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr)) + return false; + + diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.storeFunctions); + _ = basicGarnetApi.SET_Object(in diskLogRecord); + diskLogRecord.Dispose(); + i++; + } + } + finally + { + diskLogRecord.Dispose(); } } diff --git a/libs/common/RespReadUtils.cs b/libs/common/RespReadUtils.cs index c658046a674..d8b351a7c6d 100644 --- a/libs/common/RespReadUtils.cs +++ b/libs/common/RespReadUtils.cs @@ -7,6 +7,7 @@ using System.Runtime.InteropServices; using System.Text; using Garnet.common.Parsing; +using Tsavorite.core; namespace Garnet.common { @@ -1225,23 +1226,21 @@ public static bool TryReadAsSpan(out ReadOnlySpan result, ref byte* ptr, b } /// - /// Read serialized data for migration + /// Read serialized data for migration and replication. For details of the layout see . /// - public static bool TryReadSerializedRecord(out long recordStartAddress, out int recordLength, ref byte* ptr, byte* end) + public static bool GetSerializedRecordSpan(out PinnedSpanByte recordSpan, ref byte* ptr, byte* end) { - recordStartAddress = 0; - recordLength = 0; - - //1. safe read recordSize + // 1. Safe read recordSize. if (ptr + sizeof(int) > end) + { + recordSpan = default; return false; - recordLength = *(int*)ptr; + } + var recordLength = *(int*)ptr; ptr += sizeof(int); - //2. safe read keyPtr - if (ptr + recordLength > end) - return false; - recordStartAddress = (long)ptr; + // 2. The record starts immediately after the length prefix. + recordSpan = PinnedSpanByte.FromPinnedPointer(ptr, recordLength); ptr += recordLength; return true; } diff --git a/libs/server/Custom/CustomObjectBase.cs b/libs/server/Custom/CustomObjectBase.cs index c80c82201fb..e7a86f49317 100644 --- a/libs/server/Custom/CustomObjectBase.cs +++ b/libs/server/Custom/CustomObjectBase.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. using System.IO; +using Tsavorite.core; namespace Garnet.server { diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index 2a2d75a2fad..f06be9d602b 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -568,7 +568,7 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo { var valueObject = iter1.ValueObject; if (valueObject != null) - ((GarnetObjectBase)iter1.ValueObject).serialized = null; + ((GarnetObjectBase)iter1.ValueObject).ClearSerializedObjectData(); } logger?.LogInformation("Completed checkpoint for DB ID: {id}", db.Id); diff --git a/libs/server/Objects/Hash/HashObject.cs b/libs/server/Objects/Hash/HashObject.cs index 759e2a7fafd..0eaf71735b0 100644 --- a/libs/server/Objects/Hash/HashObject.cs +++ b/libs/server/Objects/Hash/HashObject.cs @@ -76,7 +76,7 @@ private bool HasExpirableItems /// Constructor /// public HashObject() - : base(new(MemoryUtils.DictionaryOverhead, sizeof(int))) + : base(new(MemoryUtils.DictionaryOverhead, sizeof(int), serializedIsExact: true)) { hash = new Dictionary(ByteArrayComparer.Instance); #if NET9_0_OR_GREATER @@ -88,7 +88,7 @@ public HashObject() /// Construct from binary serialized form /// public HashObject(BinaryReader reader) - : base(reader, new(MemoryUtils.DictionaryOverhead, sizeof(int))) + : base(reader, new(MemoryUtils.DictionaryOverhead, sizeof(int), serializedIsExact: true)) { var count = reader.ReadInt32(); hash = new Dictionary(count, ByteArrayComparer.Instance); @@ -194,7 +194,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var previousMemorySize = this.MemorySize; + var previousMemorySize = this.HeapMemorySize; switch (input.header.HashOp) { case HashOperation.HSET: @@ -261,7 +261,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {input.header.HashOp} in HashObject.Operate"); } - memorySizeChange = this.MemorySize - previousMemorySize; + memorySizeChange = this.HeapMemorySize - previousMemorySize; if (hash.Count == 0) output.OutputFlags |= OutputFlags.RemoveKey; @@ -279,15 +279,15 @@ private void UpdateSize(ReadOnlySpan key, ReadOnlySpan value, bool a if (add) { - this.MemorySize += memorySize; - this.DiskSize += kvSize; + this.HeapMemorySize += memorySize; + this.SerializedSize += kvSize; } else { - this.MemorySize -= memorySize; - this.DiskSize -= kvSize; - Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= kvSize; + Debug.Assert(this.HeapMemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } @@ -301,7 +301,7 @@ private void InitializeExpirationStructures() #if NET9_0_OR_GREATER expirationTimeSpanLookup = expirationTimes.GetAlternateLookup>(); #endif - this.MemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.HeapMemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; } } @@ -315,15 +315,15 @@ private void UpdateExpirationSize(bool add, bool includePQ = true) if (add) { - this.MemorySize += memorySize; - this.DiskSize += sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value + this.HeapMemorySize += memorySize; + this.SerializedSize += sizeof(long); // SerializedSize only needs to adjust the writing or not of the expiration value } else { - this.MemorySize -= memorySize; - this.DiskSize -= sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value - Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= sizeof(long); // SerializedSize only needs to adjust the writing or not of the expiration value + Debug.Assert(this.HeapMemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } @@ -332,9 +332,9 @@ private void CleanupExpirationStructuresIfEmpty() { if (expirationTimes.Count == 0) { - this.MemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; - this.MemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; - this.DiskSize -= sizeof(long) * expirationTimes.Count; + this.HeapMemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; + this.HeapMemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.SerializedSize -= sizeof(long) * expirationTimes.Count; expirationTimes = null; expirationQueue = null; #if NET9_0_OR_GREATER @@ -432,9 +432,7 @@ private void DeleteExpiredItemsWorker() _ = expirationQueue.Dequeue(); UpdateExpirationSize(add: false); if (hash.Remove(key, out var value)) - { UpdateSize(key, value, add: false); - } } else { @@ -442,7 +440,7 @@ private void DeleteExpiredItemsWorker() _ = expirationQueue.Dequeue(); // Adjust memory size for the priority queue entry removal. No DiskSize change needed as it was not in expirationTimes. - this.MemorySize -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); + this.HeapMemorySize -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); } } @@ -476,6 +474,11 @@ private bool Remove(ByteSpan key, out byte[] value) if (HasExpirableItems) { // We cannot remove from the PQ so just remove from expirationTimes, let the next call to DeleteExpiredItems() clean it up, and don't adjust PQ sizes. +#if NET9_0_OR_GREATER + _ = expirationTimeSpanLookup.Remove(key); +#else + _ = expirationTimes.Remove(key); +#endif UpdateExpirationSize(add: false, includePQ: false); } UpdateSize(key, value, add: false); @@ -577,8 +580,8 @@ private ExpireResult SetExpiration(ByteSpan key, long expiration, ExpireOption e #endif // MemorySize of dictionary entry already accounted for as the key already exists. - // DiskSize of expiration already accounted for as the key already exists in expirationTimes. - this.MemorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + // SerializedSize of expiration is already accounted for as the key already exists in expirationTimes. + this.HeapMemorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; } else { @@ -610,8 +613,8 @@ private int Persist(ByteSpan key) if (HasExpirableItems && expirationTimes.Remove(key, out var currentExpiration)) #endif { - this.MemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; - this.DiskSize -= sizeof(long); // expiration value size + this.HeapMemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + this.SerializedSize -= sizeof(long); // expiration value size CleanupExpirationStructuresIfEmpty(); return (int)ExpireResult.ExpireUpdated; } diff --git a/libs/server/Objects/Hash/HashObjectImpl.cs b/libs/server/Objects/Hash/HashObjectImpl.cs index beee1349ed5..3344db55438 100644 --- a/libs/server/Objects/Hash/HashObjectImpl.cs +++ b/libs/server/Objects/Hash/HashObjectImpl.cs @@ -213,9 +213,8 @@ private void HashSet(ref ObjectInput input, ref GarnetObjectStoreOutput output) else { // Adjust the size to account for the new value replacing the old one. - this.MemorySize += Utility.RoundUp(value.Length, IntPtr.Size) - - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); - this.DiskSize += value.Length - hashValueRef.Length; + this.HeapMemorySize += Utility.RoundUp(value.Length, IntPtr.Size) - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); + this.SerializedSize += value.Length - hashValueRef.Length; hashValueRef = value.ToArray(); } @@ -228,7 +227,7 @@ private void HashSet(ref ObjectInput input, ref GarnetObjectStoreOutput output) expirationTimes.Remove(key)) #endif { - this.MemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + this.HeapMemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; CleanupExpirationStructuresIfEmpty(); } } @@ -332,9 +331,8 @@ private void HashIncrement(ref ObjectInput input, ref GarnetObjectStoreOutput ou else { // Adjust the size to account for the new value replacing the old one. - this.MemorySize += Utility.RoundUp(formattedValue.Length, IntPtr.Size) - - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); - this.DiskSize += formattedValue.Length - hashValueRef.Length; + this.HeapMemorySize += Utility.RoundUp(formattedValue.Length, IntPtr.Size) - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); + this.SerializedSize += formattedValue.Length - hashValueRef.Length; hashValueRef = formattedValue.ToArray(); } @@ -416,9 +414,8 @@ private void HashIncrementFloat(ref ObjectInput input, ref GarnetObjectStoreOutp else { // Adjust the size to account for the new value replacing the old one. - this.MemorySize += Utility.RoundUp(formattedValue.Length, IntPtr.Size) - - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); - this.DiskSize += formattedValue.Length - hashValueRef.Length; + this.HeapMemorySize += Utility.RoundUp(formattedValue.Length, IntPtr.Size) - Utility.RoundUp(hashValueRef.Length, IntPtr.Size); + this.SerializedSize += formattedValue.Length - hashValueRef.Length; hashValueRef = formattedValue.ToArray(); } diff --git a/libs/server/Objects/List/ListObject.cs b/libs/server/Objects/List/ListObject.cs index 87d101563b7..1a9b333d51c 100644 --- a/libs/server/Objects/List/ListObject.cs +++ b/libs/server/Objects/List/ListObject.cs @@ -65,7 +65,7 @@ public partial class ListObject : GarnetObjectBase /// Constructor /// public ListObject() - : base(new(MemoryUtils.ListOverhead, sizeof(int))) + : base(new(MemoryUtils.ListOverhead, sizeof(int), serializedIsExact: true)) { list = new LinkedList(); } @@ -74,7 +74,7 @@ public ListObject() /// Construct from binary serialized form /// public ListObject(BinaryReader reader) - : base(reader, new(MemoryUtils.ListOverhead, sizeof(int))) + : base(reader, new(MemoryUtils.ListOverhead, sizeof(int), serializedIsExact: true)) { list = new LinkedList(); @@ -141,7 +141,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var previousMemorySize = this.MemorySize; + var previousMemorySize = this.HeapMemorySize; switch (input.header.ListOp) { case ListOperation.LPUSH: @@ -187,7 +187,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {input.header.ListOp} in ListObject.Operate"); } - memorySizeChange = this.MemorySize - previousMemorySize; + memorySizeChange = this.HeapMemorySize - previousMemorySize; if (list.Count == 0) output.OutputFlags |= OutputFlags.RemoveKey; @@ -202,15 +202,15 @@ internal void UpdateSize(byte[] item, bool add = true) if (add) { - this.MemorySize += memorySize; - this.DiskSize += diskSize; + this.HeapMemorySize += memorySize; + this.SerializedSize += diskSize; } else { - this.MemorySize -= memorySize; - this.DiskSize -= diskSize; - Debug.Assert(this.MemorySize >= MemoryUtils.ListOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= diskSize; + Debug.Assert(this.HeapMemorySize >= MemoryUtils.ListOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } diff --git a/libs/server/Objects/Set/SetObject.cs b/libs/server/Objects/Set/SetObject.cs index f0c9530b02e..76a27c0959f 100644 --- a/libs/server/Objects/Set/SetObject.cs +++ b/libs/server/Objects/Set/SetObject.cs @@ -51,7 +51,7 @@ public partial class SetObject : GarnetObjectBase /// Constructor /// public SetObject() - : base(new(MemoryUtils.HashSetOverhead, sizeof(int))) + : base(new(MemoryUtils.HashSetOverhead, sizeof(int), serializedIsExact: true)) { Set = new HashSet(ByteArrayComparer.Instance); @@ -64,7 +64,7 @@ public SetObject() /// Construct from binary serialized form /// public SetObject(BinaryReader reader) - : base(reader, new(MemoryUtils.HashSetOverhead, sizeof(int))) + : base(reader, new(MemoryUtils.HashSetOverhead, sizeof(int), serializedIsExact: true)) { int count = reader.ReadInt32(); @@ -135,7 +135,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var prevMemorySize = this.MemorySize; + var prevMemorySize = this.HeapMemorySize; switch (input.header.SetOp) { case SetOperation.SADD: @@ -169,7 +169,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {input.header.SetOp} in SetObject.Operate"); } - memorySizeChange = this.MemorySize - prevMemorySize; + memorySizeChange = this.HeapMemorySize - prevMemorySize; if (Set.Count == 0) output.OutputFlags |= OutputFlags.RemoveKey; @@ -184,15 +184,15 @@ internal void UpdateSize(ReadOnlySpan item, bool add = true) if (add) { - this.MemorySize += memorySize; - this.DiskSize += kvSize; + this.HeapMemorySize += memorySize; + this.SerializedSize += kvSize; } else { - this.MemorySize -= memorySize; - this.DiskSize -= kvSize; - Debug.Assert(this.MemorySize >= MemoryUtils.HashSetOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= kvSize; + Debug.Assert(this.HeapMemorySize >= MemoryUtils.HashSetOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } diff --git a/libs/server/Objects/SortedSet/SortedSetObject.cs b/libs/server/Objects/SortedSet/SortedSetObject.cs index d52b2f54691..8adda584c6b 100644 --- a/libs/server/Objects/SortedSet/SortedSetObject.cs +++ b/libs/server/Objects/SortedSet/SortedSetObject.cs @@ -151,7 +151,7 @@ public partial class SortedSetObject : GarnetObjectBase /// Constructor /// public SortedSetObject() - : base(new(MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int))) + : base(new(MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int), serializedIsExact: true)) { sortedSet = new(SortedSetComparer.Instance); sortedSetDict = new Dictionary(ByteArrayComparer.Instance); @@ -161,7 +161,7 @@ public SortedSetObject() /// Construct from binary serialized form /// public SortedSetObject(BinaryReader reader) - : base(reader, new(MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int))) + : base(reader, new(MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int), serializedIsExact: true)) { sortedSet = new(SortedSetComparer.Instance); sortedSetDict = new Dictionary(ByteArrayComparer.Instance); @@ -332,7 +332,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var prevMemorySize = this.MemorySize; + var prevMemorySize = this.HeapMemorySize; var op = header.SortedSetOp; switch (op) { @@ -418,7 +418,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {op} in SortedSetObject.Operate"); } - memorySizeChange = this.MemorySize - prevMemorySize; + memorySizeChange = this.HeapMemorySize - prevMemorySize; if (sortedSetDict.Count == 0) output.OutputFlags |= OutputFlags.RemoveKey; @@ -611,7 +611,7 @@ private void InitializeExpirationStructures() { expirationTimes = new Dictionary(ByteArrayComparer.Instance); expirationQueue = new PriorityQueue(); - this.MemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.HeapMemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; // No DiskSize adjustment needed yet; wait until keys are added or removed } } @@ -626,15 +626,15 @@ private void UpdateExpirationSize(bool add, bool includePQ = true) if (add) { - this.MemorySize += memorySize; - this.DiskSize += sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value + this.HeapMemorySize += memorySize; + this.SerializedSize += sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value } else { - this.MemorySize -= memorySize; - this.DiskSize -= sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value - Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= sizeof(long); // DiskSize only needs to adjust the writing or not of the expiration value + Debug.Assert(this.HeapMemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } @@ -643,9 +643,9 @@ private void CleanupExpirationStructuresIfEmpty() { if (expirationTimes.Count == 0) { - this.MemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; - this.MemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; - this.DiskSize -= sizeof(long) * expirationTimes.Count; + this.HeapMemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; + this.HeapMemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.SerializedSize -= sizeof(long) * expirationTimes.Count; expirationTimes = null; expirationQueue = null; } @@ -681,7 +681,7 @@ private void DeleteExpiredItemsWorker() _ = expirationQueue.Dequeue(); // Adjust memory size for the priority queue entry removal. No DiskSize change needed as it was not in expirationTimes. - this.MemorySize -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); + this.HeapMemorySize -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); } } @@ -717,7 +717,7 @@ private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption // MemorySize of dictionary entry already accounted for as the key already exists. // DiskSize of expiration already accounted for as the key already exists in expirationTimes. - this.MemorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + this.HeapMemorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; } else { @@ -796,15 +796,15 @@ private void UpdateSize(ReadOnlySpan item, bool add = true) if (add) { - this.MemorySize += memorySize; - this.DiskSize += kvSize; + this.HeapMemorySize += memorySize; + this.SerializedSize += kvSize; } else { - this.MemorySize -= memorySize; - this.DiskSize -= kvSize; - Debug.Assert(this.MemorySize >= MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + this.HeapMemorySize -= memorySize; + this.SerializedSize -= kvSize; + Debug.Assert(this.HeapMemorySize >= MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead); + Debug.Assert(this.SerializedSize >= sizeof(int)); } } diff --git a/libs/server/Objects/Types/GarnetObjectBase.cs b/libs/server/Objects/Types/GarnetObjectBase.cs index 2e1a115694a..16a764bbea6 100644 --- a/libs/server/Objects/Types/GarnetObjectBase.cs +++ b/libs/server/Objects/Types/GarnetObjectBase.cs @@ -3,52 +3,20 @@ using System; using System.Collections.Generic; -using System.Diagnostics; using System.IO; -using System.Text; -using System.Threading; using Garnet.common; using Tsavorite.core; namespace Garnet.server { - public struct ObjectSizes - { - /// In-memory size, including .NET object overheads - public long Memory; - - /// Serialized size, for disk IO or other storage - public long Disk; - - public ObjectSizes(long memory, long disk) - { - Memory = memory; - Disk = disk + sizeof(byte); // Additional byte for GarnetObjectBase.Type - } - - [Conditional("DEBUG")] - public void Verify() => Debug.Assert(Memory >= 0 && Disk >= 0, $"Invalid sizes [{Memory}, {Disk}]"); - } - /// /// Base class for Garnet heap objects /// - public abstract class GarnetObjectBase : IGarnetObject + public abstract class GarnetObjectBase : HeapObjectBase, IGarnetObject { - int serializationState; - public byte[] serialized; - /// public abstract byte Type { get; } - /// - public long MemorySize { get => sizes.Memory; set => sizes.Memory = value; } - - /// - public long DiskSize { get => sizes.Disk; set => sizes.Disk = value; } - - public ObjectSizes sizes; - protected GarnetObjectBase(ObjectSizes sizes) { sizes.Verify(); @@ -61,103 +29,20 @@ protected GarnetObjectBase(BinaryReader reader, ObjectSizes sizes) // Add anything here that should match DoSerialize() } - /// - public void Serialize(BinaryWriter writer) - { - while (true) - { - if (serializationState == (int)SerializationPhase.REST && MakeTransition(SerializationPhase.REST, SerializationPhase.SERIALIZING)) - { - // Directly serialize to wire, do not cache serialized state - writer.Write(Type); - DoSerialize(writer); - serializationState = (int)SerializationPhase.REST; - return; - } - - if (serializationState == (int)SerializationPhase.SERIALIZED) - { - // If serialized state is cached, use that - var _serialized = serialized; - if (_serialized != null) - { - writer.Write(Type); - writer.Write(_serialized); - } - else - { - // Write null object to stream - writer.Write((byte)GarnetObjectType.Null); - } - return; - } - - Thread.Yield(); - } - } - - /// - public IGarnetObject CopyUpdate(bool isInNewVersion, ref RMWInfo rmwInfo) - { - var newValue = Clone(); - - // If we are not currently taking a checkpoint, we can delete the old version - // since the new version of the object is already created. - if (!isInNewVersion) - { - rmwInfo.ClearSourceValueObject = true; - return newValue; - } - - // Create a serialized version for checkpoint version (v) - while (true) - { - if (serializationState == (int)SerializationPhase.REST && MakeTransition(SerializationPhase.REST, SerializationPhase.SERIALIZING)) - { - using var ms = new MemoryStream(); - using var writer = new BinaryWriter(ms, Encoding.UTF8); - DoSerialize(writer); - serialized = ms.ToArray(); - - serializationState = (int)SerializationPhase.SERIALIZED; - break; - } - - if (serializationState >= (int)SerializationPhase.SERIALIZED) - break; - - _ = Thread.Yield(); - } - - return newValue; - } - - /// - /// Clone object (shallow copy) - /// - /// - public abstract GarnetObjectBase Clone(); - /// public abstract bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, byte respProtocolVersion, out long sizeChange); - /// - public abstract void Dispose(); - /// /// Serialize to given writer /// NOTE: Make sure to first call base.DoSerialize(writer) in all derived classes. /// - public virtual void DoSerialize(BinaryWriter writer) + public override void DoSerialize(BinaryWriter writer) { // Add anything here that needs to be in front of the derived object data } - private bool MakeTransition(SerializationPhase expectedPhase, SerializationPhase nextPhase) - { - if (Interlocked.CompareExchange(ref serializationState, (int)nextPhase, (int)expectedPhase) != (int)expectedPhase) return false; - return true; - } + /// + public override void WriteType(BinaryWriter writer, bool isNull) => writer.Write(isNull ? (byte)GarnetObjectType.Null : Type); /// /// Scan the items of the collection diff --git a/libs/server/Objects/Types/IGarnetObject.cs b/libs/server/Objects/Types/IGarnetObject.cs index 772db75c0e7..a9fa0e423a7 100644 --- a/libs/server/Objects/Types/IGarnetObject.cs +++ b/libs/server/Objects/Types/IGarnetObject.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System.Collections.Generic; -using System.IO; using Tsavorite.core; namespace Garnet.server @@ -26,16 +25,6 @@ public interface IGarnetObject : IHeapObject /// bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, byte respProtocolVersion, out long sizeChange); - /// - /// Serializer - /// - void Serialize(BinaryWriter writer); - - /// - /// Create a cloned CopyUpdate of this object - /// - IGarnetObject CopyUpdate(bool isInNewVersion, ref RMWInfo rmwInfo); - /// /// Scan the items of the collection /// diff --git a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs index 53098a397de..fd9ca13350f 100644 --- a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs @@ -100,7 +100,7 @@ void CopyRespToWithInput(in TSourceLogRecord srcLogRecord, ref break; case RespCommand.MIGRATE: - DiskLogRecord.Serialize(in srcLogRecord, valueSerializer: null, ref output, functionsState.memoryPool); + DiskLogRecord.Serialize(in srcLogRecord, valueObjectSerializer: null, memoryPool: functionsState.memoryPool, output: ref output); break; case RespCommand.GET: @@ -450,6 +450,10 @@ bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, in RecordSizeInfo sizeInf return (0, 0); } + internal static bool CheckExpiry(in TSourceLogRecord srcLogRecord) + where TSourceLogRecord : ISourceLogRecord + => srcLogRecord.Info.HasExpiration && srcLogRecord.Expiration < DateTimeOffset.UtcNow.Ticks; + static bool InPlaceUpdateNumber(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, long val, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { var ndigits = NumUtils.CountDigits(val, out var isNegative); diff --git a/libs/server/Storage/Functions/MainStore/RMWMethods.cs b/libs/server/Storage/Functions/MainStore/RMWMethods.cs index 732a27793c8..910b4b14127 100644 --- a/libs/server/Storage/Functions/MainStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/MainStore/RMWMethods.cs @@ -190,7 +190,7 @@ public readonly bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo s var bOffset = input.arg1; var bSetVal = (byte)(input.parseState.GetArgSliceByRef(1).ReadOnlySpan[0] - '0'); - if (!logRecord.TrySetValueLength(BitmapManager.Length(bOffset), in sizeInfo)) + if (!logRecord.TrySetValueLength(BitmapManager.Length(bOffset), in sizeInfo, zeroInit: true)) { functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "SETBIT"); return false; @@ -211,7 +211,7 @@ public readonly bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo s case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - if (!logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), in sizeInfo)) + if (!logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), in sizeInfo, zeroInit: true)) { functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "BitField"); return false; @@ -312,7 +312,7 @@ public readonly bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo s if (input.header.cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)input.header.cmd); - if (!logRecord.TrySetValueLength(functions.GetInitialLength(ref input), in sizeInfo)) + if (!logRecord.TrySetValueLength(functions.GetInitialLength(ref input), in sizeInfo, zeroInit: true)) // ZeroInit to be safe { functionsState.logger?.LogError("Length overflow in 'default' > StartOffset: {methodName}.{caseName}", "InitialUpdater", "default"); return false; @@ -327,7 +327,7 @@ public readonly bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo s try { functions.InitialUpdater(logRecord.Key, ref input, logRecord.ValueSpan, ref writer, ref rmwInfo); - Debug.Assert(sizeInfo.FieldInfo.ValueDataSize == logRecord.ValueSpan.Length, $"Inconsistency in initial updater value length: expected {sizeInfo.FieldInfo.ValueDataSize}, actual {logRecord.ValueSpan.Length}"); + Debug.Assert(sizeInfo.FieldInfo.ValueSize == logRecord.ValueSpan.Length, $"Inconsistency in initial updater value length: expected {sizeInfo.FieldInfo.ValueSize}, actual {logRecord.ValueSpan.Length}"); } finally { @@ -494,20 +494,31 @@ private readonly bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSiz if (!logRecord.TrySetValueSpan(setValue, in sizeInfo)) return false; - if (inputHeaderHasEtag != shouldUpdateEtag) - shouldUpdateEtag = inputHeaderHasEtag; + // If shouldUpdateEtag != inputHeaderHasEtag, then if inputHeaderHasEtag is true there is one that nextUpdate will remove (so we don't want to + // update it), else there isn't one and nextUpdate will add it. + shouldUpdateEtag = inputHeaderHasEtag; + + // Update expiration + if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) + return false; + + // If withEtag is called we return the etag back in the response if (inputHeaderHasEtag) { var newETag = functionsState.etagState.ETag + 1; - logRecord.TrySetETag(newETag); + if (!logRecord.TrySetETag(newETag)) + return false; functionsState.CopyRespNumber(newETag, ref output); + // reset etag state after done using + ETagState.ResetState(ref functionsState.etagState); } else - logRecord.RemoveETag(); - shouldUpdateEtag = false; // since we already updated the ETag + { + if (!logRecord.RemoveETag()) + return false; + } - if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) - return false; + shouldUpdateEtag = false; // since we already updated the ETag break; case RespCommand.SETKEEPTTLXX: case RespCommand.SETKEEPTTL: @@ -590,7 +601,7 @@ private readonly bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSiz var bSetVal = (byte)(input.parseState.GetArgSliceByRef(1).ReadOnlySpan[0] - '0'); if (!BitmapManager.IsLargeEnough(logRecord.ValueSpan.Length, bOffset) - && !logRecord.TrySetValueLength(BitmapManager.Length(bOffset), in sizeInfo)) + && !logRecord.TrySetValueLength(BitmapManager.Length(bOffset), in sizeInfo, zeroInit: true)) return false; _ = logRecord.RemoveExpiration(); @@ -610,7 +621,7 @@ private readonly bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSiz case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); if (!BitmapManager.IsLargeEnoughForType(bitFieldArgs, logRecord.ValueSpan.Length) - && !logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), in sizeInfo)) + && !logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), in sizeInfo, zeroInit: true)) return false; _ = logRecord.RemoveExpiration(); @@ -1039,9 +1050,6 @@ public readonly bool CopyUpdater(in TSourceLogRecord srcLogRec case RespCommand.SETEXXX: bool inputHeaderHasEtag = input.header.CheckWithETagFlag(); - if (inputHeaderHasEtag != shouldUpdateEtag) - shouldUpdateEtag = inputHeaderHasEtag; - // Check if SetGet flag is set if (input.header.CheckSetGetFlag()) { @@ -1057,21 +1065,29 @@ public readonly bool CopyUpdater(in TSourceLogRecord srcLogRec if (!dstLogRecord.TrySetValueSpan(newInputValue, in sizeInfo) || !dstLogRecord.TryCopyOptionals(in srcLogRecord, in sizeInfo)) return false; - if (inputHeaderHasEtag != shouldUpdateEtag) - shouldUpdateEtag = inputHeaderHasEtag; + // Update expiration if it was supplied. + if (input.arg1 != 0 && !dstLogRecord.TrySetExpiration(input.arg1)) + return false; + + // If shouldUpdateEtag != inputHeaderHasEtag, then if inputHeaderHasEtag is true there is one that nextUpdate will remove (so we don't want to + // update it), else there isn't one and nextUpdate will add it. + shouldUpdateEtag = inputHeaderHasEtag; + if (inputHeaderHasEtag) { var newETag = functionsState.etagState.ETag + 1; - dstLogRecord.TrySetETag(newETag); + if (!dstLogRecord.TrySetETag(newETag)) + return false; functionsState.CopyRespNumber(newETag, ref output); + ETagState.ResetState(ref functionsState.etagState); } else - dstLogRecord.RemoveETag(); + { + if (!dstLogRecord.RemoveETag()) + return false; + } shouldUpdateEtag = false; // since we already updated the ETag - // Update expiration if it was supplied. - if (input.arg1 != 0 && !dstLogRecord.TrySetExpiration(input.arg1)) - return false; break; case RespCommand.SETKEEPTTLXX: @@ -1459,7 +1475,9 @@ public readonly bool CopyUpdater(in TSourceLogRecord srcLogRec if (shouldUpdateEtag) { - dstLogRecord.TrySetETag(functionsState.etagState.ETag + 1); + if (cmd is not RespCommand.SETIFGREATER) + functionsState.etagState.ETag++; + dstLogRecord.TrySetETag(functionsState.etagState.ETag); ETagState.ResetState(ref functionsState.etagState); } else if (recordHadEtagPreMutation) diff --git a/libs/server/Storage/Functions/MainStore/ReadMethods.cs b/libs/server/Storage/Functions/MainStore/ReadMethods.cs index c853a502297..c222cde3fe9 100644 --- a/libs/server/Storage/Functions/MainStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/MainStore/ReadMethods.cs @@ -7,9 +7,6 @@ namespace Garnet.server { -#pragma warning disable IDE0005 // Using directive is unnecessary. - using static LogRecordUtils; - /// /// Callback functions for main store /// diff --git a/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs index d4cb62beee9..6a101163fad 100644 --- a/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs @@ -102,8 +102,8 @@ public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref RawStr var cmd = input.header.cmd; var fieldInfo = new RecordFieldInfo() { - KeyDataSize = key.Length, - ValueDataSize = 0, + KeySize = key.Length, + ValueSize = 0, HasETag = input.header.CheckWithETagFlag() }; @@ -111,80 +111,80 @@ public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref RawStr { case RespCommand.SETBIT: var bOffset = input.arg1; - fieldInfo.ValueDataSize = BitmapManager.Length(bOffset); + fieldInfo.ValueSize = BitmapManager.Length(bOffset); return fieldInfo; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - fieldInfo.ValueDataSize = BitmapManager.LengthFromType(bitFieldArgs); + fieldInfo.ValueSize = BitmapManager.LengthFromType(bitFieldArgs); return fieldInfo; case RespCommand.PFADD: - fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.SparseInitialLength(ref input); + fieldInfo.ValueSize = HyperLogLog.DefaultHLL.SparseInitialLength(ref input); return fieldInfo; case RespCommand.PFMERGE: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; return fieldInfo; case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; fieldInfo.HasETag = true; fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; case RespCommand.SET: case RespCommand.SETEXNX: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; case RespCommand.SETKEEPTTL: // Copy input to value; do not change expiration - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; return fieldInfo; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - fieldInfo.ValueDataSize = newValue.Length + offset; + fieldInfo.ValueSize = newValue.Length + offset; return fieldInfo; case RespCommand.APPEND: var valueLength = input.parseState.GetArgSliceByRef(0).Length; - fieldInfo.ValueDataSize = valueLength; + fieldInfo.ValueSize = valueLength; return fieldInfo; case RespCommand.INCR: - fieldInfo.ValueDataSize = 1; // # of digits in "1" + fieldInfo.ValueSize = 1; // # of digits in "1" return fieldInfo; case RespCommand.DECR: - fieldInfo.ValueDataSize = 2; // # of digits in "-1" + fieldInfo.ValueSize = 2; // # of digits in "-1" return fieldInfo; case RespCommand.INCRBY: var ndigits = NumUtils.CountDigits(input.arg1, out var isNegative); - fieldInfo.ValueDataSize = ndigits + (isNegative ? 1 : 0); + fieldInfo.ValueSize = ndigits + (isNegative ? 1 : 0); return fieldInfo; case RespCommand.DECRBY: ndigits = NumUtils.CountDigits(-input.arg1, out isNegative); - fieldInfo.ValueDataSize = ndigits + (isNegative ? 1 : 0); + fieldInfo.ValueSize = ndigits + (isNegative ? 1 : 0); return fieldInfo; case RespCommand.INCRBYFLOAT: var incrByFloat = BitConverter.Int64BitsToDouble(input.arg1); - fieldInfo.ValueDataSize = NumUtils.CountCharsInDouble(incrByFloat, out var _, out var _, out var _); + fieldInfo.ValueSize = NumUtils.CountCharsInDouble(incrByFloat, out var _, out var _, out var _); return fieldInfo; default: if (cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)cmd); - fieldInfo.ValueDataSize = functions.GetInitialLength(ref input); + fieldInfo.ValueSize = functions.GetInitialLength(ref input); } else - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; fieldInfo.HasETag = input.header.CheckWithETagFlag(); fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; @@ -197,8 +197,8 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe { var fieldInfo = new RecordFieldInfo() { - KeyDataSize = srcLogRecord.Key.Length, - ValueDataSize = 0, + KeySize = srcLogRecord.Key.Length, + ValueSize = 0, HasETag = input.header.CheckWithETagFlag() || srcLogRecord.Info.HasETag, HasExpiration = srcLogRecord.Info.HasExpiration }; @@ -214,7 +214,7 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe var incrByValue = input.header.cmd == RespCommand.INCRBY ? input.arg1 : 1; var value = srcLogRecord.ValueSpan; - fieldInfo.ValueDataSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead) + fieldInfo.ValueSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead) // TODO set error as in PrivateMethods.IsValidNumber and test in caller, to avoid the log record allocation. This would require 'output' if (srcLogRecord.IsPinnedValue ? IsValidNumber(srcLogRecord.PinnedValuePointer, value.Length, out _) : IsValidNumber(value, out _)) { @@ -222,7 +222,7 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe var curr = NumUtils.ReadInt64(value); var next = curr + incrByValue; - fieldInfo.ValueDataSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); + fieldInfo.ValueSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); } return fieldInfo; @@ -231,78 +231,78 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe var decrByValue = input.header.cmd == RespCommand.DECRBY ? input.arg1 : 1; value = srcLogRecord.ValueSpan; - fieldInfo.ValueDataSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead). + fieldInfo.ValueSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead). if (srcLogRecord.IsPinnedValue ? IsValidNumber(srcLogRecord.PinnedValuePointer, value.Length, out _) : IsValidNumber(value, out _)) { var curr = NumUtils.ReadInt64(value); var next = curr - decrByValue; - fieldInfo.ValueDataSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); + fieldInfo.ValueSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); } return fieldInfo; case RespCommand.INCRBYFLOAT: var incrByFloat = BitConverter.Int64BitsToDouble(input.arg1); value = srcLogRecord.ValueSpan; - fieldInfo.ValueDataSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead) + fieldInfo.ValueSize = 2; // # of digits in "-1", in case of invalid number (which may throw instead) if (srcLogRecord.IsPinnedValue ? IsValidDouble(srcLogRecord.PinnedValuePointer, value.Length, out _) : IsValidDouble(value, out _)) { _ = NumUtils.TryReadDouble(srcLogRecord.ValueSpan, out var currVal); var nextVal = currVal + incrByFloat; - fieldInfo.ValueDataSize = NumUtils.CountCharsInDouble(nextVal, out _, out _, out _); + fieldInfo.ValueSize = NumUtils.CountCharsInDouble(nextVal, out _, out _, out _); } return fieldInfo; case RespCommand.SETBIT: var bOffset = input.arg1; - fieldInfo.ValueDataSize = BitmapManager.NewBlockAllocLength(srcLogRecord.ValueSpan.Length, bOffset); + fieldInfo.ValueSize = BitmapManager.NewBlockAllocLength(srcLogRecord.ValueSpan.Length, bOffset); return fieldInfo; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - fieldInfo.ValueDataSize = BitmapManager.NewBlockAllocLengthFromType(bitFieldArgs, srcLogRecord.ValueSpan.Length); + fieldInfo.ValueSize = BitmapManager.NewBlockAllocLengthFromType(bitFieldArgs, srcLogRecord.ValueSpan.Length); return fieldInfo; case RespCommand.PFADD: // TODO: call HyperLogLog.DefaultHLL.IsValidHYLL and check error return per RMWMethods. This would require 'output'. Also carry this result through to RMWMethods. if (srcLogRecord.IsPinnedValue) - fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.UpdateGrow(ref input, srcLogRecord.PinnedValuePointer); + fieldInfo.ValueSize = HyperLogLog.DefaultHLL.UpdateGrow(ref input, srcLogRecord.PinnedValuePointer); else fixed (byte* valuePtr = srcLogRecord.ValueSpan) - fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.UpdateGrow(ref input, valuePtr); + fieldInfo.ValueSize = HyperLogLog.DefaultHLL.UpdateGrow(ref input, valuePtr); return fieldInfo; case RespCommand.PFMERGE: // TODO: call HyperLogLog.DefaultHLL.IsValidHYLL and check error return per RMWMethods. This would require 'output'. Also carry this result through to RMWMethods. var srcHLL = input.parseState.GetArgSliceByRef(0).ToPointer(); if (srcLogRecord.IsPinnedValue) - fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.MergeGrow(srcHLL, srcLogRecord.PinnedValuePointer); + fieldInfo.ValueSize = HyperLogLog.DefaultHLL.MergeGrow(srcHLL, srcLogRecord.PinnedValuePointer); else fixed (byte* dstHLL = srcLogRecord.ValueSpan) - fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.MergeGrow(srcHLL, dstHLL); + fieldInfo.ValueSize = HyperLogLog.DefaultHLL.MergeGrow(srcHLL, dstHLL); return fieldInfo; case RespCommand.SETKEEPTTLXX: case RespCommand.SETKEEPTTL: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; return fieldInfo; case RespCommand.SET: case RespCommand.SETEXXX: case RespCommand.SETEXNX: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; case RespCommand.PERSIST: fieldInfo.HasExpiration = false; - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; return fieldInfo; case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; fieldInfo.HasETag = true; fieldInfo.HasExpiration = input.arg1 != 0 || srcLogRecord.Info.HasExpiration; return fieldInfo; @@ -344,20 +344,20 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe } } } - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; return fieldInfo; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - fieldInfo.ValueDataSize = newValue.Length + offset; - if (fieldInfo.ValueDataSize < srcLogRecord.ValueSpan.Length) - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = newValue.Length + offset; + if (fieldInfo.ValueSize < srcLogRecord.ValueSpan.Length) + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; return fieldInfo; case RespCommand.GETEX: - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; // If both EX and PERSIST were specified, EX wins if (input.arg1 > 0) @@ -371,7 +371,7 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe return fieldInfo; case RespCommand.APPEND: - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length + input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length + input.parseState.GetArgSliceByRef(0).Length; return fieldInfo; case RespCommand.GETDEL: @@ -383,7 +383,7 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe if (cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)cmd); - fieldInfo.ValueDataSize = functions.GetLength(srcLogRecord.ValueSpan, ref input); + fieldInfo.ValueSize = functions.GetLength(srcLogRecord.ValueSpan, ref input); fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; } @@ -391,7 +391,7 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe } } - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; } @@ -400,8 +400,8 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan { }); + logRecord.ClearValueIfHeap(obj => { }); return true; } } diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index aeebca56df0..ea2f8d9c898 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -87,7 +87,7 @@ public void PostInitialUpdater(ref LogRecord dstLogRecord, in RecordSizeInfo siz WriteLogRMW(dstLogRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); } - functionsState.objectStoreSizeTracker?.AddTrackedSize(dstLogRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(dstLogRecord.ValueObject.HeapMemorySize); } /// @@ -118,11 +118,11 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r // Expired data if (logRecord.Info.HasExpiration && input.header.CheckExpiry(logRecord.Expiration)) { - functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.HeapMemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. - functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); - logRecord.ClearValueObject(obj => { }); + functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Expired); + logRecord.ClearValueIfHeap(obj => { }); rmwInfo.Action = input.header.type == GarnetObjectType.DelIfExpIm ? RMWAction.ExpireAndStop : RMWAction.ExpireAndResume; return false; } @@ -139,12 +139,11 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r return true; if (output.HasRemoveKey) { - functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.HeapMemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); - logRecord.ClearValueObject(obj => { }); - + logRecord.ClearValueIfHeap(obj => { }); rmwInfo.Action = RMWAction.ExpireAndStop; return false; } @@ -202,6 +201,10 @@ public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref rmwInfo.Action = RMWAction.ExpireAndResume; return false; } + // Defer the actual copying of data to PostCopyUpdater, so we know the record has been successfully CASed into the hash chain before we potentially + // create large allocations (e.g. if srcLogRecord is from disk, we would have to allocate the overflow byte[]). Because we are doing an update we have + // and XLock, so nobody will see the unset data even after the CAS. Tsavorite will handle cloning the ValueObject and caching serialized data as needed, + // based on whether srcLogRecord is in-memory or a DiskLogRecord. return true; } @@ -211,13 +214,12 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, { // We're performing the object update here (and not in CopyUpdater) so that we are guaranteed that // the record was CASed into the hash chain before it gets modified - var oldValueSize = srcLogRecord.ValueObject.MemorySize; - var value = ((IGarnetObject)srcLogRecord.ValueObject).CopyUpdate(srcLogRecord.Info.IsInNewVersion, ref rmwInfo); + var value = Unsafe.As(srcLogRecord.ValueObject.Clone()); + var oldValueSize = srcLogRecord.ValueObject.HeapMemorySize; + _ = dstLogRecord.TrySetValueObject(value); - // First copy the new Value and optionals to the new record. This will also ensure space for expiration if it's present. // Do not set actually set dstLogRecord.Expiration until we know it is a command for which we allocated length in the LogRecord for it. - if (!dstLogRecord.TrySetValueObject(value, in sizeInfo)) - return false; + // TODO: Object store ETags functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); @@ -264,7 +266,7 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, sizeInfo.AssertOptionals(dstLogRecord.Info); // If oldValue has been set to null, subtract its size from the tracked heap size - var sizeAdjustment = rmwInfo.ClearSourceValueObject ? value.MemorySize - oldValueSize : value.MemorySize; + var sizeAdjustment = rmwInfo.ClearSourceValueObject ? value.HeapMemorySize - oldValueSize : value.HeapMemorySize; functionsState.objectStoreSizeTracker?.AddTrackedSize(sizeAdjustment); if (functionsState.appendOnlyFile != null) diff --git a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs index 6e03893779e..843b8e39a39 100644 --- a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs @@ -29,7 +29,7 @@ public bool Reader(in TSourceLogRecord srcLogRecord, ref Objec switch (input.header.type) { case GarnetObjectType.Migrate: - DiskLogRecord.Serialize(in srcLogRecord, functionsState.garnetObjectSerializer, ref output.SpanByteAndMemory, functionsState.memoryPool); + DiskLogRecord.Serialize(in srcLogRecord, functionsState.garnetObjectSerializer, functionsState.memoryPool, ref output.SpanByteAndMemory); return true; case GarnetObjectType.Ttl: var ttlValue = ConvertUtils.SecondsFromDiffUtcNowTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); diff --git a/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs b/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs index db2e93900a9..a3772fd2d46 100644 --- a/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs @@ -93,7 +93,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r { var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TrySetValueSpan(srcValue, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -118,7 +118,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TrySetValueObject(srcValue, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -130,7 +130,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r if (functionsState.appendOnlyFile != null) WriteLogUpsert(logRecord.Key, ref input, garnetObject, upsertInfo.Version, upsertInfo.SessionID); - functionsState.objectStoreSizeTracker?.AddTrackedSize(srcValue.MemorySize - oldSize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(srcValue.HeapMemorySize - oldSize); return true; } @@ -140,7 +140,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSi { var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -159,7 +159,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSi var newSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); functionsState.objectStoreSizeTracker?.AddTrackedSize(newSize - oldSize); return true; } diff --git a/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs index e25362d6e5e..92f38a1bc57 100644 --- a/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs @@ -16,8 +16,8 @@ public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref Object { return new RecordFieldInfo() { - KeyDataSize = key.Length, - ValueDataSize = ObjectIdMap.ObjectIdSize, + KeySize = key.Length, + ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true, HasETag = false // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag() // No object commands take an Expiration for InitialUpdater. @@ -30,8 +30,8 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe { return new RecordFieldInfo() { - KeyDataSize = srcLogRecord.Key.Length, - ValueDataSize = ObjectIdMap.ObjectIdSize, + KeySize = srcLogRecord.Key.Length, + ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true, HasETag = false, // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag(), HasExpiration = srcLogRecord.Info.HasExpiration @@ -42,8 +42,8 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan key, IHeapObject va { return new RecordFieldInfo() { - KeyDataSize = key.Length, - ValueDataSize = ObjectIdMap.ObjectIdSize, + KeySize = key.Length, + ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true, HasETag = false // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag() // No object commands take an Expiration for Upsert. @@ -67,8 +67,8 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan k { return new RecordFieldInfo() { - KeyDataSize = key.Length, - ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, + KeySize = key.Length, + ValueSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = true, HasETag = false // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag() // No object commands take an Expiration for Upsert. diff --git a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs index d3e5943e349..223bb1f8a37 100644 --- a/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/DeleteMethods.cs @@ -36,7 +36,7 @@ public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); - logRecord.ClearValueObject(obj => { }); + logRecord.ClearValueIfHeap(obj => { }); } return true; diff --git a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs index f2777c2d984..83d2fd99d27 100644 --- a/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/RMWMethods.cs @@ -3,6 +3,7 @@ using System; using System.Diagnostics; +using System.Runtime.CompilerServices; using Tsavorite.core; namespace Garnet.server @@ -55,7 +56,7 @@ public void PostInitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeIn if (logRecord.Info.ValueIsObject) { - functionsState.objectStoreSizeTracker?.AddTrackedSize(logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(logRecord.ValueObject.HeapMemorySize); } } @@ -80,7 +81,14 @@ public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref return false; } - if (srcLogRecord.Info.ValueIsObject) return true; + if (srcLogRecord.Info.ValueIsObject) + { + // Defer the actual copying of data to PostCopyUpdater, so we know the record has been successfully CASed into the hash chain before we potentially + // create large allocations (e.g. if srcLogRecord is from disk, we would have to allocate the overflow byte[]). Because we are doing an update we have + // and XLock, so nobody will see the unset data even after the CAS. Tsavorite will handle cloning the ValueObject and caching serialized data as needed, + // based on whether srcLogRecord is in-memory or a DiskLogRecord. + return true; + } var recordHadEtagPreMutation = srcLogRecord.Info.HasETag; var shouldUpdateEtag = recordHadEtagPreMutation; @@ -127,9 +135,9 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, { // We're performing the object update here (and not in CopyUpdater) so that we are guaranteed that // the record was CASed into the hash chain before it gets modified - var oldValueSize = srcLogRecord.ValueObject.MemorySize; - var value = ((IGarnetObject)srcLogRecord.ValueObject).CopyUpdate(srcLogRecord.Info.IsInNewVersion, - ref rmwInfo); + var value = Unsafe.As(srcLogRecord.ValueObject.Clone()); + var oldValueSize = srcLogRecord.ValueObject.HeapMemorySize; + _ = dstLogRecord.TrySetValueObject(value); // First copy the new Value and optionals to the new record. This will also ensure space for expiration if it's present. // Do not set actually set dstLogRecord.Expiration until we know it is a command for which we allocated length in the LogRecord for it. @@ -168,7 +176,7 @@ public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, sizeInfo.AssertOptionals(dstLogRecord.Info); // If oldValue has been set to null, subtract its size from the tracked heap size - var sizeAdjustment = rmwInfo.ClearSourceValueObject ? value.MemorySize - oldValueSize : value.MemorySize; + var sizeAdjustment = rmwInfo.ClearSourceValueObject ? value.HeapMemorySize - oldValueSize : value.HeapMemorySize; functionsState.objectStoreSizeTracker?.AddTrackedSize(sizeAdjustment); } @@ -204,11 +212,11 @@ bool InPlaceUpdaterWorker(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r { if (logRecord.Info.ValueIsObject) { - functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.HeapMemorySize); // Can't access 'this' in a lambda so dispose directly and pass a no-op lambda. functionsState.storeFunctions.DisposeValueObject(logRecord.ValueObject, DisposeReason.Deleted); - logRecord.ClearValueObject(_ => { }); + logRecord.ClearValueIfHeap(_ => { }); } else { diff --git a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs index 2ee711b8914..479bbff9c74 100644 --- a/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/ReadMethods.cs @@ -44,15 +44,15 @@ private bool HandleMemoryUsage(in TSourceLogRecord srcLogRecor long memoryUsage; if (srcLogRecord.Info.ValueIsObject) { - memoryUsage = RecordInfo.GetLength() + (2 * IntPtr.Size) + // Log record length + memoryUsage = RecordInfo.Size + (2 * IntPtr.Size) + // Log record length Utility.RoundUp(srcLogRecord.Key.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + // Key allocation in heap with overhead - srcLogRecord.ValueObject.MemorySize; // Value allocation in heap + srcLogRecord.ValueObject.SerializedSize; // Value allocation in heap } else { - memoryUsage = RecordInfo.GetLength() + - Utility.RoundUp(srcLogRecord.Key.TotalSize(), RecordInfo.GetLength()) + - Utility.RoundUp(srcLogRecord.ValueSpan.TotalSize(), RecordInfo.GetLength()); + memoryUsage = RecordInfo.Size + + Utility.RoundUp(srcLogRecord.Key.TotalSize(), RecordInfo.Size) + + Utility.RoundUp(srcLogRecord.ValueSpan.TotalSize(), RecordInfo.Size); } using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output.SpanByteAndMemory); diff --git a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs index 355bad98e17..40f09882516 100644 --- a/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/UpsertMethods.cs @@ -111,7 +111,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r { var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TrySetValueSpan(newValue, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -165,7 +165,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TrySetValueObject(newValue, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -177,7 +177,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, r if (functionsState.appendOnlyFile != null) WriteLogUpsert(logRecord.Key, ref input, garnetObject, upsertInfo.Version, upsertInfo.SessionID); - functionsState.objectStoreSizeTracker?.AddTrackedSize(newValue.MemorySize - oldSize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(newValue.HeapMemorySize - oldSize); return true; } @@ -187,7 +187,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSi { var oldSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); _ = logRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) @@ -206,7 +206,7 @@ public bool InPlaceWriter(ref LogRecord logRecord, in RecordSi var newSize = logRecord.Info.ValueIsInline ? 0 - : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.HeapMemorySize); functionsState.objectStoreSizeTracker?.AddTrackedSize(newSize - oldSize); return true; } diff --git a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs index 6c4c4a40696..95be17d3517 100644 --- a/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/UnifiedStore/VarLenInputMethods.cs @@ -16,8 +16,8 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe { var fieldInfo = new RecordFieldInfo { - KeyDataSize = srcLogRecord.Key.Length, - ValueDataSize = srcLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : 0, + KeySize = srcLogRecord.Key.Length, + ValueSize = srcLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : 0, ValueIsObject = srcLogRecord.Info.ValueIsObject, HasETag = !srcLogRecord.Info.ValueIsObject && (input.header.CheckWithETagFlag() || srcLogRecord.Info.HasETag), HasExpiration = srcLogRecord.Info.HasExpiration @@ -67,19 +67,19 @@ public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRe } if (!srcLogRecord.Info.ValueIsObject) - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; return fieldInfo; case RespCommand.PERSIST: fieldInfo.HasExpiration = false; if (!srcLogRecord.Info.ValueIsObject) - fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + fieldInfo.ValueSize = srcLogRecord.ValueSpan.Length; return fieldInfo; default: throw new NotImplementedException(); } } - fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.ValueSize = input.parseState.GetArgSliceByRef(0).Length; fieldInfo.HasExpiration = input.arg1 != 0; return fieldInfo; } @@ -88,8 +88,8 @@ public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref Unifie { return new RecordFieldInfo { - KeyDataSize = key.Length, - ValueDataSize = 0, + KeySize = key.Length, + ValueSize = 0, HasETag = input.header.CheckWithETagFlag() }; } @@ -99,8 +99,8 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan key, IHeapObject va { return new RecordFieldInfo { - KeyDataSize = key.Length, - ValueDataSize = ObjectIdMap.ObjectIdSize, + KeySize = key.Length, + ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true, HasETag = false }; @@ -123,8 +123,8 @@ public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan k { return new RecordFieldInfo { - KeyDataSize = key.Length, - ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, + KeySize = key.Length, + ValueSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject, HasETag = !inputLogRecord.Info.ValueIsObject && input.header.CheckWithETagFlag() }; diff --git a/libs/server/Storage/Session/Common/MemoryUtils.cs b/libs/server/Storage/Session/Common/MemoryUtils.cs index 69c4aa45b4d..3eeae78541a 100644 --- a/libs/server/Storage/Session/Common/MemoryUtils.cs +++ b/libs/server/Storage/Session/Common/MemoryUtils.cs @@ -53,7 +53,7 @@ internal static long CalculateHeapMemorySize(in LogRecord logRecord) if (logRecord.Info.ValueIsOverflow) result += Utility.RoundUp(logRecord.ValueSpan.Length, IntPtr.Size) + ByteArrayOverhead; else if (logRecord.Info.ValueIsObject) - result += logRecord.ValueObject.MemorySize; + result += logRecord.ValueObject.HeapMemorySize; return result; } } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index 97c6711fc88..1b8917fe0ab 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -319,7 +319,7 @@ public unsafe GarnetStatus DEL_Conditional(PinnedSpanByte key, ref Raw } // Deletions in RMW are done by expiring the record, hence we use expiration as the indicator of success. - if (status.Expired) + if (status.IsExpired) { incr_session_found(); return GarnetStatus.OK; diff --git a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs index 2e3e91e34a8..b5a1805c56f 100644 --- a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs +++ b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs @@ -60,7 +60,7 @@ public readonly long CalculateRecordSize(in TSourceLogRecord l { var value = logRecord.ValueObject; if (value != null) // ignore deleted values being evicted (they are accounted for by InPlaceDeleter) - size += value.MemorySize; + size += value.HeapMemorySize; } return size; } diff --git a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs index da27b144fb0..89f37cf740d 100644 --- a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs +++ b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs @@ -39,7 +39,7 @@ void SetupStore() { IndexSize = 1L << 26, LogDevice = logDevice - }, StoreFunctions.Create() + }, StoreFunctions.Create(new SpanByteComparer(), new SpanByteRecordDisposer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } diff --git a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs index 5ea43f82a6a..c8b4f85d8b3 100644 --- a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs +++ b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs @@ -37,7 +37,7 @@ void SetupStore() { IndexSize = 1L << 26, LogDevice = logDevice - }, StoreFunctions.Create() + }, StoreFunctions.Create(new SpanByteComparer(), new SpanByteRecordDisposer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs index ce7b5a413b3..dda897e09d4 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs @@ -12,9 +12,10 @@ namespace Tsavorite.benchmark { #pragma warning disable IDE0065 // Misplaced using directive - using StructStoreFunctions = StoreFunctions; + using FixedLenStoreFunctions = StoreFunctions; - internal class FixedLenYcsbBenchmark + internal class FixedLenYcsbBenchmark + where TAllocator : IAllocator { RevivificationSettings FixedLengthBins = new() { @@ -22,7 +23,7 @@ internal class FixedLenYcsbBenchmark [ new RevivificationBin() { - RecordSize = RecordInfo.GetLength() + 2 * (sizeof(int) + sizeof(long)), // We have "fixed length" for these integer bins, with long Key and Value + RecordSize = RecordInfo.Size + 2 * (sizeof(int) + sizeof(long)), // We have "fixed length" for these integer bins, with long Key and Value BestFitScanLimit = RevivificationBin.UseFirstFit } ] @@ -43,7 +44,7 @@ internal class FixedLenYcsbBenchmark readonly FixedLengthKey[] txn_keys_; readonly IDevice device; - readonly TsavoriteKV> store; + readonly TsavoriteKV> store; long idx_ = 0; long total_ops_done = 0; @@ -491,9 +492,6 @@ private void SetupYcsbSafeContext(int thread_idx) _ = bContext.CompletePending(true); } - - #region Load Data - internal static void CreateKeyVectors(TestLoader testLoader, out FixedLengthKey[] i_keys, out FixedLengthKey[] t_keys) { InitCount = YcsbConstants.kChunkSize * (testLoader.InitCount / YcsbConstants.kChunkSize); @@ -502,12 +500,10 @@ internal static void CreateKeyVectors(TestLoader testLoader, out FixedLengthKey[ i_keys = new FixedLengthKey[InitCount]; t_keys = new FixedLengthKey[TxnCount]; } + } - internal class KeySetter : IKeySetter - { - public void Set(FixedLengthKey[] vector, long idx, long value) => vector[idx].value = value; - } - - #endregion + internal class FixedLenYcsbKeySetter : IKeySetter + { + public void Set(FixedLengthKey[] vector, long idx, long value) => vector[idx].value = value; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs index 4c66d867020..d7250bb5c28 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs @@ -14,7 +14,7 @@ public struct FixedLengthKey [FieldOffset(0)] public long value; - public override string ToString() => "{ " + value + " }"; + public override readonly string ToString() => "{ " + value + " }"; // Only call this for stack-based structs, not the ones in the *_keys vectors public unsafe ReadOnlySpan AsReadOnlySpan() => new(Unsafe.AsPointer(ref this), sizeof(long)); diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs index 7532cf1adea..ac631bee2da 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs @@ -8,11 +8,14 @@ namespace Tsavorite.benchmark { + /// + /// A key in . + /// [StructLayout(LayoutKind.Explicit, Size = DataSize)] public struct KeySpanByte { - public const int DataSize = 12; - public const int TotalSize = DataSize + sizeof(int); + internal const int DataSize = 12; + internal const int TotalSize = DataSize + (sizeof(int)); /// The data of the key [FieldOffset(0)] @@ -22,18 +25,24 @@ public struct KeySpanByte /// This field is for kRecordAlignment of the key since Tsavorite no longer aligns key size (i.e. Value start) to . /// /// - /// Combined with the length prefix and followed by value - /// that is also prefixed with a length, the final record size is exactly aligned to two cache lines. - /// To illustrate why this is imporatant: during the conversion to , the change in key alignment was not correctly - /// accounted for; the record was 8 bytes shorter, and the next record's RecordInfo was in the final bytes of the previous record's cache line. - /// This resulted in about a 10% slowdown. + /// Combined with the varbyte length total of bytes, we get: + /// [RecordInfo varbytelength keydata valuedata] = [8 + 5 (NumIndicatorBytes + 2 1-byte lengths) + 12 + 100 (kValueDataSize)] = 125, so the final record size is + /// exactly aligned to two cache lines. To illustrate why this is important: during the conversion to , the change in key + /// alignment was not correctly accounted for; the record was 8 bytes shorter, and the next record's RecordInfo was in the final bytes of the previous record's + /// cache line. This resulted in about a 10% slowdown. /// [FieldOffset(sizeof(long))] - public int padding; + public int padding1; - // Only call this for stack-based structs, not the ones in the *_keys vectors + /// + /// Convert to string; Only call this for stack-based structs, not the ones in the *_keys vectors. + /// public override readonly string ToString() => "{ " + value + " }"; + /// + /// Represent the key as a + /// + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public unsafe ReadOnlySpan AsReadOnlySpan() => new(Unsafe.AsPointer(ref this), DataSize); } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs index 5eca13c6a59..259f2160056 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs @@ -5,20 +5,29 @@ //#define FIXED_SIZE_VALUE //#define FIXED_SIZE_VALUE_WITH_LOCK +using System; +using System.IO; using Tsavorite.core; namespace Tsavorite.benchmark { - public class ObjectValue : IHeapObject + public class ObjectValue : HeapObjectBase { public long value; - public long MemorySize { get => sizeof(int); set => throw new System.NotImplementedException("TestValueObject.MemorySize.set"); } - public long DiskSize { get => MemorySize; set => throw new System.NotImplementedException("TestValueObject.DiskSize.set"); } + public override string ToString() => value.ToString(); - public void Dispose() { } + public override void Dispose() { } - public override string ToString() => value.ToString(); + public override HeapObjectBase Clone() => throw new NotImplementedException(); + public override void DoSerialize(BinaryWriter writer) => throw new NotImplementedException(); + public override void WriteType(BinaryWriter writer, bool isNull) => throw new NotImplementedException(); + + public ObjectValue() + { + HeapMemorySize = sizeof(long); + SerializedSize = HeapMemorySize; + } public class Serializer : BinaryObjectSerializer { diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs index 2ca66308768..2bcbec1de54 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs @@ -38,7 +38,7 @@ internal class ObjectYcsbBenchmark long total_ops_done = 0; volatile bool done = false; - internal const int kValueDataSize = SpanByteYcsbBenchmark.kValueDataSize; + internal const int kValueDataSize = SpanByteYcsbConstants.kValueDataSize; internal ObjectYcsbBenchmark(FixedLengthKey[] i_keys_, FixedLengthKey[] t_keys_, TestLoader testLoader) { @@ -75,7 +75,7 @@ internal ObjectYcsbBenchmark(FixedLengthKey[] i_keys_, FixedLengthKey[] t_keys_, [ new RevivificationBin() { - RecordSize = RecordInfo.GetLength() + KeySpanByte.TotalSize + kValueDataSize + 8, // extra to ensure rounding up of value + RecordSize = RecordInfo.Size + KeySpanByte.TotalSize + kValueDataSize + 8, // extra to ensure rounding up of value NumberOfRecords = testLoader.Options.RevivBinRecordCount, BestFitScanLimit = RevivificationBin.UseFirstFit } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs index 74252ab45c9..d3fdecc1751 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs @@ -50,6 +50,10 @@ class Options HelpText = "#,#,#,#: Percentages of [(r)eads,(u)pserts,r(m)ws,(d)eletes] (summing to 100) operations in this run")] public IEnumerable RumdPercents { get; set; } + [Option("sba", Required = false, Default = false, + HelpText = "Use SpanByteAllocator (default is to use ObjectAllocator)")] + public bool UseSBA { get; set; } + [Option("reviv", Required = false, Default = RevivificationLevel.None, HelpText = "Revivification of tombstoned records:" + $"\n {nameof(RevivificationLevel.None)} = No revivification" + @@ -124,9 +128,10 @@ class Options public string GetOptionsString() { static string boolStr(bool value) => value ? "y" : "n"; - return $"b: {Benchmark}; d: {DistributionName.ToLower()}; n: {NumaStyle}; rumd: {string.Join(',', RumdPercents)}; reviv: {RevivificationLevel}; revivbinrecs: {RevivBinRecordCount};" + var allocator = UseSBA ? "sba" : "oa"; + return $"b: {Benchmark}; a: {allocator}; d: {DistributionName.ToLower()}; n: {NumaStyle}; rumd: {string.Join(',', RumdPercents)}; reviv: {RevivificationLevel}; revivbinrecs: {RevivBinRecordCount};" + $" revivfrac {RevivifiableFraction}; t: {ThreadCount}; i: {IterationCount}; ov: {boolStr(UseOverflowValues)}; obj: {boolStr(UseObjectValues)}; hp: {HashPacking};" - + $" sd: {boolStr(UseSmallData)}; sm: {boolStr(UseSmallMemoryLog)}; sy: {boolStr(UseSyntheticData)}; safectx: {boolStr(UseSafeContext)};" + + $" sd: {boolStr(UseSmallData)}; sm: {boolStr(UseSmallMemoryLog)}; synth: {boolStr(UseSyntheticData)}; safectx: {boolStr(UseSafeContext)};" + $" chkptms: {PeriodicCheckpointMilliseconds}; chkpttype: {(PeriodicCheckpointMilliseconds > 0 ? PeriodicCheckpointType.ToString() : "None")};" + $" chkptincr: {boolStr(PeriodicCheckpointTryIncremental)}"; } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs index 7c247f334d3..797d1112bae 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs @@ -3,9 +3,14 @@ using System; using System.Threading; +using Tsavorite.core; namespace Tsavorite.benchmark { +#pragma warning disable IDE0065 // Misplaced using directive + using FixedLenStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; + public class Program { const int kTrimResultCount = 3; // Use some high value like int.MaxValue to disable @@ -16,6 +21,9 @@ public static void Main(string[] args) if (testLoader.error) return; + // Output the options at the start, for easy verification (and to stop immediately if we forgot something...). + Console.WriteLine(testLoader.Options.GetOptionsString()); + TestStats testStats = new(testLoader.Options); testLoader.LoadData(); var options = testLoader.Options; // shortcut @@ -29,15 +37,29 @@ public static void Main(string[] args) switch (testLoader.BenchmarkType) { case BenchmarkType.FixedLen: + if (options.UseSBA) + { + var tester = new FixedLenYcsbBenchmark>(testLoader.init_keys, testLoader.txn_keys, testLoader); + testStats.AddResult(tester.Run(testLoader)); + tester.Dispose(); + } + else { - var tester = new FixedLenYcsbBenchmark(testLoader.init_keys, testLoader.txn_keys, testLoader); + var tester = new FixedLenYcsbBenchmark>(testLoader.init_keys, testLoader.txn_keys, testLoader); testStats.AddResult(tester.Run(testLoader)); tester.Dispose(); } break; case BenchmarkType.SpanByte: + if (options.UseSBA) + { + var tester = new SpanByteYcsbBenchmark>(testLoader.init_span_keys, testLoader.txn_span_keys, testLoader); + testStats.AddResult(tester.Run(testLoader)); + tester.Dispose(); + } + else { - var tester = new SpanByteYcsbBenchmark(testLoader.init_span_keys, testLoader.txn_span_keys, testLoader); + var tester = new SpanByteYcsbBenchmark>(testLoader.init_span_keys, testLoader.txn_span_keys, testLoader); testStats.AddResult(tester.Run(testLoader)); tester.Dispose(); } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs index 9d3abf511b0..b32603c71a7 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs @@ -7,138 +7,133 @@ using System.Runtime.InteropServices; using Tsavorite.core; +#pragma warning disable CS1591 // Missing XML comment for publicly visible type or member +#pragma warning disable IDE0130 // Namespace does not match folder structure + namespace Tsavorite.benchmark { public struct SessionFixedLenFunctions : ISessionFunctions { - public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) + public readonly void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) { } - public void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) + public readonly void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) { } // Read functions [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool Reader(in TSourceLogRecord srcLogRecord, ref Input input, ref Output output, ref ReadInfo readInfo) + public readonly bool Reader(in TSourceLogRecord srcLogRecord, ref Input input, ref Output output, ref ReadInfo readInfo) where TSourceLogRecord : ISourceLogRecord { output.value = srcLogRecord.ValueSpan.AsRef(); return true; } - public bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; + public readonly bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; - public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; + public readonly bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; // Upsert functions [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) + public readonly bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) { srcValue.CopyTo(logRecord.ValueSpan); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) - { - logRecord.TrySetValueObject(srcValue, in sizeInfo); - return true; - } + public readonly bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) + => logRecord.TrySetValueObject(srcValue, in sizeInfo); - public bool InitialWriter(ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + public readonly bool InitialWriter(ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord => true; // not used [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) + public readonly bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) { srcValue.CopyTo(logRecord.ValueSpan); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) - { - logRecord.TrySetValueObject(srcValue, in sizeInfo); - return true; - } + public readonly bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) + => logRecord.TrySetValueObject(srcValue, in sizeInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceWriter(ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + public readonly bool InPlaceWriter(ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord - { - return dstLogRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); - } + => dstLogRecord.TryCopyFrom(in inputLogRecord, in sizeInfo); // RMW functions [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) + public readonly bool InitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) { logRecord.ValueSpan.AsRef().value = input.value; return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfoo) + public readonly bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfoo) { logRecord.ValueSpan.AsRef().value = input.value; return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) + public readonly bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord { dstLogRecord.ValueSpan.AsRef().value = input.value; return true; } - public bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) + public readonly bool PostCopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => true; - public bool NeedInitialUpdate(ReadOnlySpan key, ref Input input, ref Output output, ref RMWInfo rmwInfo) => true; + public readonly bool NeedInitialUpdate(ReadOnlySpan key, ref Input input, ref Output output, ref RMWInfo rmwInfo) => true; - public void PostInitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) { } + public readonly void PostInitialUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) { } - public bool NeedCopyUpdate(in TSourceLogRecord srcLogRecord, ref Input input, ref Output output, ref RMWInfo rmwInfo) + public readonly bool NeedCopyUpdate(in TSourceLogRecord srcLogRecord, ref Input input, ref Output output, ref RMWInfo rmwInfo) where TSourceLogRecord : ISourceLogRecord => true; - public RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref Input input) + public readonly RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref Input input) where TSourceLogRecord : ISourceLogRecord => GetFieldInfo(); /// Initial expected length of value object when populated by RMW using given input - public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref Input input) => GetFieldInfo(); + public readonly RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref Input input) => GetFieldInfo(); /// Length of value object, when populated by Upsert using given value and input - public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref Input input) => GetFieldInfo(); + public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref Input input) => GetFieldInfo(); /// Length of value object, when populated by Upsert using given value and input public unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref Input input) - => new() { KeyDataSize = sizeof(FixedLengthKey), ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = sizeof(FixedLengthKey), ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; /// Length of value object, when populated by Upsert using given log record and input - public unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref Input input) + public readonly unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref Input input) where TSourceLogRecord : ISourceLogRecord => throw new NotImplementedException("GetUpsertFieldInfo(TSourceLogRecord)"); - static unsafe RecordFieldInfo GetFieldInfo() => new () { KeyDataSize = sizeof(FixedLengthKey), ValueDataSize = sizeof(FixedLengthValue) }; + static unsafe RecordFieldInfo GetFieldInfo() => new () { KeySize = sizeof(FixedLengthKey), ValueSize = sizeof(FixedLengthValue) }; - public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { } + public readonly void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { } - public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) { } + public readonly void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) { } - public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) { } + public readonly void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) { } - public void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + public readonly void PostInitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref Input input, in TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) where TSourceLogRecord : ISourceLogRecord { } - public void ConvertOutputToHeap(ref Input input, ref Output output) { } + public readonly void ConvertOutputToHeap(ref Input input, ref Output output) { } } static class StaticUtilities diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs index 9a6e2934dd4..67beb353d9e 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs @@ -14,7 +14,13 @@ namespace Tsavorite.benchmark #pragma warning disable IDE0065 // Misplaced using directive using SpanByteStoreFunctions = StoreFunctions; - internal class SpanByteYcsbBenchmark + internal static class SpanByteYcsbConstants + { + internal const int kValueDataSize = 100; + } + + internal class SpanByteYcsbBenchmark + where TAllocator : IAllocator { // Ensure sizes are aligned to chunk sizes static long InitCount; @@ -37,8 +43,6 @@ internal class SpanByteYcsbBenchmark long total_ops_done = 0; volatile bool done = false; - internal const int kValueDataSize = 96; // 100 minus 4-byte length prefix. - internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, TestLoader testLoader) { if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) @@ -72,7 +76,7 @@ internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, Tes [ new RevivificationBin() { - RecordSize = RecordInfo.GetLength() + KeySpanByte.TotalSize + kValueDataSize + 8, // extra to ensure rounding up of value + RecordSize = RecordInfo.Size + KeySpanByte.TotalSize + SpanByteYcsbConstants.kValueDataSize + 8, // extra to ensure rounding up of value NumberOfRecords = testLoader.Options.RevivBinRecordCount, BestFitScanLimit = RevivificationBin.UseFirstFit } @@ -107,7 +111,7 @@ internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, Tes } store = new(kvSettings - , StoreFunctions.Create() + , StoreFunctions.Create(new SpanByteComparer(), new SpanByteRecordDisposer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -133,9 +137,9 @@ private void RunYcsbUnsafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Span value = stackalloc byte[kValueDataSize]; - Span input = stackalloc byte[kValueDataSize]; - Span output = stackalloc byte[kValueDataSize]; + Span value = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; + Span input = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; + Span output = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); @@ -229,9 +233,9 @@ private void RunYcsbSafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Span value = stackalloc byte[kValueDataSize]; - Span input = stackalloc byte[kValueDataSize]; - Span output = stackalloc byte[kValueDataSize]; + Span value = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; + Span input = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; + Span output = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); @@ -431,7 +435,7 @@ private void SetupYcsbUnsafeContext(int thread_idx) var uContext = session.UnsafeContext; uContext.BeginUnsafe(); - Span value = stackalloc byte[kValueDataSize]; + Span value = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; try { @@ -474,7 +478,7 @@ private void SetupYcsbSafeContext(int thread_idx) using var session = store.NewSession(functions); var bContext = session.BasicContext; - Span value = stackalloc byte[kValueDataSize]; + Span value = stackalloc byte[SpanByteYcsbConstants.kValueDataSize]; for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; chunk_idx < InitCount; @@ -497,8 +501,6 @@ private void SetupYcsbSafeContext(int thread_idx) bContext.CompletePending(true); } - #region Load Data - internal static void CreateKeyVectors(TestLoader testLoader, out KeySpanByte[] i_keys, out KeySpanByte[] t_keys) { InitCount = YcsbConstants.kChunkSize * (testLoader.InitCount / YcsbConstants.kChunkSize); @@ -507,12 +509,9 @@ internal static void CreateKeyVectors(TestLoader testLoader, out KeySpanByte[] i i_keys = new KeySpanByte[InitCount]; t_keys = new KeySpanByte[TxnCount]; } - - internal class KeySetter : IKeySetter - { - public void Set(KeySpanByte[] vector, long idx, long value) => vector[idx].value = value; - } - - #endregion + } + internal class SpanByteYcsbKeySetter : IKeySetter + { + public void Set(KeySpanByte[] vector, long idx, long value) => vector[idx].value = value; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs index 995bcf4811c..449bc52eb7d 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs @@ -13,6 +13,10 @@ namespace Tsavorite.benchmark { +#pragma warning disable IDE0065 // Misplaced using directive + using FixedLenStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; + internal interface IKeySetter { void Set(TKey[] vector, long idx, long value); @@ -87,6 +91,11 @@ static bool verifyOption(bool isValid, string name, string info = null) Console.WriteLine($"Can only specify UseOverflowValues or UseObjectValues with BenchmarkType.Object"); return; } + if (Options.UseSBA && BenchmarkType == BenchmarkType.Object) + { + Console.WriteLine($"SpanByteAllocator is not supported with BenchmarkType.Object"); + return; + } ReadPercent = rumdPercents[0]; UpsertPercent = ReadPercent + rumdPercents[1]; RmwPercent = UpsertPercent + rumdPercents[2]; @@ -117,12 +126,18 @@ private void LoadDataThreadProc() switch (BenchmarkType) { case BenchmarkType.FixedLen: - FixedLenYcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); - LoadData(this, init_keys, txn_keys, new FixedLenYcsbBenchmark.KeySetter()); + if (Options.UseSBA) + FixedLenYcsbBenchmark>.CreateKeyVectors(this, out init_keys, out txn_keys); + else + FixedLenYcsbBenchmark>.CreateKeyVectors(this, out init_keys, out txn_keys); + LoadData(this, init_keys, txn_keys, new FixedLenYcsbKeySetter()); break; case BenchmarkType.SpanByte: - SpanByteYcsbBenchmark.CreateKeyVectors(this, out init_span_keys, out txn_span_keys); - LoadData(this, init_span_keys, txn_span_keys, new SpanByteYcsbBenchmark.KeySetter()); + if (Options.UseSBA) + SpanByteYcsbBenchmark>.CreateKeyVectors(this, out init_span_keys, out txn_span_keys); + else + SpanByteYcsbBenchmark>.CreateKeyVectors(this, out init_span_keys, out txn_span_keys); + LoadData(this, init_span_keys, txn_span_keys, new SpanByteYcsbKeySetter()); break; case BenchmarkType.Object: ObjectYcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestStats.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestStats.cs index a0cabf6d35b..adb6707aa9e 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestStats.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestStats.cs @@ -28,7 +28,7 @@ internal void AddResult((double ips, double ops, long tailAddress) result) { initsPerRun.Add(result.ips); opsPerRun.Add(result.ops); - tailAddresses.Add(LogAddress.AbsoluteAddress(result.tailAddress)); + tailAddresses.Add(result.tailAddress); } internal void ShowAllStats(AggregateType aggregateType, string discardMessage = "") diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs index b351c3d9160..443336520e7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs @@ -5,6 +5,7 @@ using System.Diagnostics; using System.IO; using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; using System.Threading; using System.Threading.Tasks; using Microsoft.Extensions.Logging; @@ -12,12 +13,14 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; + using static VarbyteLengthUtility; /// /// Base class for hybrid log memory allocator. Contains utility methods, some of which are not performance-critical so can be virtual. /// - public abstract partial class AllocatorBase : IDisposable + public abstract unsafe partial class AllocatorBase : IDisposable where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { @@ -32,8 +35,11 @@ public abstract partial class AllocatorBase : IDisp /// The fully-derived allocator struct wrapper (so calls on it are inlined rather than virtual) for this log. internal readonly TAllocator _wrapper; + /// The to hold the objects for transient instances. + internal ObjectIdMap transientObjectIdMap; + /// Sometimes it's useful to know this explicitly rather than rely on method overrides etc. - internal bool IsObjectAllocator = false; + internal bool IsObjectAllocator => transientObjectIdMap is not null; #region Protected size definitions /// Buffer size @@ -51,7 +57,7 @@ public abstract partial class AllocatorBase : IDisp /// Buffer size mask protected readonly int BufferSizeMask; - /// Aligned page size in bytes + /// Aligned (to sector size) page size in bytes protected readonly int AlignedPageSizeBytes; /// Total hybrid log size (bits) @@ -84,6 +90,10 @@ public abstract partial class AllocatorBase : IDisp /// ReadOnlyAddress offset from tail (currently page-aligned) protected long ReadOnlyAddressLagOffset; + /// Circular buffer definition + /// The long is actually a byte*, but storing as 'long' makes going through logicalAddress/physicalAddress translation more easily + protected long* pagePointers; + #endregion #region Public addresses @@ -159,8 +169,6 @@ public override string ToString() internal SectorAlignedBufferPool bufferPool; /// Address type for this hlog's records' - private long addressTypeMask; - /// Read cache eviction callback protected readonly Action EvictCallback = null; @@ -190,11 +198,9 @@ public override string ToString() #endregion #region Abstract and virtual methods - /// Initialize fully derived allocator - public abstract void Initialize(); - /// Write async to device /// + /// /// /// /// @@ -202,18 +208,16 @@ public override string ToString() /// /// /// - /// /// Start address of fuzzy region, which contains old and new version records (we use this to selectively flush only old-version records during snapshot checkpoint) - protected abstract void WriteAsyncToDevice(long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, PageAsyncFlushResult result, IDevice device, IDevice objectLogDevice, long[] localSegmentOffsets, long fuzzyStartLogicalAddress); - - /// Read objects to memory (async) - protected abstract unsafe void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default); + protected abstract void WriteAsyncToDevice(CircularDiskWriteBuffer flushBuffers, long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, + PageAsyncFlushResult result, IDevice device, IDevice objectLogDevice, long fuzzyStartLogicalAddress); /// Read page from device (async) - protected abstract void ReadAsync(ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device, IDevice objlogDevice); + protected abstract void ReadAsync(CircularDiskReadBuffer readBuffers, ulong alignedSourceAddress, IntPtr destinationPtr, uint aligned_read_length, + DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device); /// Write page to device (async) - protected abstract void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult); + protected abstract void WriteAsync(CircularDiskWriteBuffer flushBuffers, long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult); /// Flush checkpoint Delta to the Device [MethodImpl(MethodImplOptions.NoInlining)] @@ -224,6 +228,7 @@ internal virtual unsafe void AsyncFlushDeltaToDevice(long startAddress, long end var _completedSemaphore = new SemaphoreSlim(0); completedSemaphore = _completedSemaphore; + // If throttled, convert rest of the method into a truly async task run because issuing IO can take up synchronous time if (throttleCheckpointFlushDelayMs >= 0) _ = Task.Run(FlushRunner); else @@ -233,7 +238,7 @@ void FlushRunner() { long startPage = GetPage(startAddress); long endPage = GetPage(endAddress); - if (endAddress > _wrapper.GetStartLogicalAddress(endPage)) + if (endAddress > GetLogicalAddressOfStartOfPage(endPage)) endPage++; long prevEndPage = GetPage(prevEndAddress); @@ -241,19 +246,14 @@ void FlushRunner() int destOffset = 0; // We perform delta capture under epoch protection with page-wise refresh for latency reasons - bool epochTaken = false; - if (!epoch.ThisInstanceProtected()) - { - epochTaken = true; - epoch.Resume(); - } + bool epochTaken = epoch.ResumeIfNotProtected(); try { for (long p = startPage; p < endPage; p++) { // Check if we have the entire page safely available to process in memory - if (HeadAddress >= GetStartLogicalAddressOfPage(p) + PageSize) + if (HeadAddress >= GetLogicalAddressOfStartOfPage(p) + PageSize) continue; // All RCU pages need to be added to delta @@ -261,8 +261,8 @@ void FlushRunner() if ((p < prevEndPage || endAddress == prevEndAddress) && PageStatusIndicator[p % BufferSize].Dirty < version) continue; - var logicalAddress = GetStartLogicalAddressOfPage(p); - var physicalAddress = _wrapper.GetPhysicalAddress(logicalAddress); + var logicalAddress = GetLogicalAddressOfStartOfPage(p); + var physicalAddress = GetPhysicalAddress(logicalAddress); var endLogicalAddress = logicalAddress + PageSize; if (endAddress < endLogicalAddress) endLogicalAddress = endAddress; @@ -364,11 +364,11 @@ public virtual void Reset() /// Wraps when an allocator potentially has to interact with multiple devices protected virtual void TruncateUntilAddress(long toAddress) { - _ = Task.Run(() => device.TruncateUntilAddress(AbsoluteAddress(toAddress))); + _ = Task.Run(() => device.TruncateUntilAddress(toAddress)); } /// Wraps when an allocator potentially has to interact with multiple devices - protected virtual void TruncateUntilAddressBlocking(long toAddress) => device.TruncateUntilAddress(AbsoluteAddress(toAddress)); + protected virtual void TruncateUntilAddressBlocking(long toAddress) => device.TruncateUntilAddress(toAddress); /// Remove disk segment protected virtual void RemoveSegment(int segment) => device.RemoveSegment(segment); @@ -380,6 +380,17 @@ public virtual void Dispose() { disposed = true; + if (pagePointers is not null) + { + for (var ii = 0; ii < BufferSize; ii++) + { + if (pagePointers[ii] != 0) + NativeMemory.AlignedFree((void*)pagePointers[ii]); + } + NativeMemory.AlignedFree((void*)pagePointers); + pagePointers = null; + } + if (ownedEpoch) epoch.Dispose(); bufferPool.Free(); @@ -393,30 +404,6 @@ public virtual void Dispose() #endregion abstract and virtual methods - #region LogRecord functions - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal unsafe void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord, int maxInlineKeySize, ObjectIdMap objectIdMap) - { - Span keySpan; - if (key.Length <= maxInlineKeySize) - { - logRecord.InfoRef.SetKeyIsInline(); - keySpan = LogField.SetInlineDataLength(logRecord.KeyAddress, key.Length); - } - else - { - Debug.Assert(objectIdMap is not null, "Inconsistent setting of maxInlineKeySize with null objectIdMap"); - - // There is no "overflow" bit; the lack of "KeyIsInline" marks that. But if it's a revivified record, it may have KeyIsInline set, so clear that. - logRecord.InfoRef.ClearKeyIsInline(); - keySpan = LogField.SetOverflowAllocation(logRecord.KeyAddress, key.Length, objectIdMap); - } - key.CopyTo(keySpan); - } - - #endregion LogRecord functions - private protected void VerifyCompatibleSectorSize(IDevice device) { if (sectorSize % device.SectorSize != 0) @@ -428,8 +415,8 @@ internal unsafe void ApplyDelta(DeltaLog log, long startPage, long endPage, long { if (log == null) return; - long startLogicalAddress = _wrapper.GetStartLogicalAddress(startPage); - long endLogicalAddress = _wrapper.GetStartLogicalAddress(endPage); + long startLogicalAddress = GetLogicalAddressOfStartOfPage(startPage); + long endLogicalAddress = GetLogicalAddressOfStartOfPage(endPage); log.Reset(); while (log.GetNext(out long physicalAddress, out int entryLength, out var type)) @@ -506,18 +493,26 @@ internal void MarkPageAtomic(long logicalAddress, long version) MonotonicUpdate(ref PageStatusIndicator[pageIndex].Dirty, version, out _); } + /// + /// This writes data from a page (or pages) for allocators that support only inline data. + /// + /// The source address, aligned to start of allocator page + /// The destination address, aligned to start of allocator page + /// Number of bytes to be written, based on allocator page range + /// The callback for the operation + /// The callback state information, including information for the flush operation + /// The device to write to [MethodImpl(MethodImplOptions.NoInlining)] - internal void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDestinationAddress, uint numBytesToWrite, - DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult, - IDevice device) + internal void WriteInlinePageAsync(IntPtr alignedSourceAddress, ulong alignedDestinationAddress, uint numBytesToWrite, + DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult, IDevice device) { if (asyncResult.partial) { // Write only required bytes within the page - int aligned_start = (int)(asyncResult.fromAddress - GetStartLogicalAddressOfPage(asyncResult.page)); + int aligned_start = (int)(asyncResult.fromAddress - GetLogicalAddressOfStartOfPage(asyncResult.page)); aligned_start = (aligned_start / sectorSize) * sectorSize; - int aligned_end = (int)(asyncResult.untilAddress - GetStartLogicalAddressOfPage(asyncResult.page)); + int aligned_end = (int)(asyncResult.untilAddress - GetLogicalAddressOfStartOfPage(asyncResult.page)); aligned_end = (aligned_end + (sectorSize - 1)) & ~(sectorSize - 1); numBytesToWrite = (uint)(aligned_end - aligned_start); @@ -530,31 +525,34 @@ internal void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDes } } - internal long SetAddressType(long address) => address | addressTypeMask; - internal long GetReadOnlyAddressLagOffset() => ReadOnlyAddressLagOffset; protected readonly ILogger logger; /// Instantiate base allocator implementation [MethodImpl(MethodImplOptions.NoInlining)] - private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunctions, Func wrapperCreator, Action evictCallback, LightEpoch epoch, Action flushCallback, ILogger logger = null) + private protected AllocatorBase(LogSettings logSettings, TStoreFunctions storeFunctions, Func wrapperCreator, Action evictCallback, + LightEpoch epoch, Action flushCallback, ILogger logger = null, ObjectIdMap transientObjectIdMap = null) { this.storeFunctions = storeFunctions; _wrapper = wrapperCreator(this); + this.transientObjectIdMap = transientObjectIdMap; + // Validation - if (settings.PageSizeBits < LogSettings.kMinPageSizeBits || settings.PageSizeBits > LogSettings.kMaxPageSizeBits) - throw new TsavoriteException($"{nameof(settings.PageSizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxPageSizeBits}"); - if (settings.SegmentSizeBits < LogSettings.kMinSegmentSizeBits || settings.SegmentSizeBits > LogSettings.kMaxSegmentSizeBits) - throw new TsavoriteException($"{nameof(settings.SegmentSizeBits)} must be between {LogSettings.kMinSegmentSizeBits} and {LogSettings.kMaxSegmentSizeBits}"); - if (settings.MemorySizeBits != 0 && (settings.MemorySizeBits < LogSettings.kMinMemorySizeBits || settings.MemorySizeBits > LogSettings.kMaxMemorySizeBits)) - throw new TsavoriteException($"{nameof(settings.MemorySizeBits)} must be between {LogSettings.kMinMemorySizeBits} and {LogSettings.kMaxMemorySizeBits}, or may be 0 for ReadOnly TsavoriteLog"); - if (settings.MutableFraction < 0.0 || settings.MutableFraction > 1.0) - throw new TsavoriteException($"{nameof(settings.MutableFraction)} must be >= 0.0 and <= 1.0"); - if (settings.ReadCacheSettings is not null) + if (logSettings.PageSizeBits < LogSettings.kMinPageSizeBits || logSettings.PageSizeBits > LogSettings.kMaxPageSizeBits) + throw new TsavoriteException($"{nameof(logSettings.PageSizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxPageSizeBits}"); + if (logSettings.PageSizeBits < PageHeader.SizeBits) + throw new TsavoriteException($"{nameof(logSettings.PageSizeBits)} must be >= PageHeader.SizeBits {PageHeader.SizeBits}"); + if (logSettings.SegmentSizeBits < LogSettings.kMinMainLogSegmentSizeBits || logSettings.SegmentSizeBits > LogSettings.kMaxSegmentSizeBits) + throw new TsavoriteException($"{nameof(logSettings.SegmentSizeBits)} must be between {LogSettings.kMinMainLogSegmentSizeBits} and {LogSettings.kMaxSegmentSizeBits}"); + if (logSettings.MemorySizeBits != 0 && (logSettings.MemorySizeBits < LogSettings.kMinMemorySizeBits || logSettings.MemorySizeBits > LogSettings.kMaxMemorySizeBits)) + throw new TsavoriteException($"{nameof(logSettings.MemorySizeBits)} must be between {LogSettings.kMinMemorySizeBits} and {LogSettings.kMaxMemorySizeBits}, or may be 0 for ReadOnly TsavoriteLog"); + if (logSettings.MutableFraction < 0.0 || logSettings.MutableFraction > 1.0) + throw new TsavoriteException($"{nameof(logSettings.MutableFraction)} must be >= 0.0 and <= 1.0"); + if (logSettings.ReadCacheSettings is not null) { - var rcs = settings.ReadCacheSettings; + var rcs = logSettings.ReadCacheSettings; if (rcs.PageSizeBits < LogSettings.kMinPageSizeBits || rcs.PageSizeBits > LogSettings.kMaxPageSizeBits) throw new TsavoriteException($"{nameof(rcs.PageSizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxPageSizeBits}"); if (rcs.MemorySizeBits < LogSettings.kMinMemorySizeBits || rcs.MemorySizeBits > LogSettings.kMaxMemorySizeBits) @@ -563,23 +561,22 @@ private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunct throw new TsavoriteException($"{rcs.SecondChanceFraction} must be >= 0.0 and <= 1.0"); } - if (settings.MaxInlineKeySizeBits < LogSettings.kLowestMaxInlineSizeBits || settings.PageSizeBits > LogSettings.kMaxStringSizeBits - 1) - throw new TsavoriteException($"{nameof(settings.MaxInlineKeySizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxStringSizeBits - 1}"); - if (settings.MaxInlineValueSizeBits < LogSettings.kLowestMaxInlineSizeBits || settings.PageSizeBits > LogSettings.kMaxStringSizeBits - 1) - throw new TsavoriteException($"{nameof(settings.MaxInlineValueSizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxStringSizeBits - 1}"); + if (logSettings.MaxInlineKeySizeBits < LogSettings.kLowestMaxInlineSizeBits || logSettings.PageSizeBits > LogSettings.kMaxStringSizeBits - 1) + throw new TsavoriteException($"{nameof(logSettings.MaxInlineKeySizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxStringSizeBits - 1}"); + if (logSettings.MaxInlineValueSizeBits < LogSettings.kLowestMaxInlineSizeBits || logSettings.PageSizeBits > LogSettings.kMaxStringSizeBits - 1) + throw new TsavoriteException($"{nameof(logSettings.MaxInlineValueSizeBits)} must be between {LogSettings.kMinPageSizeBits} and {LogSettings.kMaxStringSizeBits - 1}"); this.logger = logger; - if (settings.LogDevice == null) + if (logSettings.LogDevice == null) throw new TsavoriteException("LogSettings.LogDevice needs to be specified (e.g., use Devices.CreateLogDevice, AzureStorageDevice, or NullDevice)"); EvictCallback = evictCallback; - addressTypeMask = GetLogAddressType(EvictCallback is not null); FlushCallback = flushCallback; - PreallocateLog = settings.PreallocateLog; + PreallocateLog = logSettings.PreallocateLog; FlushEvent.Initialize(); - IsNullDevice = settings.LogDevice is NullDevice; + IsNullDevice = logSettings.LogDevice is NullDevice; if (epoch == null) { @@ -589,24 +586,24 @@ private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunct else this.epoch = epoch; - settings.LogDevice.Initialize(1L << settings.SegmentSizeBits, epoch); - settings.ObjectLogDevice?.Initialize(-1, epoch); + logSettings.LogDevice.Initialize(1L << logSettings.SegmentSizeBits, epoch); + logSettings.ObjectLogDevice?.Initialize(-1, epoch); // Page size - LogPageSizeBits = settings.PageSizeBits; + LogPageSizeBits = logSettings.PageSizeBits; PageSize = 1 << LogPageSizeBits; PageSizeMask = PageSize - 1; // Total HLOG size - LogTotalSizeBits = settings.MemorySizeBits; + LogTotalSizeBits = logSettings.MemorySizeBits; LogTotalSizeBytes = 1L << LogTotalSizeBits; BufferSize = (int)(LogTotalSizeBytes / PageSize); BufferSizeMask = BufferSize - 1; - LogMutableFraction = settings.MutableFraction; + LogMutableFraction = logSettings.MutableFraction; // Segment size - LogSegmentSizeBits = settings.SegmentSizeBits; + LogSegmentSizeBits = logSettings.SegmentSizeBits; SegmentSize = 1L << LogSegmentSizeBits; SegmentBufferSize = 1 + (LogTotalSizeBytes / SegmentSize < 1 ? 1 : (int)(LogTotalSizeBytes / SegmentSize)); @@ -617,11 +614,11 @@ private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunct throw new TsavoriteException($"Memory size ({LogTotalSizeBytes}) must be at least twice the page size ({PageSize})"); // Readonlymode has MemorySizeBits 0 => skip the check - if (settings.MemorySizeBits > 0 && settings.MinEmptyPageCount > MaxEmptyPageCount) - throw new TsavoriteException($"MinEmptyPageCount ({settings.MinEmptyPageCount}) can't be more than MaxEmptyPageCount ({MaxEmptyPageCount})"); + if (logSettings.MemorySizeBits > 0 && logSettings.MinEmptyPageCount > MaxEmptyPageCount) + throw new TsavoriteException($"MinEmptyPageCount ({logSettings.MinEmptyPageCount}) can't be more than MaxEmptyPageCount ({MaxEmptyPageCount})"); - MinEmptyPageCount = settings.MinEmptyPageCount; - EmptyPageCount = settings.MinEmptyPageCount; + MinEmptyPageCount = logSettings.MinEmptyPageCount; + EmptyPageCount = logSettings.MinEmptyPageCount; PageStatusIndicator = new FullPageStatus[BufferSize]; @@ -631,15 +628,35 @@ private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunct for (int i = 0; i < BufferSize; i++) PendingFlush[i] = new PendingFlushList(); } - device = settings.LogDevice; + device = logSettings.LogDevice; sectorSize = (int)device.SectorSize; if (PageSize < sectorSize) throw new TsavoriteException($"Page size must be at least of device sector size ({sectorSize} bytes). Set PageSizeBits accordingly."); AlignedPageSizeBytes = RoundUp(PageSize, sectorSize); + + if (BufferSize > 0) + { + var bufferSizeInBytes = (nuint)RoundUp(sizeof(long*) * BufferSize, Constants.kCacheLineBytes); + pagePointers = (long*)NativeMemory.AlignedAlloc(bufferSizeInBytes, Constants.kCacheLineBytes); + NativeMemory.Clear(pagePointers, bufferSizeInBytes); + } } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal long GetPhysicalAddress(long logicalAddress) + { + // Index of page within the circular buffer, and offset on the page. + var pageIndex = GetPageIndexForAddress(logicalAddress); + var offset = GetOffsetOnPage(logicalAddress); + return *(pagePointers + pageIndex) + offset; + } + internal bool IsAllocated(int pageIndex) => pagePointers[pageIndex] != 0; + + internal void ClearPage(long page, int offset = 0) + => NativeMemory.Clear((byte*)pagePointers[page % BufferSize] + offset, (nuint)(PageSize - offset)); + [MethodImpl(MethodImplOptions.NoInlining)] internal void VerifyRecoveryInfo(HybridLogCheckpointInfo recoveredHLCInfo, bool trimLog = false) { @@ -667,7 +684,7 @@ internal void VerifyRecoveryInfo(HybridLogCheckpointInfo recoveredHLCInfo, bool var firstAvailSegment = device.StartSegment; var lastAvailSegment = device.EndSegment; - if (FlushedUntilAddress > _wrapper.GetFirstValidLogicalAddress(0)) + if (FlushedUntilAddress > GetFirstValidLogicalAddressOnPage(0)) { var flushedUntilAddress = FlushedUntilAddress; int currTailSegment = (int)GetSegment(flushedUntilAddress); @@ -701,23 +718,21 @@ internal void VerifyRecoveryInfo(HybridLogCheckpointInfo recoveredHLCInfo, bool /// Initialize allocator [MethodImpl(MethodImplOptions.NoInlining)] - protected void Initialize(long firstValidAddress) + protected internal virtual void Initialize() { - Debug.Assert(AbsoluteAddress(firstValidAddress) <= PageSize, $"AbsoluteAddress(firstValidAddress) {AbsoluteAddress(firstValidAddress)} should be <= PageSize {PageSize}"); - bufferPool ??= new SectorAlignedBufferPool(1, sectorSize); - firstValidAddress = SetAddressType(AbsoluteAddress(firstValidAddress)); // Initially InLogMemory but may need to be changed to InReadCache + var firstValidAddress = FirstValidAddress; if (BufferSize > 0) { long tailPage = GetPage(firstValidAddress); int tailPageIndex = GetPageIndexForPage(tailPage); - if (!_wrapper.IsAllocated(tailPageIndex)) + if (!IsAllocated(tailPageIndex)) _wrapper.AllocatePage(tailPageIndex); // Allocate next page as well int nextPageIndex = GetPageIndexForPage(tailPage + 1); - if (!_wrapper.IsAllocated(nextPageIndex)) + if (!IsAllocated(nextPageIndex)) _wrapper.AllocatePage(nextPageIndex); } @@ -725,7 +740,7 @@ protected void Initialize(long firstValidAddress) { for (int pageIndex = 0; pageIndex < BufferSize; pageIndex++) { - if (!_wrapper.IsAllocated(pageIndex)) + if (!IsAllocated(pageIndex)) _wrapper.AllocatePage(pageIndex); } } @@ -759,9 +774,7 @@ public int MinEmptyPageCount { minEmptyPageCount = value; if (emptyPageCount != minEmptyPageCount) - { EmptyPageCount = minEmptyPageCount; - } } } @@ -788,16 +801,14 @@ public int EmptyPageCount headOffsetLagSize -= emptyPageCount; // Address lag offsets correspond to the number of pages "behind" TailPageOffset (the tail in the circular buffer). - ReadOnlyAddressLagOffset = GetStartAbsoluteLogicalAddressOfPage((long)(LogMutableFraction * headOffsetLagSize)); - HeadAddressLagOffset = GetStartAbsoluteLogicalAddressOfPage(headOffsetLagSize); + ReadOnlyAddressLagOffset = GetLogicalAddressOfStartOfPage((long)(LogMutableFraction * headOffsetLagSize)); + HeadAddressLagOffset = GetLogicalAddressOfStartOfPage(headOffsetLagSize); } // Force eviction now if empty page count has increased if (value >= oldEPC) { - var prot = epoch.ThisInstanceProtected(); - - if (!prot) epoch.Resume(); + var epochTaken = epoch.ResumeIfNotProtected(); try { // These shifts adjust via application of the lag addresses. @@ -807,7 +818,8 @@ public int EmptyPageCount } finally { - if (!prot) epoch.Suspend(); + if (epochTaken) + epoch.Suspend(); } } } @@ -836,9 +848,8 @@ public long GetTailAddress() { var local = TailPageOffset; - // Handle corner cases during page overflow - // The while loop is guaranteed to terminate because HandlePageOverflow - // ensures that it fixes the unstable TailPageOffset immediately. + // Handle corner cases during page overflow. + // The while loop is guaranteed to terminate because HandlePageOverflow ensures that it fixes the unstable TailPageOffset immediately. while (local.Offset >= PageSize) { if (local.Offset == PageSize) @@ -848,15 +859,15 @@ public long GetTailAddress() break; } // Offset is being adjusted by overflow thread, spin-wait - Thread.Yield(); + _ = Thread.Yield(); local = TailPageOffset; } - return GetStartLogicalAddressOfPage(local.Page) | (uint)local.Offset; + return GetLogicalAddressOfStartOfPage(local.Page) | (uint)local.Offset; } /// Get page index from [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetPage(long logicalAddress) => LogAddress.GetPage(logicalAddress, LogPageSizeBits); + public long GetPage(long logicalAddress) => GetPageOfAddress(logicalAddress, LogPageSizeBits); /// Get page index for page [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -864,7 +875,7 @@ public long GetTailAddress() /// Get page index for address [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetPageIndexForAddress(long address) => GetPageIndexForPage(GetPage(address)); + public int GetPageIndexForAddress(long logicalAddress) => GetPageIndexForPage(LogAddress.GetPageOfAddress(logicalAddress, LogPageSizeBits)); /// Get capacity (number of pages) [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -876,39 +887,32 @@ public long GetTailAddress() /// Get logical address of start of page [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetAddressOfStartOfPage(long address) => address & ~PageSizeMask; + public long GetAddressOfStartOfPageOfAddress(long address) => address & ~PageSizeMask; /// Get offset in page [MethodImpl(MethodImplOptions.AggressiveInlining)] public long GetOffsetOnPage(long address) => address & PageSizeMask; - /// Get start absolute logical address - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetStartAbsoluteLogicalAddressOfPage(long page) => LogAddress.GetStartAbsoluteLogicalAddressOfPage(page, LogPageSizeBits); - - /// Get start logical address + /// Get start logical address; this is the 0'th byte on the page, i.e. the start; it is *not* a valid record address + /// (for that see ). [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetStartLogicalAddressOfPage(long page) => SetAddressType(GetStartAbsoluteLogicalAddressOfPage(page)); + public long GetLogicalAddressOfStartOfPage(long page) => LogAddress.GetLogicalAddressOfStartOfPage(page, LogPageSizeBits); - /// Get first valid address + /// Get first valid address on a page (which is the start of the page plus sizeof()). [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetFirstValidLogicalAddressOnPage(long page) => page == 0 ? FirstValidAddress : GetStartLogicalAddressOfPage(page); + public long GetFirstValidLogicalAddressOnPage(long page) => GetLogicalAddressOfStartOfPage(page) + FirstValidAddress; /// Get log segment index from [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetSegment(long logicalAddress) => AbsoluteAddress(logicalAddress) >> LogSegmentSizeBits; + public long GetSegment(long logicalAddress) => logicalAddress >> LogSegmentSizeBits; /// Get offset in page [MethodImpl(MethodImplOptions.AggressiveInlining)] public long GetOffsetOnSegment(long address) => address & (SegmentSize - 1); - /// Get start absolute logical address - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetStartAbsoluteLogicalAddressOfSegment(long segment) => segment << LogSegmentSizeBits; - /// Get start logical address [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetStartLogicalAddressOfSegment(long segment) => SetAddressType(GetStartAbsoluteLogicalAddressOfSegment(segment)); + public long GetStartLogicalAddressOfSegment(long segment) => segment << LogSegmentSizeBits; /// Get sector size for main hlog device [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -920,18 +924,18 @@ void AllocatePagesWithException(int pageIndex, PageOffset localTailPageOffset, i try { // Allocate this page, if needed - if (!_wrapper.IsAllocated(pageIndex % BufferSize)) + if (!IsAllocated(pageIndex % BufferSize)) _wrapper.AllocatePage(pageIndex % BufferSize); // Allocate next page in advance, if needed - if (!_wrapper.IsAllocated((pageIndex + 1) % BufferSize)) + if (!IsAllocated((pageIndex + 1) % BufferSize)) _wrapper.AllocatePage((pageIndex + 1) % BufferSize); } catch { // Reset to previous tail localTailPageOffset.PageAndOffset -= numSlots; - Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); + _ = Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); throw; } } @@ -955,8 +959,8 @@ static void ThrowTsavoriteException(string message) /// bool NeedToShiftAddress(long pageIndex, PageOffset localTailPageOffset, int numSlots) { - var tailAddress = GetStartLogicalAddressOfPage(localTailPageOffset.Page) | ((long)(localTailPageOffset.Offset - numSlots)); - var shiftAddress = GetStartLogicalAddressOfPage(pageIndex); + var tailAddress = GetLogicalAddressOfStartOfPage(localTailPageOffset.Page) | ((long)(localTailPageOffset.Offset - numSlots)); + var shiftAddress = GetLogicalAddressOfStartOfPage(pageIndex); // Check whether we need to shift ROA var desiredReadOnlyAddress = shiftAddress - ReadOnlyAddressLagOffset; @@ -985,24 +989,24 @@ bool NeedToShiftAddress(long pageIndex, PageOffset localTailPageOffset, int numS void IssueShiftAddress(long pageIndex) { // Issue the shift of address - var shiftAddress = GetStartLogicalAddressOfPage(pageIndex); + var shiftAddress = GetLogicalAddressOfStartOfPage(pageIndex); var tailAddress = GetTailAddress(); - long desiredReadOnlyAddress = shiftAddress - ReadOnlyAddressLagOffset; + var desiredReadOnlyAddress = shiftAddress - ReadOnlyAddressLagOffset; if (desiredReadOnlyAddress > tailAddress) desiredReadOnlyAddress = tailAddress; - ShiftReadOnlyAddress(desiredReadOnlyAddress); + _ = ShiftReadOnlyAddress(desiredReadOnlyAddress); - long desiredHeadAddress = shiftAddress - HeadAddressLagOffset; + var desiredHeadAddress = shiftAddress - HeadAddressLagOffset; if (desiredHeadAddress > tailAddress) desiredHeadAddress = tailAddress; - ShiftHeadAddress(desiredHeadAddress); + _ = ShiftHeadAddress(desiredHeadAddress); } [MethodImpl(MethodImplOptions.NoInlining)] long HandlePageOverflow(ref PageOffset localTailPageOffset, int numSlots) { - int pageIndex = localTailPageOffset.Page + 1; + var pageIndex = localTailPageOffset.Page + 1; // This thread is trying to allocate at an offset past where one or more previous threads // already overflowed; exit and allow the first overflow thread to proceed. Do not try to remove @@ -1022,7 +1026,7 @@ long HandlePageOverflow(ref PageOffset localTailPageOffset, int numSlots) { // Reset to previous tail so that next attempt can retry localTailPageOffset.PageAndOffset -= numSlots; - Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); + _ = Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); // Shift only after TailPageOffset is reset to a valid state IssueShiftAddress(pageIndex); @@ -1039,7 +1043,7 @@ long HandlePageOverflow(ref PageOffset localTailPageOffset, int numSlots) { // Reset to previous tail so that next attempt can retry localTailPageOffset.PageAndOffset -= numSlots; - Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); + _ = Interlocked.Exchange(ref TailPageOffset.PageAndOffset, localTailPageOffset.PageAndOffset); // Shift only after TailPageOffset is reset to a valid state IssueShiftAddress(pageIndex); @@ -1048,17 +1052,17 @@ long HandlePageOverflow(ref PageOffset localTailPageOffset, int numSlots) } // Allocate next page and set new tail - if (!_wrapper.IsAllocated(pageIndex % BufferSize) || !_wrapper.IsAllocated((pageIndex + 1) % BufferSize)) + if (!IsAllocated(pageIndex % BufferSize) || !IsAllocated((pageIndex + 1) % BufferSize)) AllocatePagesWithException(pageIndex, localTailPageOffset, numSlots); + // Set up the TailPageOffset to account for the page header and then this allocation. localTailPageOffset.Page++; - localTailPageOffset.Offset = numSlots; + localTailPageOffset.Offset = PageHeader.Size + numSlots; TailPageOffset = localTailPageOffset; // At this point the slot is allocated and we are not allowed to refresh epochs any longer. - - // Offset is zero for the first allocation on the new page - return GetStartLogicalAddressOfPage(localTailPageOffset.Page); + // Return the first logical address after the page header. + return GetFirstValidLogicalAddressOnPage(localTailPageOffset.Page); } /// Try allocate, no thread spinning allowed @@ -1086,6 +1090,7 @@ private long TryAllocate(int numSlots = 1) // it will see that another thread got there first because the subsequent "back up by numSlots" will still be past PageSize, // so they will exit and RETRY in HandlePageOverflow; the first thread "owns" the overflow operation and must stabilize it. localTailPageOffset.PageAndOffset = Interlocked.Add(ref TailPageOffset.PageAndOffset, numSlots); + Debug.Assert(localTailPageOffset.Offset >= PageHeader.Size, $"localTailPageOffset.Offset ({localTailPageOffset.Offset}) must be past PageHeader.Size ({PageHeader.Size})"); // Note: Below here we defer SetAddressType(..) to TryAllocateRetryNow, so 0 and -1 returns are preserved. @@ -1098,24 +1103,23 @@ private long TryAllocate(int numSlots = 1) return HandlePageOverflow(ref localTailPageOffset, numSlots); } - return GetStartLogicalAddressOfPage(localTailPageOffset.Page) | ((long)(localTailPageOffset.Offset - numSlots)); + return GetLogicalAddressOfStartOfPage(localTailPageOffset.Page) | ((long)(localTailPageOffset.Offset - numSlots)); } /// Try allocate, spin for RETRY_NOW (logicalAddress is less than 0) case /// Number of slots to allocate - /// The allocated logical address, or 0 in case of inability to allocate + /// Returned address, or RETRY_LATER (if 0) indicator + /// True if we were able to allocate, else false [MethodImpl(MethodImplOptions.AggressiveInlining)] public bool TryAllocateRetryNow(int numSlots, out long logicalAddress) { - long address; - while ((address = TryAllocate(numSlots)) < 0) + while ((logicalAddress = TryAllocate(numSlots)) < 0) { _ = TryComplete(); epoch.ProtectAndDrain(); _ = Thread.Yield(); } - logicalAddress = SetAddressType(address); - return address != 0; + return logicalAddress != 0; } /// @@ -1221,8 +1225,8 @@ public void ShiftBeginAddress(long newBeginAddress, bool truncateLog, bool noFlu /// Invokes eviction observer if set and then frees the page. internal virtual void EvictPage(long page) { - var start = GetStartLogicalAddressOfPage(page); - var end = GetStartLogicalAddressOfPage(page + 1); + var start = GetLogicalAddressOfStartOfPage(page); + var end = GetLogicalAddressOfStartOfPage(page + 1); if (OnEvictionObserver is not null) MemoryPageScan(start, end, OnEvictionObserver); _wrapper.FreePage(page); @@ -1234,7 +1238,7 @@ internal virtual void EvictPage(long page) /// private void OnPagesMarkedReadOnly(long newSafeReadOnlyAddress, bool noFlush = false) { - if (MonotonicUpdate(ref SafeReadOnlyAddress, newSafeReadOnlyAddress, out long oldSafeReadOnlyAddress)) + if (MonotonicUpdate(ref SafeReadOnlyAddress, newSafeReadOnlyAddress, out var oldSafeReadOnlyAddress)) { // Debug.WriteLine("SafeReadOnly shifted from {0:X} to {1:X}", oldSafeReadOnlyAddress, newSafeReadOnlyAddress); if (OnReadOnlyObserver != null) @@ -1244,7 +1248,7 @@ private void OnPagesMarkedReadOnly(long newSafeReadOnlyAddress, bool noFlush = f using var iter = Scan(store: null, oldSafeReadOnlyAddress, newSafeReadOnlyAddress, DiskScanBufferingMode.NoBuffering); OnReadOnlyObserver?.OnNext(iter); } - AsyncFlushPages(oldSafeReadOnlyAddress, newSafeReadOnlyAddress, noFlush); + AsyncFlushPagesForReadOnly(oldSafeReadOnlyAddress, newSafeReadOnlyAddress, noFlush); } } @@ -1257,7 +1261,7 @@ private void OnPagesClosed(long newSafeHeadAddress) // This thread is responsible for [oldSafeHeadAddress -> newSafeHeadAddress] while (true) { - long _ongoingCloseUntilAddress = OngoingCloseUntilAddress; + var _ongoingCloseUntilAddress = OngoingCloseUntilAddress; // If we are closing in the middle of an ongoing OPCWorker loop, exit. if (_ongoingCloseUntilAddress >= newSafeHeadAddress) @@ -1293,8 +1297,10 @@ private void OnPagesClosedWorker() if (EvictCallback is not null) EvictCallback(closeStartAddress, closeEndAddress); - for (long closePageAddress = GetAddressOfStartOfPage(closeStartAddress); closePageAddress < closeEndAddress; closePageAddress += PageSize) + // Process a page (possibly fragment) at a time. + for (long closePageAddress = GetAddressOfStartOfPageOfAddress(closeStartAddress); closePageAddress < closeEndAddress; closePageAddress += PageSize) { + // Get the range on this page: the start may be 0 or greater, and the end may be end-of-page or less. long start = closeStartAddress > closePageAddress ? closeStartAddress : closePageAddress; long end = closeEndAddress < closePageAddress + PageSize ? closeEndAddress : closePageAddress + PageSize; @@ -1303,7 +1309,7 @@ private void OnPagesClosedWorker() if (OnEvictionObserver is not null) MemoryPageScan(start, end, OnEvictionObserver); - // If we are using a null storage device, we must also shift BeginAddress + // If we are using a null storage device, we must also shift BeginAddress (leave it in-memory) if (IsNullDevice) _ = MonotonicUpdate(ref BeginAddress, end, out _); @@ -1348,9 +1354,7 @@ private void DebugPrintAddresses() /// private void PageAlignedShiftReadOnlyAddress(long currentTailAddress) { - long desiredReadOnlyAddress = GetAddressOfStartOfPage(currentTailAddress) - ReadOnlyAddressLagOffset; - if (desiredReadOnlyAddress < addressTypeMask) // ReadOnlyAddressLagOffset is not scaled with addressTypeMask - return; + long desiredReadOnlyAddress = GetAddressOfStartOfPageOfAddress(currentTailAddress) - ReadOnlyAddressLagOffset; if (MonotonicUpdate(ref ReadOnlyAddress, desiredReadOnlyAddress, out _)) { // Debug.WriteLine("Allocate: Moving read-only offset from {0:X} to {1:X}", oldReadOnlyAddress, desiredReadOnlyAddress); @@ -1366,14 +1370,14 @@ private void PageAlignedShiftReadOnlyAddress(long currentTailAddress) [MethodImpl(MethodImplOptions.AggressiveInlining)] private void PageAlignedShiftHeadAddress(long currentTailAddress) { - var desiredHeadAddress = GetAddressOfStartOfPage(currentTailAddress) - HeadAddressLagOffset; - if (desiredHeadAddress < addressTypeMask) // HeadAddressLagOffset is not scaled with addressTypeMask - return; + var desiredHeadAddress = GetAddressOfStartOfPageOfAddress(currentTailAddress) - HeadAddressLagOffset; - // Obtain local values of variables that can change + // Obtain local values of variables that can change. var currentFlushedUntilAddress = FlushedUntilAddress; + + // If the new head address would be higher than the last flushed address, cap it at the start of the last flushed address' page start. if (desiredHeadAddress > currentFlushedUntilAddress) - desiredHeadAddress = GetAddressOfStartOfPage(currentFlushedUntilAddress); + desiredHeadAddress = GetAddressOfStartOfPageOfAddress(currentFlushedUntilAddress); if (MonotonicUpdate(ref HeadAddress, desiredHeadAddress, out _)) { @@ -1391,13 +1395,14 @@ public long ShiftHeadAddress(long desiredHeadAddress) // Obtain local values of variables that can change long currentFlushedUntilAddress = FlushedUntilAddress; + // If the new head address would be higher than the last flushed address, cap it at the start of the last flushed address' page start. long newHeadAddress = desiredHeadAddress; if (newHeadAddress > currentFlushedUntilAddress) newHeadAddress = currentFlushedUntilAddress; if (GetOffsetOnPage(newHeadAddress) != 0) { - + // TODO: HeadAddress advancement at a finer grain than page-level } if (MonotonicUpdate(ref HeadAddress, newHeadAddress, out _)) { @@ -1410,7 +1415,7 @@ public long ShiftHeadAddress(long desiredHeadAddress) /// /// Every async flush callback tries to update the flushed until address to the latest value possible - /// Is there a better way to do this with enabling fine-grained addresses (not necessarily at page boundaries)? + /// TODO: Is there a better way to do this with enabling fine-grained addresses (not necessarily at page boundaries)? /// protected void ShiftFlushedUntilAddress() { @@ -1419,7 +1424,7 @@ protected void ShiftFlushedUntilAddress() bool update = false; long pageLastFlushedAddress = PageStatusIndicator[page % BufferSize].LastFlushedUntilAddress; - while (pageLastFlushedAddress >= currentFlushedUntilAddress && currentFlushedUntilAddress >= GetStartLogicalAddressOfPage(page)) + while (pageLastFlushedAddress >= currentFlushedUntilAddress && currentFlushedUntilAddress >= GetLogicalAddressOfStartOfPage(page)) { currentFlushedUntilAddress = pageLastFlushedAddress; update = true; @@ -1477,12 +1482,12 @@ public void RecoveryReset(long tailAddress, long headAddress, long beginAddress, // Allocate current page if necessary var pageIndex = TailPageOffset.Page % BufferSize; - if (!_wrapper.IsAllocated(pageIndex)) + if (!IsAllocated(pageIndex)) _wrapper.AllocatePage(pageIndex); // Allocate next page as well - this is an invariant in the allocator! var nextPageIndex = (pageIndex + 1) % BufferSize; - if (!_wrapper.IsAllocated(nextPageIndex)) + if (!IsAllocated(nextPageIndex)) _wrapper.AllocatePage(nextPageIndex); BeginAddress = beginAddress; @@ -1497,7 +1502,7 @@ public void RecoveryReset(long tailAddress, long headAddress, long beginAddress, pageIndex = GetPageIndexForAddress(tailAddress); // clear the last page starting from tail address - _wrapper.ClearPage(pageIndex, (int)GetOffsetOnPage(tailAddress)); + ClearPage(pageIndex, (int)GetOffsetOnPage(tailAddress)); // Printing debug info logger?.LogInformation("******* Recovered HybridLog Stats *******"); @@ -1508,81 +1513,47 @@ public void RecoveryReset(long tailAddress, long headAddress, long beginAddress, logger?.LogInformation("Tail Address: {tailAddress}", tailAddress); } - /// Invoked by users to obtain a record from disk. It uses sector aligned memory to read the record efficiently into memory. + /// Read a main log record to - used for RUMD operations. [MethodImpl(MethodImplOptions.NoInlining)] internal unsafe void AsyncReadRecordToMemory(long fromLogicalAddress, int numBytes, DeviceIOCompletionCallback callback, ref AsyncIOContext context) { - var fileOffset = (ulong)(AlignedPageSizeBytes * GetPage(fromLogicalAddress) + GetOffsetOnPage(fromLogicalAddress)); - var alignedFileOffset = (ulong)(((long)fileOffset / sectorSize) * sectorSize); - - var alignedReadLength = (uint)((long)fileOffset + numBytes - (long)alignedFileOffset); - alignedReadLength = (uint)((alignedReadLength + (sectorSize - 1)) & ~(sectorSize - 1)); - - var record = bufferPool.Get((int)alignedReadLength); - record.valid_offset = (int)(fileOffset - alignedFileOffset); - record.available_bytes = (int)(alignedReadLength - (fileOffset - alignedFileOffset)); - record.required_bytes = numBytes; - var asyncResult = default(AsyncGetFromDiskResult); asyncResult.context = context; - asyncResult.context.record = record; - device.ReadAsync(alignedFileOffset, - (IntPtr)asyncResult.context.record.aligned_pointer, - alignedReadLength, - callback, - asyncResult); + asyncResult.context.record = GetAndPopulateReadBuffer(fromLogicalAddress, numBytes, out var alignedFileOffset, out var alignedReadLength); + device.ReadAsync(alignedFileOffset, (IntPtr)asyncResult.context.record.aligned_pointer, alignedReadLength, callback, asyncResult); } - /// - /// Read record to memory - simple read context version - /// - /// - /// - /// - /// + /// Read inline blittable record to > - simple read context version. Used by TsavoriteLog. [MethodImpl(MethodImplOptions.NoInlining)] - internal unsafe void AsyncReadRecordToMemory(long fromLogicalAddress, int numBytes, DeviceIOCompletionCallback callback, ref SimpleReadContext context) + internal unsafe void AsyncReadBlittableRecordToMemory(long fromLogicalAddress, int numBytes, DeviceIOCompletionCallback callback, ref SimpleReadContext context) { - var fileOffset = (ulong)(AlignedPageSizeBytes * GetPage(fromLogicalAddress) + GetOffsetOnPage(fromLogicalAddress)); - var alignedFileOffset = (ulong)(((long)fileOffset / sectorSize) * sectorSize); - - var alignedReadLength = (uint)((long)fileOffset + numBytes - (long)alignedFileOffset); - alignedReadLength = (uint)((alignedReadLength + (sectorSize - 1)) & ~(sectorSize - 1)); + context.record = GetAndPopulateReadBuffer(fromLogicalAddress, numBytes, out var alignedFileOffset, out var alignedReadLength); + device.ReadAsync(alignedFileOffset, (IntPtr)context.record.aligned_pointer, alignedReadLength, callback, context); + } - context.record = bufferPool.Get((int)alignedReadLength); - context.record.valid_offset = (int)(fileOffset - alignedFileOffset); - context.record.available_bytes = (int)(alignedReadLength - (fileOffset - alignedFileOffset)); - context.record.required_bytes = numBytes; + private SectorAlignedMemory GetAndPopulateReadBuffer(long fromLogicalAddress, int numBytes, out ulong alignedFileOffset, out uint alignedReadLength) + { + var fileOffset = (ulong)(AlignedPageSizeBytes * GetPage(fromLogicalAddress) + GetOffsetOnPage(fromLogicalAddress)); + alignedFileOffset = (ulong)RoundDown((long)fileOffset, sectorSize); + alignedReadLength = (uint)((long)fileOffset + numBytes - (long)alignedFileOffset); + alignedReadLength = (uint)RoundUp(alignedReadLength, sectorSize); - device.ReadAsync(alignedFileOffset, - (IntPtr)context.record.aligned_pointer, - alignedReadLength, - callback, - context); + var record = bufferPool.Get((int)alignedReadLength); + record.valid_offset = (int)(fileOffset - alignedFileOffset); + record.available_bytes = (int)(alignedReadLength - record.valid_offset); + record.required_bytes = numBytes; + return record; } - /// Read pages from specified device - public void AsyncReadPagesFromDevice( - long readPageStart, - int numPages, - long untilAddress, - DeviceIOCompletionCallback callback, - TContext context, - long devicePageOffset = 0, - IDevice logDevice = null, IDevice objectLogDevice = null) - => AsyncReadPagesFromDevice(readPageStart, numPages, untilAddress, callback, context, out _, devicePageOffset, logDevice, objectLogDevice); - - /// Read pages from specified device + /// Read pages from specified device(s) for recovery, with no output of the countdown event + public void AsyncReadPagesForRecovery(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, DeviceIOCompletionCallback callback, + TContext context, long devicePageOffset = 0, IDevice logDevice = null) + => AsyncReadPagesForRecovery(readBuffers, readPageStart, numPages, untilAddress, callback, context, out _, devicePageOffset, logDevice); + + /// Read pages from specified device for recovery, returning the countdown event [MethodImpl(MethodImplOptions.NoInlining)] - private void AsyncReadPagesFromDevice( - long readPageStart, - int numPages, - long untilAddress, - DeviceIOCompletionCallback callback, - TContext context, - out CountdownEvent completed, - long devicePageOffset = 0, - IDevice device = null, IDevice objectLogDevice = null) + private void AsyncReadPagesForRecovery(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, DeviceIOCompletionCallback callback, + TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null) { var usedDevice = device ?? this.device; IDevice usedObjlogDevice = objectLogDevice; @@ -1591,15 +1562,15 @@ private void AsyncReadPagesFromDevice( for (long readPage = readPageStart; readPage < (readPageStart + numPages); readPage++) { var pageIndex = (int)(readPage % BufferSize); - if (!_wrapper.IsAllocated(pageIndex)) + if (!IsAllocated(pageIndex)) _wrapper.AllocatePage(pageIndex); else - _wrapper.ClearPage(readPage); + ClearPage(readPage, offset: 0); var asyncResult = new PageAsyncReadResult() { page = readPage, - offset = devicePageOffset, + devicePageOffset = devicePageOffset, context = context, handle = completed, maxPtr = PageSize @@ -1619,10 +1590,16 @@ private void AsyncReadPagesFromDevice( if (device != null) offsetInFile = (ulong)(AlignedPageSizeBytes * (readPage - devicePageOffset)); - ReadAsync(offsetInFile, pageIndex, readLength, callback, asyncResult, usedDevice, usedObjlogDevice); + // Call the overridden ReadAsync for the derived allocator class + ReadAsync(readBuffers, offsetInFile, (IntPtr)pagePointers[pageIndex], readLength, callback, asyncResult, usedDevice); } } + /// Create the circular buffers for flushing to device. Only implemented by ObjectAllocator. + internal virtual CircularDiskWriteBuffer CreateCircularFlushBuffers(IDevice objectLogDevice, ILogger logger) => default; + /// Create the circular flush buffers for object dexerialization from device. Only implemented by ObjectAllocator. + internal virtual CircularDiskReadBuffer CreateCircularReadBuffers(IDevice objectLogDevice, ILogger logger) => default; + /// /// Flush page range to disk /// Called when all threads have agreed that a page range is sealed. @@ -1631,7 +1608,7 @@ private void AsyncReadPagesFromDevice( /// /// [MethodImpl(MethodImplOptions.NoInlining)] - public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = false) + public void AsyncFlushPagesForReadOnly(long fromAddress, long untilAddress, bool noFlush = false) { long startPage = GetPage(fromAddress); long endPage = GetPage(untilAddress); @@ -1648,12 +1625,15 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = // queue for previous pages indexes. Also, flush callbacks will attempt to dequeue from PendingFlushes for FlushedUntilAddress, which again // increases monotonically. + // For OA, create the buffers we will use for all ranges of the flush. This calls our callback and disposes itself when the last write of a range completes. + using var flushBuffers = CreateCircularFlushBuffers(objectLogDevice: null, logger); + // Request asynchronous writes to the device. If waitForPendingFlushComplete is set, then a CountDownEvent is set in the callback handle. for (long flushPage = startPage; flushPage < (startPage + numPages); flushPage++) { // Default to writing the full page. - long pageStartAddress = GetStartLogicalAddressOfPage(flushPage); - long pageEndAddress = GetStartLogicalAddressOfPage(flushPage + 1); + long pageStartAddress = GetLogicalAddressOfStartOfPage(flushPage); + long pageEndAddress = GetLogicalAddressOfStartOfPage(flushPage + 1); var asyncResult = new PageAsyncFlushResult { @@ -1661,7 +1641,8 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = count = 1, partial = false, fromAddress = pageStartAddress, - untilAddress = pageEndAddress + untilAddress = pageEndAddress, + flushBuffers = flushBuffers }; // If either fromAddress or untilAddress is in the middle of the page, this will be a partial page flush. @@ -1714,14 +1695,17 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = // Perform work from shared queue if possible: When a flush completes it updates FlushedUntilAddress. If there // is an item in the shared queue that starts at FlushedUntilAddress, it can now be flushed. Flush callbacks // will RemoveNextAdjacent(FlushedUntilAddress, ...) to continue the chain of flushes until the queue is empty. + // This will issue a write that completes in the background as we move to the next adjacent chunk (or page if + // this is the last chunk on the current page). if (PendingFlush[index].RemoveNextAdjacent(FlushedUntilAddress, out PageAsyncFlushResult request)) - WriteAsync(GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class + WriteAsync(flushBuffers, GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class } else { - // Write the entire page up to asyncResult.untilAddress (there can be no previous items in the queue). Flush callbacks + // Write the entire page up to asyncResult.untilAddress (there can be no previous items in the queue). + // This will issue a write that completes in the background as we move to the next page, and the Flush callbacks // will RemoveNextAdjacent(FlushedUntilAddress, ...) to continue the chain of flushes until the queue is empty. - WriteAsync(flushPage, AsyncFlushPageCallback, asyncResult); // Call the overridden WriteAsync for the derived allocator class + WriteAsync(flushBuffers, flushPage, AsyncFlushPageCallback, asyncResult); // Call the overridden WriteAsync for the derived allocator class } } } @@ -1734,8 +1718,11 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = /// /// /// - public void AsyncFlushPages(long flushPageStart, int numPages, DeviceIOCompletionCallback callback, TContext context) + public void AsyncFlushPagesForRecovery(long flushPageStart, int numPages, DeviceIOCompletionCallback callback, TContext context) { + // For OA, create the buffers we will use for all ranges of the flush. This calls our callback and disposes itself when the last write of a range completes. + using var flushBuffers = CreateCircularFlushBuffers(objectLogDevice: null, logger); + for (long flushPage = flushPageStart; flushPage < (flushPageStart + numPages); flushPage++) { var asyncResult = new PageAsyncFlushResult() @@ -1744,10 +1731,11 @@ public void AsyncFlushPages(long flushPageStart, int numPages, DeviceI context = context, count = 1, partial = false, - untilAddress = GetStartLogicalAddressOfPage(flushPage + 1) + untilAddress = GetLogicalAddressOfStartOfPage(flushPage + 1), + flushBuffers = flushBuffers }; - WriteAsync(flushPage, callback, asyncResult); + WriteAsync(flushBuffers, flushPage, callback, asyncResult); } } @@ -1764,15 +1752,14 @@ public void AsyncFlushPages(long flushPageStart, int numPages, DeviceI /// /// [MethodImpl(MethodImplOptions.NoInlining)] - public void AsyncFlushPagesToDevice(long startPage, long endPage, long endLogicalAddress, long fuzzyStartLogicalAddress, IDevice device, IDevice objectLogDevice, out SemaphoreSlim completedSemaphore, int throttleCheckpointFlushDelayMs) + public void AsyncFlushPagesForSnapshot(long startPage, long endPage, long endLogicalAddress, long fuzzyStartLogicalAddress, IDevice device, IDevice objectLogDevice, out SemaphoreSlim completedSemaphore, int throttleCheckpointFlushDelayMs) { logger?.LogTrace("Starting async full log flush with throttling {throttlingEnabled}", throttleCheckpointFlushDelayMs >= 0 ? $"enabled ({throttleCheckpointFlushDelayMs}ms)" : "disabled"); var _completedSemaphore = new SemaphoreSlim(0); completedSemaphore = _completedSemaphore; - // If throttled, convert rest of the method into a truly async task run - // because issuing IO can take up synchronous time + // If throttled, convert rest of the method into a truly async task run because issuing IO can take up synchronous time if (throttleCheckpointFlushDelayMs >= 0) _ = Task.Run(FlushRunner); else @@ -1785,9 +1772,12 @@ void FlushRunner() var flushCompletionTracker = new FlushCompletionTracker(_completedSemaphore, throttleCheckpointFlushDelayMs >= 0 ? new SemaphoreSlim(0) : null, totalNumPages); var localSegmentOffsets = new long[SegmentBufferSize]; + // Create the buffers we will use for all ranges of the flush (if we are ObjectAllocator). This calls our callback when the last write of a partial flush completes. + using var flushBuffers = CreateCircularFlushBuffers(objectLogDevice, logger); + for (long flushPage = startPage; flushPage < endPage; flushPage++) { - long flushPageAddress = GetStartLogicalAddressOfPage(flushPage); + long flushPageAddress = GetLogicalAddressOfStartOfPage(flushPage); var pageSize = PageSize; if (flushPage == endPage - 1) pageSize = (int)(endLogicalAddress - flushPageAddress); @@ -1798,11 +1788,12 @@ void FlushRunner() page = flushPage, fromAddress = flushPageAddress, untilAddress = flushPageAddress + pageSize, - count = 1 + count = 1, + flushBuffers = flushBuffers, }; // Intended destination is flushPage - WriteAsyncToDevice(startPage, flushPage, pageSize, AsyncFlushPageToDeviceCallback, asyncResult, device, objectLogDevice, localSegmentOffsets, fuzzyStartLogicalAddress); + WriteAsyncToDevice(flushBuffers, startPage, flushPage, pageSize, AsyncFlushPageToDeviceCallback, asyncResult, device, objectLogDevice, fuzzyStartLogicalAddress); if (throttleCheckpointFlushDelayMs >= 0) { @@ -1813,9 +1804,11 @@ void FlushRunner() } } - internal void AsyncGetFromDisk(long fromLogicalAddress, int numBytes, AsyncIOContext context, SectorAlignedMemory result = default) + internal void AsyncGetFromDisk(long fromLogicalAddress, int numBytes, AsyncIOContext context) { - if (epoch.ThisInstanceProtected()) // Do not spin for unprotected IO threads + // If this is a protected thread, we must wait to issue the Read operation. Spin until the device is not throttled, + // draining events on each iteration, but do not release the epoch. + if (epoch.ThisInstanceProtected()) { while (device.Throttle()) { @@ -1825,20 +1818,14 @@ internal void AsyncGetFromDisk(long fromLogicalAddress, int numBytes, AsyncIOCon } } - // Convert to absolute address as we're going to disk - fromLogicalAddress = AbsoluteAddress(fromLogicalAddress); - - if (result == null) - AsyncReadRecordToMemory(fromLogicalAddress, numBytes, AsyncGetFromDiskCallback, ref context); - else - AsyncReadRecordObjectsToMemory(fromLogicalAddress, numBytes, AsyncGetFromDiskCallback, context, result); + AsyncReadRecordToMemory(fromLogicalAddress, numBytes, AsyncGetFromDiskCallback, ref context); } /// /// Read pages from specified device /// [MethodImpl(MethodImplOptions.NoInlining)] - internal void AsyncReadPagesFromDeviceToFrame( + internal void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, @@ -1847,7 +1834,7 @@ internal void AsyncReadPagesFromDeviceToFrame( BlittableFrame frame, out CountdownEvent completed, long devicePageOffset = 0, - IDevice device = null, IDevice objectLogDevice = null) + IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) { var usedDevice = device ?? this.device; @@ -1865,7 +1852,7 @@ internal void AsyncReadPagesFromDeviceToFrame( page = readPage, context = context, handle = completed, - frame = frame + cts = cts }; ulong offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); @@ -1881,10 +1868,86 @@ internal void AsyncReadPagesFromDeviceToFrame( if (device != null) offsetInFile = (ulong)(AlignedPageSizeBytes * (readPage - devicePageOffset)); - usedDevice.ReadAsync(offsetInFile, (IntPtr)frame.pointers[pageIndex], readLength, callback, asyncResult); + ReadAsync(readBuffers, offsetInFile, (IntPtr)frame.GetPhysicalAddress(pageIndex), readLength, callback, asyncResult, usedDevice); } } + /// + /// Checks to see if we have a full record, or at least enough to compare the key. + /// + /// The context from the IO operation + /// If we return false, the address to issue the next IO for + /// If we return false, the number of bytes to issue the next IO for + /// True if we have the full record and the key was the requested key; if the record is fully inline, then the ctx.diskLogRecord is set and the ctx.record is transferred to it. + /// Otherwise it is false, and: + /// + /// If the key was present, it did not match ctx.request_key; is recordInfo.PreviousAddress, and + /// is the initial IO size. + /// Otherwise, the data we have is not sufficient to determine record length, or we know the length and it is greater than the data we have now. + /// is the same address we just read, and is one of: + /// + /// If we did not have enough data to determine required length, we use the initial IO size. This should seldom happen as we issue the initial + /// IO request with this size, but perhaps this is called with a partial buffer. + /// Otherwise, we know the data length needed, and we set to that. + /// + /// + /// + /// If we have a complete record and the key passes the comparison and we have overflow or objects, then this will be overridden by a derived class (see + /// ) which will issue additional reads to retrieve those objects. + private protected virtual bool VerifyRecordFromDiskCallback(ref AsyncIOContext ctx, out long prevAddressToRead, out int prevLengthToRead) + { + // Initialize to "key is not present (data too small) or does not match so get previous record" length to read + prevLengthToRead = IStreamBuffer.InitialIOSize; + + // See if we have a complete record. + var currentLength = ctx.record.available_bytes; + if (currentLength >= RecordInfo.Size + MinLengthMetadataBytes) + { + var ptr = ctx.record.GetValidPointer(); + var (keyLengthBytes, valueLengthBytes, _ /*usesChainedChunks*/) = DeconstructIndicatorByte(*(ptr + RecordInfo.Size)); + var recordInfo = *(RecordInfo*)ptr; + + // Initialize to "key does not match so get previous record" address to read + prevAddressToRead = recordInfo.PreviousAddress; + + if (recordInfo.Invalid) // includes IsNull + return false; + + var optionalLength = LogRecord.GetOptionalLength(recordInfo); + var offsetToKeyStart = RecordInfo.Size + NumIndicatorBytes + keyLengthBytes + valueLengthBytes; + + // If the length is up to offsetToKeyStart, we can read the full lengths. + if (currentLength >= offsetToKeyStart) + { + var keyLengthPtr = ptr + RecordInfo.Size + NumIndicatorBytes; + var keyLength = GetKeyLength(keyLengthBytes, keyLengthPtr); + + // We have the full key if it is inline, so check for a match if we had a requested key, and return if not. + if (!ctx.request_key.IsEmpty && recordInfo.KeyIsInline && !storeFunctions.KeysEqual(ctx.request_key, new ReadOnlySpan(ptr + offsetToKeyStart, keyLength))) + return false; + + // Values in SpanByteAllocator will always be string, thus limited to 512MB, so cast to int is OK. + var valueLength = (int)GetValueLength(valueLengthBytes, keyLengthPtr + keyLengthBytes); + var recordLength = offsetToKeyStart + keyLength + valueLength + optionalLength; + + // If we have the full record and the keys match, success. + if (currentLength >= recordLength) + { + ctx.diskLogRecord = DiskLogRecord.TransferFrom(ref ctx.record, transientObjectIdMap); + return true; + } + + // We need to read the same address, but with the full record length we now know. + prevLengthToRead = recordLength; + } + } + + // Either we didn't have the full record size, or we didn't have enough bytes to even read the full record size. Either way, prevLengthToRead + // is set for a re-read of the same record. + prevAddressToRead = ctx.logicalAddress; + return false; + } + [MethodImpl(MethodImplOptions.NoInlining)] private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, object context) { @@ -1895,53 +1958,35 @@ private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, obje var ctx = result.context; try { - bool hasFullRecord = DiskLogRecord.IsComplete(ctx.record, out bool hasFullKey, out int requiredBytes); // TODO: support 'long' lengths - if (hasFullKey || ctx.request_key.IsEmpty) - { - var diskLogRecord = new DiskLogRecord((long)ctx.record.GetValidPointer()); // This ctor does not test for having the complete record; we've already set hasFullRecord - Debug.Assert(!diskLogRecord.Info.Invalid, "Invalid records should not be in the hash chain for pending IO"); + // Note: don't test for (numBytes >= ctx.record.required_bytes) for this initial read, as the file may legitimately end before the + // InitialIOSize request can be fulfilled. + ctx.record.available_bytes = (int)numBytes; - // If request_key is null we're called from ReadAtAddress, so it is an implicit match. - var currentRecordIsInRange = ctx.logicalAddress >= BeginAddress && ctx.logicalAddress >= ctx.minAddress; - if (!ctx.request_key.IsEmpty && !storeFunctions.KeysEqual(ctx.request_key, diskLogRecord.Key)) - { - // Keys don't match so request the previous record in the chain if it is in the range to resolve. - ctx.logicalAddress = diskLogRecord.Info.PreviousAddress; - if (ctx.logicalAddress >= BeginAddress && ctx.logicalAddress >= ctx.minAddress) - { - ctx.record.Return(); - ctx.record = ctx.objBuffer = default; - AsyncGetFromDisk(ctx.logicalAddress, requiredBytes, ctx); - return; - } - } + if (!VerifyRecordFromDiskCallback(ref ctx, out var prevAddressToRead, out var prevLengthToRead)) + { + Debug.Assert(!(*(RecordInfo*)ctx.record.GetValidPointer()).Invalid, "Invalid records should not be in the hash chain for pending IO"); - if (hasFullRecord) + // Either we had an incomplete record and we're re-reading the current record, or the record Key didn't match and we're reading the previous record + // in the chain. If the record to read is in the range to resolve then issue the read, else fall through to signal "IO complete". + ctx.logicalAddress = prevAddressToRead; + if (ctx.logicalAddress >= BeginAddress && ctx.logicalAddress >= ctx.minAddress) { - // Either the keys match or we are below the range to retrieve (which ContinuePending* will detect), so we're done. - if (currentRecordIsInRange && diskLogRecord.Info.ValueIsObject) - ctx.ValueObject = diskLogRecord.DeserializeValueObject(storeFunctions.CreateValueObjectSerializer()); - - if (ctx.completionEvent is not null) - ctx.completionEvent.Set(ref ctx); - else - ctx.callbackQueue.Enqueue(ctx); + ctx.DisposeRecord(); + AsyncGetFromDisk(ctx.logicalAddress, prevLengthToRead, ctx); + return; } } - if (!hasFullRecord) - { - // We don't have the full record and may not even have gotten a full key, so we need to do another IO - if (requiredBytes > int.MaxValue) - throw new TsavoriteException("Records exceeding 2GB are not yet supported"); // TODO note: We should not have written this yet; serialization is int-limited - - ctx.record.Return(); - AsyncGetFromDisk(ctx.logicalAddress, requiredBytes, ctx); - } + // Either we have a full record with a key match or we are below the range to retrieve (which ContinuePending* will detect), so we're done. + if (ctx.completionEvent is not null) + ctx.completionEvent.Set(ref ctx); + else + ctx.callbackQueue.Enqueue(ctx); } catch (Exception e) { logger?.LogError(e, "AsyncGetFromDiskCallback error"); + ctx.DisposeRecord(); if (ctx.completionEvent is not null) ctx.completionEvent.SetException(e); else @@ -1983,9 +2028,10 @@ private void AsyncFlushPageCallback(uint errorCode, uint numBytes, object contex result.Free(); } + // Continue the chained flushes, popping the next request from the queue if it is adjacent. var _flush = FlushedUntilAddress; if (GetOffsetOnPage(_flush) > 0 && PendingFlush[GetPage(_flush) % BufferSize].RemoveNextAdjacent(_flush, out PageAsyncFlushResult request)) - WriteAsync(GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class + WriteAsync(result.flushBuffers, GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class } catch when (disposed) { } } @@ -2000,7 +2046,7 @@ internal void UnsafeSkipError(CommitInfo info) ShiftFlushedUntilAddress(); var _flush = FlushedUntilAddress; if (GetOffsetOnPage(_flush) > 0 && PendingFlush[GetPage(_flush) % BufferSize].RemoveNextAdjacent(_flush, out PageAsyncFlushResult request)) - WriteAsync(GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class + WriteAsync(request.flushBuffers, GetPage(request.fromAddress), AsyncFlushPageCallback, request); // Call the overridden WriteAsync for the derived allocator class } catch when (disposed) { } } @@ -2019,18 +2065,12 @@ protected void AsyncFlushPageToDeviceCallback(uint errorCode, uint numBytes, obj logger?.LogError("AsyncFlushPageToDeviceCallback error: {0}", errorCode); var result = (PageAsyncFlushResult)context; - - var epochTaken = false; - if (!epoch.ThisInstanceProtected()) - { - epochTaken = true; - epoch.Resume(); - } + var epochTaken = epoch.ResumeIfNotProtected(); // Unset dirty bit for flushed pages try { - var startAddress = GetStartLogicalAddressOfPage(result.page); + var startAddress = GetLogicalAddressOfStartOfPage(result.page); var endAddress = startAddress + PageSize; if (result.fromAddress > startAddress) @@ -2048,7 +2088,7 @@ protected void AsyncFlushPageToDeviceCallback(uint errorCode, uint numBytes, obj if (flushWidth > 0) { - var physicalAddress = _wrapper.GetPhysicalAddress(startAddress); + var physicalAddress = GetPhysicalAddress(startAddress); var endPhysicalAddress = physicalAddress + flushWidth; while (physicalAddress < endPhysicalAddress) diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs index b7dd84212a6..a7b72535dee 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs @@ -8,6 +8,7 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; public abstract partial class AllocatorBase : IDisposable @@ -62,7 +63,6 @@ internal bool PushScanImpl(long beginAddress, lon { if (!scanFunctions.OnStart(beginAddress, endAddress)) return false; - var headAddress = HeadAddress; long numRecords = 1; var stop = false; @@ -154,7 +154,7 @@ internal unsafe bool GetFromDiskAndPushToReader(ReadOnlySpan(ReadOnlySpan - /// Key; this is a shallow copy of the key in pendingContext, pointing to its diskLogRecord + /// Key; this is a shallow copy of the key in pendingContext, pointing to its request_key. /// public PinnedSpanByte request_key; - /// - /// Deserialized ValueObject if RecordInfo.ValueIsObject, else null - /// - public IHeapObject ValueObject; + /// The retrieved record, including deserialized ValueObject if RecordInfo.ValueIsObject, and key or value Overflows + public DiskLogRecord diskLogRecord; /// /// Logical address @@ -67,12 +65,18 @@ public unsafe struct AsyncIOContext /// /// Dispose /// - public void Dispose() + public void DisposeRecord() { // Do not dispose request_key as it is a shallow copy of the key in pendingContext + diskLogRecord.Dispose(); + diskLogRecord = default; record?.Return(); record = null; } + + /// + public override readonly string ToString() + => $"id {id}, key {request_key}, LogAddr {AddressString(logicalAddress)}, MinAddr {minAddress}, LogRec [{diskLogRecord}]"; } // Wrapper class so we can communicate back the context.record even if it has to retry due to incomplete records. @@ -98,7 +102,7 @@ internal AsyncIOContextCompletionEvent() /// internal void Prepare(PinnedSpanByte request_key, long logicalAddress) { - request.Dispose(); + request.DisposeRecord(); request.request_key = request_key; request.logicalAddress = logicalAddress; } @@ -106,7 +110,7 @@ internal void Prepare(PinnedSpanByte request_key, long logicalAddress) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void Set(ref AsyncIOContext ctx) { - request.Dispose(); + request.DisposeRecord(); request = ctx; exception = null; _ = semaphore.Release(1); @@ -115,7 +119,7 @@ internal void Set(ref AsyncIOContext ctx) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void SetException(Exception ex) { - request.Dispose(); + request.DisposeRecord(); request = default; exception = ex; _ = semaphore.Release(1); @@ -126,7 +130,7 @@ internal void SetException(Exception ex) /// public void Dispose() { - request.Dispose(); + request.DisposeRecord(); semaphore?.Dispose(); } } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs index 69c11b33e73..0ddcfd0d770 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs @@ -6,6 +6,8 @@ namespace Tsavorite.core { + using static Utility; + /// /// A frame is an in-memory circular buffer of log pages /// @@ -29,9 +31,9 @@ public unsafe void Allocate(int index) { var adjustedSize = pageSize + 2 * sectorSize; - byte[] tmp = GC.AllocateArray(adjustedSize, true); - long p = (long)Unsafe.AsPointer(ref tmp[0]); - pointers[index] = (p + (sectorSize - 1)) & ~((long)sectorSize - 1); + var tmp = GC.AllocateArray(adjustedSize, pinned:true); + var p = (long)Unsafe.AsPointer(ref tmp[0]); + pointers[index] = RoundUp(p, sectorSize); frame[index] = tmp; } @@ -40,7 +42,7 @@ public void Clear(int pageIndex) Array.Clear(frame[pageIndex], 0, frame[pageIndex].Length); } - public long GetPhysicalAddress(long frameNumber, long offset) + public long GetPhysicalAddress(long frameNumber, long offset = 0) { return pointers[frameNumber % frameSize] + offset; } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs index 21c228bd261..ff0bcb0e6bc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs @@ -6,841 +6,418 @@ using System.Diagnostics; using System.IO; using System.Runtime.CompilerServices; -using static Tsavorite.core.Utility; #pragma warning disable CS1591 // Missing XML comment for publicly visible type or member namespace Tsavorite.core { - /// The record on the disk: header, optional fields, key, value - /// The space is laid out as one of: - /// - /// Identical to - /// As a "varbyte" encoding: - /// - /// [RecordInfo][Indicator byte][KeyLength varbytes][ValueLength varbytes][key Span][value Span][ETag?][Expiration?][filler bytes--rounded up to 8 byte boundary] - /// Indicator byte: 3 bits for version, 2 bits for key length, 3 bits for value length - /// Key and Value Length varbytes: from 1-4 bytes for Key and 1-8 bytes for Value - /// - /// - /// - /// This lets us get to the optional fields for comparisons without loading the full record (GetIOSize should cover the space for optionals). - /// +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + + /// A wrapper around LogRecord for retrieval from disk or carrying through pending operations public unsafe struct DiskLogRecord : ISourceLogRecord, IDisposable { - /// The initial size to IO from disk when reading a record; by default a single page. If we don't get the full record, - /// at least we'll likely get the full Key and value length, and can read the full record using that. - /// Must be a power of 2 - public const int InitialIOSize = 4 * 1024; - - /// The physicalAddress in the log. - internal long physicalAddress; + /// The > around the record data. + internal LogRecord logRecord; - /// The deserialized ValueObject if this is a disk record for the Object Store. Held directly; does not use . - internal IHeapObject valueObject; + /// The buffer containing the record data, from either disk IO or a copy from a LogRecord that is carried through pending operations + /// such as Compact or ConditionalCopyToTail. The contains its + /// as its . + /// We always own the record buffer; it is either transferred to us, or allocated as a copy of the record memory. However, it may be + /// null if we transferred it out. + SectorAlignedMemory recordBuffer; - /// If this is non-null, it must be freed on . - internal SectorAlignedMemory recordBuffer; - - /// Constructor that takes a physical address, which may come from a or some other allocation - /// that will have at least the lifetime of this . This does not own the memory - /// allocation so will not free it on . - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public DiskLogRecord(long physicalAddress, int length) : this(physicalAddress) - { - if (!IsComplete(physicalAddress, length, out _, out _)) - throw new TsavoriteException("DiskLogRecord is not complete"); - } + /// The action to perform when disposing the contained LogRecord; the objects may have been transferred. + internal Action objectDisposer; - /// Constructor that takes a physical address, which may come from a or some other allocation - /// that will have at least the lifetime of this and is known to contain the entire record (as there is no length - /// supplied). This does not own the memory allocation so will not free it on . - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal DiskLogRecord(long physicalAddress) - { - this.physicalAddress = physicalAddress; - InfoRef.ClearBitsForDiskImages(); - } + public override readonly string ToString() + => $"logRec [{logRecord}], recordBuffer [{recordBuffer}], objDisp [{objectDisposer}]"; - /// Constructor that takes a from which it obtains the physical address. - /// This owns the memory allocation and must free it on . - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public DiskLogRecord(SectorAlignedMemory allocatedRecord) : this((long)allocatedRecord.GetValidPointer()) - { - recordBuffer = allocatedRecord; - if (!IsComplete(allocatedRecord, out _, out _)) - throw new TsavoriteException("DiskLogRecord is not complete"); + /// + /// Constructor taking the record buffer and out-of-line objects. Private; use either CopyFrom or TransferFrom. + /// + /// The record buffer, either from IO or a copy for pending operations such as Compact or ConditionalCopyToTail. + /// The to hold the objects for the for the lifetime of this . + /// The key overflow byte[] wrapper, if any + /// The value overflow byte[] wrapper, if any + /// The value object, if any + /// The action to invoke when disposing the value object if it is present when we dispose the + /// We always own the record buffer; it is either transferred to us by TransferFrom, or allocated as a copy of the record memory by CopyFrom + private DiskLogRecord(SectorAlignedMemory recordBuffer, ObjectIdMap transientObjectIdMap, OverflowByteArray keyOverflow, + OverflowByteArray valueOverflow, IHeapObject valueObject, Action objectDisposer) + { + this.recordBuffer = recordBuffer; + this.objectDisposer = objectDisposer; + logRecord = new((long)recordBuffer.GetValidPointer(), transientObjectIdMap); + if (!keyOverflow.IsEmpty) + logRecord.KeyOverflow = keyOverflow; + if (!valueOverflow.IsEmpty) + logRecord.ValueOverflow = valueOverflow; + else if (valueObject is not null) + logRecord.ValueObject = valueObject; } - /// Constructor that takes an from which it obtains the physical address and ValueObject. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal DiskLogRecord(ref AsyncIOContext ctx) : this(ctx.record) + /// + /// Constructs the from an already-constructed LogRecord (e.g. from which + /// has transient ObjectIds if it has objects). + /// + internal DiskLogRecord(in LogRecord memoryLogRecord, Action objectDisposer) { - valueObject = ctx.ValueObject; + logRecord = memoryLogRecord; + this.objectDisposer = objectDisposer; } - /// A ref to the record header - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref RecordInfo GetInfoRef(long physicalAddress) => ref Unsafe.AsRef((byte*)physicalAddress); - - /// Fast access returning a copy of the record header - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static RecordInfo GetInfo(long physicalAddress) => *(RecordInfo*)physicalAddress; - - /// Serialized length of the record - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetSerializedLength() - => Info.IsNull ? RecordInfo.GetLength() : RoundUp(GetOptionalStartAddress() + OptionalLength - physicalAddress, Constants.kRecordAlignment); - - /// Called by IO to determine whether the record is complete (full serialized length has been read) - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool IsComplete(SectorAlignedMemory recordBuffer, out bool hasFullKey, out int requiredBytes) - => IsComplete((long)recordBuffer.GetValidPointer(), recordBuffer.available_bytes, out hasFullKey, out requiredBytes); - - private static bool IsComplete(long physicalAddress, int availableBytes, out bool hasFullKey, out int requiredBytes) + /// + /// Transfers a transient inline record buffer and creates our contained from it. Private; use either CopyFrom or TransferFrom. + /// + /// The record buffer, either from IO or a copy for pending operations such as Compact or ConditionalCopyToTail. + /// The to hold the objects for the for the lifetime of this . + /// We always own the record buffer; it is either transferred to us, or allocated as a copy of the record memory + private DiskLogRecord(SectorAlignedMemory recordBuffer, ObjectIdMap transientObjectIdMap) { - hasFullKey = false; - - // Check for RecordInfo and either indicator byte or key length; InlineLengthPrefixSize is the larger. - if (availableBytes < RecordInfo.GetLength() + LogField.InlineLengthPrefixSize) - { - requiredBytes = InitialIOSize; - return false; - } - - // Now we can tell if this is a fully inline vs. varbyte record. - var address = physicalAddress; - if ((*(RecordInfo*)address).RecordIsInline) - { - address += RecordInfo.GetLength(); - address += sizeof(int) + *(int*)address; // Inline key length is after RecordInfo; position address after the Key (but don't dereference it yet!) - requiredBytes = (int)(address + sizeof(int) - physicalAddress); // Include value length int in the calculation - if (availableBytes < requiredBytes) - { - // We have the RecordInfo and key length, but not the full key data. Get another page. - requiredBytes = RoundUp(requiredBytes, InitialIOSize); - return false; - } - - // We have the full Key and the value length available. - hasFullKey = true; - requiredBytes += *(int*)address; - } - else - { - // We are in varbyte format. We need to check the indicator byte for the key and value length. - address += RecordInfo.GetLength(); // Point to indicator byte - var keyLengthBytes = (int)((*(byte*)address & kKeyLengthBitMask) >> 3); - var valueLengthBytes = (int)(*(byte*)address & kValueLengthBitMask); - - requiredBytes = RecordInfo.GetLength() + 1 + keyLengthBytes + valueLengthBytes; // Include the indicator byte in the calculation - if (availableBytes < requiredBytes) - return false; - - var ptr = (byte*)++address; // Move past the indicator byte; the next bytes are key length, then value length - var keyLength = ReadVarBytes(keyLengthBytes, ref ptr); - var valueLength = ReadVarBytes(valueLengthBytes, ref ptr); - - requiredBytes += (int)keyLength; - hasFullKey = availableBytes >= requiredBytes; - requiredBytes += (int)valueLength; // TODO: Handle long values - } - - var info = *(RecordInfo*)physicalAddress; - var eTagLen = info.HasETag ? LogRecord.ETagSize : 0; - var expirationLen = info.HasExpiration ? LogRecord.ExpirationSize : 0; - - requiredBytes += eTagLen + expirationLen; - return availableBytes >= requiredBytes; + this.recordBuffer = recordBuffer; + logRecord = new((long)recordBuffer.GetValidPointer(), transientObjectIdMap); } - /// If true, this DiskLogRecord owns the buffer and must free it on - public readonly bool OwnsMemory => recordBuffer is not null; - - // Indicator bits for version and varlen int. Since the record is always aligned to 8 bytes, we can use long operations (on values only in - // the low byte) which are faster than byte or int. -#pragma warning disable IDE1006 // Naming Styles: Must begin with uppercase letter - const long kVersionBitMask = 7 << 5; // 3 bits for version - const long kKeyLengthBitMask = 3 << 3; // 2 bits for the number of bytes for the key length (this is limited to 512MB) - const long kValueLengthBitMask = 7; // 3 bits for the number of bytes for the value length -#pragma warning restore IDE1006 // Naming Styles - - const long CurrentVersion = 0 << 5; // Initial version is 0; shift will always be 5 - - /// This contains the leading byte which are the indicators, plus the up-to-int length for the key, and then some or all of the length for the value. - internal readonly long IndicatorAddress => physicalAddress + RecordInfo.GetLength(); - - /// Version of the variable-length byte encoding for key and value lengths. There is no version info for - /// records as these are image-identical to LogRecord. TODO: Include a major version for this in the Recovery version-compatibility detection - internal readonly long Version => (*(byte*)IndicatorAddress & kVersionBitMask) >> 6; - - internal readonly (int length, long dataAddress) KeyInfo - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - var address = IndicatorAddress; - if (Info.RecordIsInline) // For inline, the key length int starts at the same offset as IndicatorAddress - return (*(int*)address, address + LogField.InlineLengthPrefixSize); - - var keyLengthBytes = (int)((*(byte*)address & kKeyLengthBitMask) >> 3) + 1; - var valueLengthBytes = (int)(*(byte*)address & kValueLengthBitMask) + 1; - - byte* ptr = (byte*)++address; // Move past the indicator byte; the next bytes are key length - var keyLength = ReadVarBytes(keyLengthBytes, ref ptr); - - // Move past the key and value length bytes to the start of the key - return ((int)keyLength, address + keyLengthBytes + valueLengthBytes); - } - } + /// + /// Creates a from an already-constructed LogRecord (e.g. from which + /// has transient ObjectIds if it has objects). + /// + internal static DiskLogRecord CreateFromTransientLogRecord(in LogRecord memoryLogRecord, Action objectDisposer) => new(memoryLogRecord, objectDisposer); - internal readonly (long length, long dataAddress) ValueInfo + /// + /// Allocates and copies the LogRecord's record memory into it; any out-of-line objects are shallow-copied. + /// + /// The to copy + /// The buffer pool to allocate from + /// The to hold the objects for the for the lifetime of this . + /// The action to invoke when disposing the value object if it is present when we dispose the + internal static DiskLogRecord CopyFrom(in LogRecord logRecord, SectorAlignedBufferPool bufferPool, ObjectIdMap transientObjectIdMap, Action objectDisposer) { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - var keyInfo = KeyInfo; - if (Info.RecordIsInline) // For inline records the value length is an int, stored immediately after key data - { - var valueLengthAddress = keyInfo.dataAddress + keyInfo.length; - return (*(int*)valueLengthAddress, valueLengthAddress + sizeof(int)); - } - - var address = IndicatorAddress; - var keyLengthBytes = (int)((*(byte*)address & kKeyLengthBitMask) >> 3) + 1; // add 1 due to 0-based - var valueLengthBytes = (int)(*(byte*)address & kValueLengthBitMask) + 1; // add 1 due to 0-based - - byte* ptr = (byte*)IndicatorAddress + 1 + keyLengthBytes; // Skip over the key length bytes; the value length bytes are immediately after (before the key data) - var valueLength = ReadVarBytes(valueLengthBytes, ref ptr); - return (valueLength, keyInfo.dataAddress + keyInfo.length); // Value data (without length prefix) starts immediately after key data - } + var recordBuffer = AllocateBuffer(in logRecord, bufferPool); + return new DiskLogRecord(recordBuffer, transientObjectIdMap, + logRecord.Info.KeyIsOverflow ? logRecord.KeyOverflow : default, + logRecord.Info.ValueIsOverflow ? logRecord.ValueOverflow : default, + logRecord.Info.ValueIsObject ? logRecord.ValueObject : default, objectDisposer); } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static byte CreateIndicatorByte(int keyLength, long valueLength, out int keyByteCount, out int valueByteCount) + /// + /// Copies a LogRecord with no out-of-line objects into our contained . + /// + /// The record buffer, either from IO or a copy for pending operations such as Compact or ConditionalCopyToTail. + /// The to hold the objects for the for the lifetime of this . + /// The key overflow byte[] wrapper, if any + /// The value overflow byte[] wrapper, if any + /// The value object, if any + /// The action to invoke when disposing the value object if it is present when we dispose the + internal static DiskLogRecord TransferFrom(ref SectorAlignedMemory recordBuffer, ObjectIdMap transientObjectIdMap, OverflowByteArray keyOverflow, + OverflowByteArray valueOverflow, IHeapObject valueObject, Action objectDisposer) { - keyByteCount = GetByteCount(keyLength); - valueByteCount = GetByteCount(valueLength); - return (byte)(CurrentVersion // Already shifted - | ((long)(keyByteCount - 1) << 3) // Shift key into position; subtract 1 for 0-based - | (long)(valueByteCount - 1)); // Value does not need to be shifted; subtract 1 for 0-based + var diskLogRecord = new DiskLogRecord(recordBuffer, transientObjectIdMap, keyOverflow, valueOverflow, valueObject, objectDisposer); + recordBuffer = default; // Transfer ownership to us + return diskLogRecord; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int GetByteCount(long num) + internal static DiskLogRecord TransferFrom(ref DiskLogRecord src) { - var result = 0; - do - { - num >>= 8; - result++; - } while (num > 0); - return result; + var diskLogRecord = new DiskLogRecord(in src.logRecord, src.objectDisposer) { recordBuffer = src.recordBuffer }; + src = default; // Transfer ownership to us, and make sure we don't try to clear the logRecord + return diskLogRecord; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - static void WriteVarBytes(long value, int len, ref byte* ptr) + /// + /// Transfers a transient inline record buffer and creates our contained from it. + /// + /// The record buffer, either from IO or a copy for pending operations such as Compact or ConditionalCopyToTail. + /// The to hold the objects for the for the lifetime of this . + internal static DiskLogRecord TransferFrom(ref SectorAlignedMemory recordBuffer, ObjectIdMap transientObjectIdMap) { - for (; len > 0; len--) - { - *ptr++ = (byte)(value & 0xFF); - value >>= 8; - } - Debug.Assert(value == 0, "len too short"); + var diskLogRecord = new DiskLogRecord(recordBuffer, transientObjectIdMap); + recordBuffer = default; // Transfer ownership to us + return diskLogRecord; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - static long ReadVarBytes(int len, ref byte* ptr) + private static SectorAlignedMemory AllocateBuffer(in LogRecord logRecord, SectorAlignedBufferPool bufferPool) { - long value = 0; - for (var ii = 0; ii < len; ii++) - value |= (long)*ptr++ << (ii * 8); - return value; + var allocatedSize = RoundUp(logRecord.GetInlineRecordSizes().actualSize, Constants.kRecordAlignment); + var recordBuffer = bufferPool.Get(allocatedSize); + logRecord.RecordSpan.CopyTo(recordBuffer.RequiredValidSpan); + return recordBuffer; } public void Dispose() { - recordBuffer?.Dispose(); - recordBuffer = null; + logRecord.Dispose(objectDisposer); + logRecord = default; + + recordBuffer?.Return(); + recordBuffer = default; + objectDisposer = default; } #region ISourceLogRecord /// - public readonly bool IsPinnedKey => true; + public readonly bool IsPinnedKey => logRecord.Info.KeyIsInline; + + /// + public readonly byte* PinnedKeyPointer => logRecord.PinnedKeyPointer; /// - public readonly byte* PinnedKeyPointer + public OverflowByteArray KeyOverflow { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get { return IsPinnedKey ? (byte*)KeyInfo.dataAddress : null; } + readonly get => logRecord.KeyOverflow; + set => logRecord.KeyOverflow = value; } /// - public readonly bool IsPinnedValue => !Info.ValueIsObject; // We store all bytes inline, but we don't set ValueIsInline, per discussion in SerializeCommonVarByteFields. + public readonly bool IsPinnedValue => logRecord.Info.ValueIsInline; + + /// + public readonly byte* PinnedValuePointer => logRecord.PinnedValuePointer; /// - public readonly byte* PinnedValuePointer + public OverflowByteArray ValueOverflow { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get { return IsPinnedValue ? (byte*)ValueInfo.dataAddress : null; } + readonly get => logRecord.ValueOverflow; + set => logRecord.ValueOverflow = value; } /// - public readonly bool IsSet => physicalAddress != 0; - /// - public readonly ref RecordInfo InfoRef => ref Unsafe.AsRef((byte*)physicalAddress); + public readonly byte RecordType => logRecord.IsSet ? logRecord.RecordType : (byte)0; + /// - public readonly RecordInfo Info => *(RecordInfo*)physicalAddress; + public readonly byte Namespace => logRecord.IsSet ? logRecord.Namespace : (byte)0; + /// - public readonly ReadOnlySpan Key - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - var (length, dataAddress) = KeyInfo; - return new((byte*)dataAddress, length); - } - } + public readonly ObjectIdMap ObjectIdMap => logRecord.objectIdMap; /// - public readonly Span ValueSpan - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - if (Info.ValueIsObject) - throw new TsavoriteException("DiskLogRecord with Info.ValueIsObject does not support Span values"); - var (length, dataAddress) = ValueInfo; - return new((byte*)dataAddress, (int)length); // TODO: handle long value length - } - } + public readonly bool IsSet => logRecord.IsSet; /// - public readonly IHeapObject ValueObject - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - if (!Info.ValueIsObject) - throw new TsavoriteException("DiskLogRecord without Info.ValueIsObject does not allow ValueObject"); - Debug.Assert(valueObject is not null, "Should have deserialized valueObject by this point, or received it directly from LogRecord or PendingContext"); - return valueObject; - } - } + public ref RecordInfo InfoRef => ref logRecord.InfoRef; + /// + public readonly RecordInfo Info => logRecord.Info; /// - public readonly ReadOnlySpan RecordSpan - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - var valueInfo = ValueInfo; - if (valueInfo.length == 0) - throw new TsavoriteException("RecordSpan is not valid for records with unserialized ValueObjects"); - return new((byte*)physicalAddress, (int)GetSerializedLength()); // TODO: Handle long object sizes - } - } + public readonly ReadOnlySpan Key => logRecord.Key; /// - public readonly long ETag => Info.HasETag ? *(long*)GetETagAddress() : LogRecord.NoETag; + public readonly Span ValueSpan => logRecord.ValueSpan; /// - public readonly long Expiration => Info.HasExpiration ? *(long*)GetExpirationAddress() : 0; + public readonly IHeapObject ValueObject => logRecord.ValueObject; /// - public readonly void ClearValueObject(Action disposer) { } // Nothing done here; we dispose the object in the pending operation completion + public readonly long ETag => logRecord.ETag; /// - public readonly bool AsLogRecord(out LogRecord logRecord) - { - logRecord = default; - return false; - } + public readonly long Expiration => logRecord.Expiration; /// - public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) - { - diskLogRecord = this; - return true; - } + public readonly void ClearValueIfHeap(Action disposer) { } // Nothing to do here; we dispose the object in the pending operation or iteration completion /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly RecordFieldInfo GetRecordFieldInfo() => new() - { - KeyDataSize = Key.Length, - ValueDataSize = Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : (int)ValueInfo.length, - ValueIsObject = Info.ValueIsObject, - HasETag = Info.HasETag, - HasExpiration = Info.HasExpiration - }; - #endregion //ISourceLogRecord + public readonly bool IsMemoryLogRecord => false; - public readonly int OptionalLength => ETagLen + ExpirationLen; + /// + public readonly unsafe ref LogRecord AsMemoryLogRecordRef() => throw new TsavoriteException("DiskLogRecord cannot be returned as MemoryLogRecord"); - private readonly int ETagLen => Info.HasETag ? LogRecord.ETagSize : 0; - private readonly int ExpirationLen => Info.HasExpiration ? LogRecord.ExpirationSize : 0; + /// + public readonly bool IsDiskLogRecord => true; - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private readonly long GetOptionalStartAddress() - { - var (length, dataAddress) = ValueInfo; - return dataAddress + length; - } + /// + public readonly unsafe ref DiskLogRecord AsDiskLogRecordRef() => ref Unsafe.AsRef(in this); + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private readonly long GetETagAddress() => GetOptionalStartAddress(); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private readonly long GetExpirationAddress() => GetETagAddress() + ETagLen; + public readonly RecordFieldInfo GetRecordFieldInfo() => logRecord.GetRecordFieldInfo(); - #region Serialized Record Creation - /// - /// Serialize for Read or RMW operations, called by PendingContext; these have no Value but have TInput, TOutput, and TContext, which are handled by PendingContext. - /// - /// Record key - /// Allocator for backing storage - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SerializeForPendingReadOrRMW(ReadOnlySpan key, SectorAlignedBufferPool bufferPool) - => SerializeForPendingReadOrRMW(key, bufferPool, ref recordBuffer); + #endregion //ISourceLogRecord + #region Serialization to and from expanded record format /// - /// Serialize for Read or RMW operations, called by PendingContext; these have no Value but have TInput, TOutput, and TContext, which are handled by PendingContext. + /// Serialize a log record (which may be in-memory or IO'd ) to the + /// in inline-expanded format, with the Overflow Keys and Values and Object Values serialized inline to the Key and Value spans. + /// The serialized layout is: + /// + /// Inline portion of the LogRecord: RecordInfo, IndicatorWord, Key and Value data (each of 4 byte length, which restores to object Id), optionals (ETag, Expiration, ObjectLogPosition, ...) + /// Key data, if key is Overflow + /// Value data, if value is Overflow or Object + /// /// - /// This overload may be called either directly for a caller who owns the , or with this.allocatedRecord. - /// Record key - /// Allocator for backing storage - /// The allocated record; may be owned by this instance, or owned by the caller for reuse - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SerializeForPendingReadOrRMW(ReadOnlySpan key, SectorAlignedBufferPool bufferPool, ref SectorAlignedMemory allocatedRecord) + /// + /// This is used for migration and replication, and output.SpanByteAndMemory is a span of the remaining space in the network buffer. + /// This allocates if needed; in that case the caller will flush the network buffer and retry with the full length. + /// The record stream is prefixed with the int length of the stream. RespReadUtils.GetSerializedRecordSpan sets up for deserialization from the network buffer. + /// + /// If ., it points directly to the network buffer so we include the length prefix in the output. + public static void Serialize(in TSourceLogRecord srcLogRecord, IObjectSerializer valueObjectSerializer, MemoryPool memoryPool, ref SpanByteAndMemory output) + where TSourceLogRecord : ISourceLogRecord { - // OptionalSize (ETag and Expiration) is not considered here; those are specified in the Input, which is serialized separately by PendingContext. - - long recordSize; - byte* ptr; - - // Value is a span so we can use RecordIsInline format, so both key and value are int length. - recordSize = RecordInfo.GetLength() + key.TotalSize() + LogField.InlineLengthPrefixSize; - - if (allocatedRecord is not null) - allocatedRecord.pool.EnsureSize(ref allocatedRecord, (int)recordSize); + if (srcLogRecord.IsMemoryLogRecord) + SerializeLogRecord(in srcLogRecord.AsMemoryLogRecordRef(), valueObjectSerializer, memoryPool, ref output); else - allocatedRecord = bufferPool.Get((int)recordSize); - - physicalAddress = (long)allocatedRecord.GetValidPointer(); - ptr = (byte*)physicalAddress; - - *(RecordInfo*)ptr = default; - ptr += RecordInfo.GetLength(); - - InfoRef.SetKeyIsInline(); - *(int*)ptr = key.Length; - ptr += LogField.InlineLengthPrefixSize; - key.CopyTo(new Span(ptr, key.Length)); - ptr += key.Length; - - InfoRef.SetValueIsInline(); - *(int*)ptr = 0; - - allocatedRecord.available_bytes = (int)recordSize; - return; - } - - /// - /// Serialize for Compact, Scan, Conditional Pending Operations, Migration, Replication, etc. The logRecord comes from the in-memory log; there is no associated TInput, TOutput, TContext. - /// - /// The log record. This may be either in-memory or from disk IO - /// Allocator for backing storage - /// Serializer for the value object; if null, do not serialize (carry the valueObject (if any) through from the logRecord instead) - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void Serialize(in LogRecord logRecord, SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer) - => Serialize(in logRecord, bufferPool, valueSerializer, ref recordBuffer); - - /// - /// Serialize for Compact, Pending Operations, etc. There is no associated TInput, TOutput, TContext for these as it is just a direct copy of data. - /// - /// The log record. This may be either in-memory or from disk IO - /// Allocator for backing storage - /// Serializer for the value object; if null, do not serialize (carry the valueObject (if any) through from the logRecord instead) - /// The allocated record; may be owned by this instance, or owned by the caller for reuse - /// This overload may be called either directly for a caller who owns the , or with this.allocatedRecord. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void Serialize(in LogRecord logRecord, SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer, ref SectorAlignedMemory allocatedRecord) - { - if (logRecord.Info.RecordIsInline) { - DirectCopyRecord(logRecord.ActualRecordSize, logRecord.physicalAddress, bufferPool, ref allocatedRecord); - return; + if (!srcLogRecord.IsDiskLogRecord) + throw new TsavoriteException("Unknown TSourceLogRecord type"); + SerializeLogRecord(in srcLogRecord.AsDiskLogRecordRef().logRecord, valueObjectSerializer, memoryPool, ref output); } - // Record is not inline so we must use the varbyte format: create the indicator byte and space-optimized length representation. - - // If we have an object and we don't serialize it, we don't need to allocate space for it. - // Value length prefix on the disk is a long, as it may be an object. - var valueLength = !logRecord.Info.ValueIsObject - ? logRecord.ValueSpan.Length - : (valueSerializer is not null ? logRecord.ValueObject.DiskSize : 0); - - var indicatorByte = CreateIndicatorByte(logRecord.Key.Length, valueLength, out var keyLengthByteCount, out var valueLengthByteCount); - - var recordSize = RecordInfo.GetLength() - + 1 // indicator byte - + keyLengthByteCount + logRecord.Key.Length - + valueLengthByteCount + valueLength - + logRecord.OptionalLength; - - // This writes the value length, but not value data - var ptr = SerializeCommonVarByteFields(logRecord.Info, indicatorByte, logRecord.Key, keyLengthByteCount, valueLength, valueLengthByteCount, recordSize, bufferPool, ref allocatedRecord); - - // Set the value - if (!logRecord.Info.ValueIsObject) - logRecord.ValueSpan.CopyTo(new Span(ptr, (int)valueLength)); - else + static void SerializeLogRecord(in LogRecord logRecord, IObjectSerializer valueObjectSerializer, MemoryPool memoryPool, ref SpanByteAndMemory output) { - if (valueSerializer is not null) + if (logRecord.Info.RecordIsInline) + _ = DirectCopyInlinePortionOfRecord(in logRecord, heapSize: 0, memoryPool, ref output); + else { - var stream = new UnmanagedMemoryStream(ptr, logRecord.ValueObject.DiskSize, logRecord.ValueObject.DiskSize, FileAccess.ReadWrite); - valueSerializer.BeginSerialize(stream); - valueSerializer.Serialize(logRecord.ValueObject); - valueSerializer.EndSerialize(); + // TODO: long value sizes (larger than the network buffer) are currently not supported; need to create a chunked protocol that will write incrementally to the + // network buffer, possibly using a callback to tell the network buffer to flush and reset and update the output available length. + if (!logRecord.ValueObject.SerializedSizeIsExact) + throw new TsavoriteException("Currently we do not support in-memory serialization of objects that do not support SerializedSizeIsExact"); + + var heapSize = logRecord.Info.KeyIsOverflow ? logRecord.KeyOverflow.Length : 0; + if (logRecord.Info.ValueIsOverflow) + heapSize += logRecord.ValueOverflow.Length; + else if (logRecord.Info.ValueIsObject) + heapSize += (int)logRecord.ValueObject.SerializedSize; + var inlineRecordSize = DirectCopyInlinePortionOfRecord(in logRecord, heapSize, memoryPool, ref output); + SerializeHeapObjects(in logRecord, inlineRecordSize, heapSize, valueObjectSerializer, ref output); } - else - valueObject = logRecord.ValueObject; - InfoRef.SetValueIsObject(); - } - ptr += valueLength; - CopyOptionals(in logRecord, ref ptr); - - allocatedRecord.available_bytes = (int)recordSize; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private unsafe byte* SerializeCommonVarByteFields(RecordInfo recordInfo, byte indicatorByte, ReadOnlySpan key, int keyLengthByteCount, - long valueLength, int valueLengthByteCount, long recordSize, SectorAlignedBufferPool bufferPool, ref SectorAlignedMemory allocatedRecord) - { - EnsureAllocation(recordSize, bufferPool, ref allocatedRecord); - return SerializeCommonVarByteFields(recordInfo, indicatorByte, key, keyLengthByteCount, valueLength, valueLengthByteCount, allocatedRecord.GetValidPointer()); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static unsafe byte* SerializeCommonVarByteFields(RecordInfo recordInfo, byte indicatorByte, ReadOnlySpan key, int keyLengthByteCount, - long valueLength, int valueLengthByteCount, byte* ptr) - { - // RecordInfo is a stack copy so we can modify it here. Write the key "inline" status; keys are always inline in DiskLogRecord, as they cannot - // be object. Value, however, does not set ValueIsInline for varbyte records, because then Info.RecordIsInline would be true. Instead we clear - // ValueIsInline and use ValueIsObject to determine whether the serialized bytes are string or object, and whether DeserializeValueObject can - // be used. WARNING: ToString() may AV when stepping through here in the debugger, until we have the lengths correctly set. - recordInfo.SetKeyIsInline(); - recordInfo.ClearValueIsInline(); - *(RecordInfo*)ptr = recordInfo; - ptr += RecordInfo.GetLength(); - - // Set the indicator and lengths - *ptr++ = indicatorByte; - WriteVarBytes(key.Length, keyLengthByteCount, ref ptr); - WriteVarBytes(valueLength, valueLengthByteCount, ref ptr); - - // Copy the key but not the value; the caller does that. - key.CopyTo(new Span(ptr, key.Length)); - - // Return the pointer to the value data space (immediately following the key data space; the value length was already written above). - return ptr + key.Length; - } - - /// - /// Deserialize the current value span to a valueObject. - /// - /// Serializer for the value object; if null, do not serialize (carry the valueObject (if any) through from the logRecord instead) - /// This overload converts from LogRecord to DiskLogRecord. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public IHeapObject DeserializeValueObject(IObjectSerializer valueSerializer) - { - if (valueObject is not null) - return valueObject; - if (!Info.ValueIsObject) - return valueObject = default; - - var (length, dataAddress) = ValueInfo; - var stream = new UnmanagedMemoryStream((byte*)dataAddress, length); - valueSerializer.BeginDeserialize(stream); - valueSerializer.Deserialize(out valueObject); - valueSerializer.EndDeserialize(); - return valueObject; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void DirectCopyRecord(long recordSize, long srcPhysicalAddress, SectorAlignedBufferPool bufferPool, ref SectorAlignedMemory allocatedRecord) - { - EnsureAllocation(recordSize, bufferPool, ref allocatedRecord); - Buffer.MemoryCopy((byte*)srcPhysicalAddress, (byte*)physicalAddress, recordSize, recordSize); - allocatedRecord.available_bytes = (int)recordSize; - } - - /// - /// Directly copies a record in varbyte format to the SpanByteAndMemory. Allocates if needed. - /// - /// If ., it points directly to the network buffer so we include the length prefix in the output. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static void DirectCopyRecord(long srcPhysicalAddress, long srcRecordSize, ref SpanByteAndMemory output, MemoryPool memoryPool) - { - // TotalSize includes the length prefix, which is included in the output stream if we can write to the SpanByte. - if (output.IsSpanByte && output.SpanByte.TotalSize >= (int)srcRecordSize) // TODO: long value sizes - { - var outPtr = output.SpanByte.ToPointer(); - *(int*)outPtr = (int)srcRecordSize; - Buffer.MemoryCopy((byte*)srcPhysicalAddress, outPtr + sizeof(int), output.SpanByte.Length, srcRecordSize); - output.SpanByte.Length = (int)srcRecordSize; - return; - } - - // Do not include the length prefix in the output stream; this is done by the caller before writing the stream, from the SpanByte.Length we set here. - output.EnsureHeapMemorySize((int)srcRecordSize + sizeof(int), memoryPool); - fixed (byte* outPtr = output.MemorySpan) - { - Buffer.MemoryCopy((byte*)srcPhysicalAddress, outPtr, srcRecordSize, srcRecordSize); - output.Length = (int)srcRecordSize; } } /// - /// Serializes a record in varbyte format to the SpanByteAndMemory. Allocates if needed. + /// Directly copies a record in inline varbyte format to the SpanByteAndMemory. Allocates if needed. /// /// If ., it points directly to the network buffer so we include the length prefix in the output. [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static long SerializeVarbyteRecord(in TSourceLogRecord srcLogRecord, long srcPhysicalAddress, IObjectSerializer valueSerializer, - ref SpanByteAndMemory output, MemoryPool memoryPool) - where TSourceLogRecord : ISourceLogRecord + private static int DirectCopyInlinePortionOfRecord(in LogRecord logRecord, int heapSize, MemoryPool memoryPool, ref SpanByteAndMemory output) { - // If we have an object and we don't serialize it, we don't need to allocate space for it. - // Value length prefix on the disk is a long, as it may be an object. - var valueLength = !srcLogRecord.Info.ValueIsObject - ? srcLogRecord.ValueSpan.Length - : srcLogRecord.ValueObject.DiskSize; - - var indicatorByte = CreateIndicatorByte(srcLogRecord.Key.Length, valueLength, out var keyLengthByteCount, out var valueLengthByteCount); - - var recordSize = RecordInfo.GetLength() - + 1 // indicator byte - + keyLengthByteCount + srcLogRecord.Key.Length - + valueLengthByteCount + valueLength - + (srcLogRecord.Info.HasETag ? LogRecord.ETagSize : 0) - + (srcLogRecord.Info.HasExpiration ? LogRecord.ExpirationSize : 0); - - // Copy in varbyte format. If the object was not serialized in srcLogRecord we need to revise the length which is varbyte, so we can't just Buffer copy the whole record. - var serializedSize = 0L; - - // TotalSize includes the length prefix, which is included in the output stream if we can write to the SpanByte. - if (output.IsSpanByte && output.SpanByte.TotalSize >= (int)recordSize) // TODO: long value sizes + // TotalSize includes the length prefix, which is included in the output stream if we can write directly to the SpanByte, + // which is a span in the network buffer. + var inlineRecordSize = logRecord.ActualRecordSize; + var totalSize = inlineRecordSize + sizeof(int) + heapSize; + if (output.IsSpanByte && output.SpanByte.TotalSize >= totalSize) { var outPtr = output.SpanByte.ToPointer(); - *(int*)outPtr = (int)recordSize; - serializedSize = SerializeVarbyteRecordToPinnedPointer(in srcLogRecord, outPtr + sizeof(int), indicatorByte, keyLengthByteCount, valueLength, valueLengthByteCount, valueSerializer); - output.SpanByte.Length = (int)serializedSize; + *(int*)outPtr = inlineRecordSize; + Buffer.MemoryCopy((byte*)logRecord.physicalAddress, outPtr + sizeof(int), inlineRecordSize, inlineRecordSize); } else { // Do not include the length prefix in the output stream; this is done by the caller before writing the stream, from the SpanByte.Length we set here. - output.EnsureHeapMemorySize((int)recordSize + sizeof(int), memoryPool); + totalSize -= sizeof(int); + output.EnsureHeapMemorySize(totalSize, memoryPool); fixed (byte* outPtr = output.MemorySpan) - { - serializedSize = SerializeVarbyteRecordToPinnedPointer(in srcLogRecord, outPtr, indicatorByte, keyLengthByteCount, valueLength, valueLengthByteCount, valueSerializer); - output.Length = (int)recordSize; - } + Buffer.MemoryCopy((byte*)logRecord.physicalAddress, outPtr, inlineRecordSize, inlineRecordSize); } - Debug.Assert(serializedSize == recordSize, $"Serialized size {serializedSize} does not match expected size {recordSize}"); - return serializedSize; + return inlineRecordSize; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static long SerializeVarbyteRecordToPinnedPointer(in TSourceLogRecord srcLogRecord, byte* ptr, byte indicatorByte, - int keyLengthByteCount, long valueLength, int valueLengthByteCount, IObjectSerializer valueSerializer) - where TSourceLogRecord : ISourceLogRecord + private static void SerializeHeapObjects(in LogRecord logRecord, int inlineRecordSize, int heapSize, IObjectSerializer valueObjectSerializer, ref SpanByteAndMemory output) { - var dstRecordInfoPtr = (RecordInfo*)ptr; - ptr = SerializeCommonVarByteFields(srcLogRecord.Info, indicatorByte, srcLogRecord.Key, keyLengthByteCount, valueLength, valueLengthByteCount, ptr); + if (logRecord.Info.RecordIsInline) + return; - // If the srcLogRecord has an object already serialized to its ValueSpan, then we will not be here; LogRecord does not serialize values to the ValueSpan, - // and the DiskLogRecord case where valueInfo.length > 0 has already been tested for in the caller. - if (!srcLogRecord.Info.ValueIsObject) - srcLogRecord.ValueSpan.CopyTo(new Span(ptr, (int)valueLength)); - else + // Serialize Key then Value, just like the Object log file. And this will be easy to modify for future chunking of multi-networkBuffer Keys and Values. + var outputOffset = inlineRecordSize; + if (logRecord.Info.KeyIsOverflow) { - var stream = new UnmanagedMemoryStream(ptr, srcLogRecord.ValueObject.DiskSize, srcLogRecord.ValueObject.DiskSize, FileAccess.ReadWrite); - valueSerializer.BeginSerialize(stream); - valueSerializer.Serialize(srcLogRecord.ValueObject); - valueSerializer.EndSerialize(); + var overflow = logRecord.KeyOverflow; + overflow.ReadOnlySpan.CopyTo(output.Span.Slice(inlineRecordSize)); + outputOffset += overflow.Length; } - ptr += valueLength; - if (srcLogRecord.Info.HasETag) + var valueObjectLength = 0UL; + if (logRecord.Info.ValueIsOverflow) { - dstRecordInfoPtr->SetHasETag(); - *(long*)ptr = srcLogRecord.ETag; - ptr += LogRecord.ETagSize; + var overflow = logRecord.ValueOverflow; + overflow.ReadOnlySpan.CopyTo(output.Span.Slice(inlineRecordSize)); } - - if (srcLogRecord.Info.HasExpiration) + else { - dstRecordInfoPtr->SetHasExpiration(); - *(long*)ptr = srcLogRecord.Expiration; - ptr += LogRecord.ExpirationSize; + Debug.Assert(logRecord.Info.ValueIsObject, "Expected ValueIsObject to be true"); + if (output.IsSpanByte) + valueObjectLength = DoSerialize(logRecord.ValueObject, valueObjectSerializer, output.SpanByte.ToPointer(), output.Length); + else + { + fixed (byte* ptr = output.MemorySpan.Slice(inlineRecordSize)) + valueObjectLength = DoSerialize(logRecord.ValueObject, valueObjectSerializer, ptr, output.Length); + } } - return ptr - (byte*)dstRecordInfoPtr; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void EnsureAllocation(long recordSize, SectorAlignedBufferPool bufferPool, ref SectorAlignedMemory allocatedRecord) - { - var allocatedSize = RoundUp((int)recordSize, Constants.kRecordAlignment); - if (allocatedRecord is not null) - allocatedRecord.pool.EnsureSize(ref allocatedRecord, allocatedSize); + // Create a temp LogRecord over the output data so we can store the lengths in serialized format, using the offset to the serialized + // part of the buffer as a fake file offset (implicitly for segment 0). + var fakeFilePos = new ObjectLogFilePositionInfo((ulong)inlineRecordSize, segSizeBits: 0); + if (output.IsSpanByte) + { + var serializedLogRecord = new LogRecord((long)output.SpanByte.ToPointer()); + serializedLogRecord.SetObjectLogRecordStartPositionAndLength(fakeFilePos, valueObjectLength); + } else - allocatedRecord = bufferPool.Get(allocatedSize); - physicalAddress = (long)allocatedRecord.GetValidPointer(); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private readonly void CopyOptionals(ref readonly TSourceLogRecord logRecord, ref byte* ptr) - where TSourceLogRecord : ISourceLogRecord - { - if (logRecord.Info.HasETag) { - InfoRef.SetHasETag(); - *(long*)ptr = logRecord.ETag; - ptr += LogRecord.ETagSize; + fixed (byte* ptr = output.MemorySpan.Slice(0, inlineRecordSize)) + { + var serializedLogRecord = new LogRecord((long)ptr, logRecord.objectIdMap); + serializedLogRecord.SetObjectLogRecordStartPositionAndLength(fakeFilePos, valueObjectLength); + serializedLogRecord = new LogRecord((long)ptr); // Reset to clear objectIdMap because it may be the one in the main log and we pass in a transient one when deserializing + } } - if (logRecord.Info.HasExpiration) + static ulong DoSerialize(IHeapObject valueObject, IObjectSerializer valueObjectSerializer, byte* destPtr, int destLength) { - InfoRef.SetHasExpiration(); - *(long*)ptr = logRecord.Expiration; - ptr += LogRecord.ExpirationSize; + var stream = new UnmanagedMemoryStream(destPtr, destLength, destLength, FileAccess.ReadWrite); + valueObjectSerializer.BeginSerialize(stream); + valueObjectSerializer.Serialize(valueObject); + valueObjectSerializer.EndSerialize(); + var valueLength = (ulong)stream.Position; + Debug.Assert((ulong)valueObject.SerializedSize == valueLength, $"valueObject.SerializedSize ({valueObject.SerializedSize}) != valueLength ({valueLength})"); + return valueLength; } } /// - /// The record is directly copyable if it has a serialized value; in that case it is in linear format and any deserialized object can be ignored. - /// - public readonly bool IsDirectlyCopyable => ValueInfo.length > 0; - - /// - /// Clone from a temporary (having no overflow ) to this instance's . - /// - /// - /// Allocator for backing storage - /// If true, prefer the deserialized object over copying the serialized value; this saves space for pending operations - public void CloneFrom(ref DiskLogRecord inputDiskLogRecord, SectorAlignedBufferPool bufferPool, bool preferDeserializedObject) - => CloneFrom(ref inputDiskLogRecord, bufferPool, ref recordBuffer, preferDeserializedObject); - - /// - /// Clone from a temporary (having no overflow ) to a longer-lasting one. + /// Deserialize from a over a stream of bytes created by . /// - /// - /// Allocator for backing storage - /// The allocated record; may be owned by this instance, or owned by the caller for reuse - /// If true, prefer the deserialized object over copying the serialized value; this saves space for pending operations - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CloneFrom(ref DiskLogRecord inputDiskLogRecord, SectorAlignedBufferPool bufferPool, ref SectorAlignedMemory allocatedRecord, bool preferDeserializedObject) - { - Debug.Assert(inputDiskLogRecord.IsSet, "inputDiskLogRecord is not set"); - - if (!inputDiskLogRecord.Info.ValueIsObject || !preferDeserializedObject) + /// + public static DiskLogRecord Deserialize(PinnedSpanByte recordSpan, IObjectSerializer valueObjectSerializer, ObjectIdMap transientObjectIdMap, + TStoreFunctions storeFunctions) + where TStoreFunctions : IStoreFunctions + { + // Serialize() did not change the state of the KeyIsInline/ValueIsInline/ValueIsObject bits, but it did change the value at the ObjectId + // location to be serialized length. Create a transient logRecord to decode these and restore the objectId values. + var ptr = recordSpan.ToPointer(); + var serializedLogRecord = new LogRecord((long)ptr, transientObjectIdMap); + if (serializedLogRecord.Info.RecordIsInline) + return new(serializedLogRecord, obj => { }); + var offset = serializedLogRecord.GetObjectLogRecordStartPositionAndLengths(out var keyLength, out var valueLength); + + // Note: Similar logic to this is in ObjectLogReader.ReadObjects. + if (serializedLogRecord.Info.KeyIsOverflow) { - Debug.Assert(inputDiskLogRecord.ValueInfo.length > 0, "inputDiskLogRecord value length should be > 0"); - DirectCopyRecord(inputDiskLogRecord.GetSerializedLength(), inputDiskLogRecord.physicalAddress, bufferPool, ref allocatedRecord); - return; + // This assignment also allocates the slot in ObjectIdMap. The varbyte length info should be unchanged from ObjectIdSize. + serializedLogRecord.KeyOverflow = new OverflowByteArray(keyLength, startOffset: 0, endOffset: 0, zeroInit: false); + recordSpan.ReadOnlySpan.Slice((int)offset, keyLength).CopyTo(serializedLogRecord.KeyOverflow.Span); + offset += (uint)keyLength; } - // The source DiskLogRecord has a Value object rather than serialized bytes so it is in varbyte format. Copy everything up to the end of the key; ignore - // the serialized value data even if the source source still has it. inputDiskLogRecord can Return() its recordBuffer, releasing any serialized value data. - var (length, dataAddress) = inputDiskLogRecord.KeyInfo; - var partialRecordSize = dataAddress + length - inputDiskLogRecord.physicalAddress; - var allocatedRecordSize = partialRecordSize + inputDiskLogRecord.OptionalLength; - - if (allocatedRecord is not null) - allocatedRecord.pool.EnsureSize(ref allocatedRecord, (int)allocatedRecordSize); // TODO handle 'long' valuelength - else - allocatedRecord = bufferPool.Get((int)allocatedRecordSize); // TODO handle 'long' valuelength - physicalAddress = (long)allocatedRecord.GetValidPointer(); - - Buffer.MemoryCopy((void*)inputDiskLogRecord.physicalAddress, (void*)physicalAddress, partialRecordSize, partialRecordSize); - - // Clear the value length in the indicator byte, as we did not copy any serialized data. - var ptr = (byte*)physicalAddress + RecordInfo.GetLength(); - *ptr = (byte)(*ptr & ~kValueLengthBitMask); - - // Set the Value - InfoRef.SetValueIsObject(); - valueObject = inputDiskLogRecord.valueObject; - - // Set the Optionals - ptr = (byte*)physicalAddress + partialRecordSize; - CopyOptionals(ref inputDiskLogRecord, ref ptr); - allocatedRecord.available_bytes = (int)allocatedRecordSize; - } - - /// - /// Transfer memory ownership from a temporary to a longer-lasting one. - /// - /// This is separate from to ensure the caller - /// is prepared to handle the implications of the transfer - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void TransferFrom(ref DiskLogRecord inputDiskLogRecord) - { - Debug.Assert(inputDiskLogRecord.IsSet, "inputDiskLogRecord is not set"); - Debug.Assert(inputDiskLogRecord.recordBuffer is not null, "inputDiskLogRecord does not own its memory"); - - recordBuffer?.Return(); - recordBuffer = inputDiskLogRecord.recordBuffer; - inputDiskLogRecord.recordBuffer = null; // Transfers ownership - physicalAddress = (long)recordBuffer.GetValidPointer(); - } - - /// - /// Serialize a log record to the in DiskLogRecord format, with the objectValue - /// serialized to the Value span if it is not already there in the source (there will be no ValueObject in the result). - /// Allocates if needed. This is used for migration. - /// - /// If ., it points directly to the network buffer so we include the length prefix in the output. - public static void Serialize(in TSourceLogRecord srcLogRecord, IObjectSerializer valueSerializer, ref SpanByteAndMemory output, MemoryPool memoryPool) - where TSourceLogRecord : ISourceLogRecord - { - if (srcLogRecord.AsLogRecord(out var logRecord)) + if (serializedLogRecord.Info.ValueIsOverflow) { - if (logRecord.Info.RecordIsInline) - DirectCopyRecord(logRecord.physicalAddress, logRecord.ActualRecordSize, ref output, memoryPool); - else - _ = SerializeVarbyteRecord(in logRecord, logRecord.physicalAddress, valueSerializer, ref output, memoryPool); - return; + // This assignment also allocates the slot in ObjectIdMap. The varbyte length info should be unchanged from ObjectIdSize. + serializedLogRecord.ValueOverflow = new OverflowByteArray((int)valueLength, startOffset: 0, endOffset: 0, zeroInit: false); + recordSpan.ReadOnlySpan.Slice((int)offset, (int)valueLength).CopyTo(serializedLogRecord.KeyOverflow.Span); } - - if (!srcLogRecord.AsDiskLogRecord(out var diskLogRecord)) - throw new TsavoriteException("Unknown TSourceLogRecord type"); - Debug.Assert(diskLogRecord.Info.KeyIsInline, "DiskLogRecord key should always be inline"); - - var valueInfo = diskLogRecord.ValueInfo; - - // Either the value is present in serialized byte form or there should be an object. If there is no object we have nothing to serialize - if (diskLogRecord.Info.RecordIsInline - || (diskLogRecord.Info.KeyIsInline && (valueInfo.length > 0 || diskLogRecord.valueObject is null))) + else { - DirectCopyRecord(diskLogRecord.physicalAddress, diskLogRecord.GetSerializedLength(), ref output, memoryPool); - return; + var stream = new UnmanagedMemoryStream(ptr + offset, (int)valueLength); + valueObjectSerializer.BeginDeserialize(stream); + valueObjectSerializer.Deserialize(out var valueObject); + serializedLogRecord.ValueObject = valueObject; + valueObjectSerializer.EndDeserialize(); } - _ = SerializeVarbyteRecord(in diskLogRecord, diskLogRecord.physicalAddress, valueSerializer, ref output, memoryPool); + return new(serializedLogRecord, obj => storeFunctions.DisposeValueObject(obj, DisposeReason.DeserializedFromDisk)); } - #endregion //Serialized Record Creation - - /// - public override readonly string ToString() - { - static string bstr(bool value) => value ? "T" : "F"; - var valueString = Info.ValueIsObject ? $"obj:{ValueObject}" : ValueSpan.ToString(); - var eTag = Info.HasETag ? ETag.ToString() : "-"; - var expiration = Info.HasExpiration ? Expiration.ToString() : "-"; - return $"ri {Info} | key {Key.ToShortString(20)} | val {valueString} | HasETag {bstr(Info.HasETag)}:{eTag} | HasExpiration {bstr(Info.HasExpiration)}:{expiration}"; - } + #endregion Serialization to and from expanded record format } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/HeapObjectBase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/HeapObjectBase.cs new file mode 100644 index 00000000000..77da9190ce9 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/HeapObjectBase.cs @@ -0,0 +1,183 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; +using System.IO; +using System.Text; +using System.Threading; +using static Tsavorite.core.Utility; + +namespace Tsavorite.core +{ + public struct ObjectSizes + { + /// In-memory size, including .NET object overheads + public long HeapMemory; + + /// Serialized size, for disk IO or other storage + public long Serialized; + + /// Serialized size, for disk IO or other storage + public bool SerializedIsExact; + public ObjectSizes(long heap, long serialized) + { + HeapMemory = heap; + Serialized = serialized + sizeof(byte); // Additional byte for GarnetObjectBase.Type + } + + public ObjectSizes(long heap, long serialized, bool serializedIsExact) + : this(heap, serialized) + { + this.SerializedIsExact = serializedIsExact; + } + + [Conditional("DEBUG")] + public void Verify() => Debug.Assert(HeapMemory >= 0 && Serialized >= 0, $"Invalid sizes [{HeapMemory}, {Serialized}]"); + } + + /// + /// The base class for heap Value Objects in Tsavorite. + /// + public abstract class HeapObjectBase : IHeapObject + { + /// + public long HeapMemorySize { get => sizes.HeapMemory; set => sizes.HeapMemory = value; } + + /// + public long SerializedSize { get => sizes.Serialized; set => sizes.Serialized = value; } + + /// + public bool SerializedSizeIsExact { get => sizes.SerializedIsExact; internal set => sizes.SerializedIsExact = value; } + + /// Combination of object sizes for memory and disk. + public ObjectSizes sizes; + + /// The current internal serialization phase of the object. + SerializationPhase SerializationPhase + { + get => (SerializationPhase)serializationPhaseInt; + set => serializationPhaseInt = (int)value; + } + + int serializationPhaseInt; + + /// The internal serialized bytes of the object. + byte[] serializedBytes; + + /// + /// Create a cloned (shallow copy) of this object + /// + /// The implementation of this method should NOT copy . + public abstract HeapObjectBase Clone(); + + /// + /// Serialize to the binary writer. + /// + public abstract void DoSerialize(BinaryWriter writer); + + /// + /// Transition the serialization phase of the object. + /// + public bool MakeTransition(SerializationPhase expectedPhase, SerializationPhase nextPhase) + => Interlocked.CompareExchange(ref serializationPhaseInt, (int)nextPhase, (int)expectedPhase) == (int)expectedPhase; + + /// + public abstract void Dispose(); + + /// + public abstract void WriteType(BinaryWriter writer, bool isNull); + + /// + public void Serialize(BinaryWriter writer) + { + while (true) + { + // This is probably called from Flush, including for checkpoints. If CopyUpdate() has already serialized the object, we will use that + // serialized state. Otherwise, we will serialize the object directly to the writer, and not create the serialized byte[]; only + // CopyUpdater does that, as it must ensure the object's (v1) data is not changed during the checkpoint. + if (SerializationPhase == SerializationPhase.REST && MakeTransition(SerializationPhase.REST, SerializationPhase.SERIALIZING)) + { + // Directly serialize to wire, do not cache serialized state + WriteType(writer, isNull: false); + DoSerialize(writer); + SerializationPhase = SerializationPhase.REST; + return; + } + + // If we are here, SerializationPhase is one of the .SERIALIZ* states. This means that one of the following is true: + // - Another thread is currently serializing this object (e.g. checkpoint and eviction) + // - CopyUpdate() is serializing this object + // - Serialization is complete. If the serializedBytes array is null, it means the checkpoint has completed and cleared it + // and the object has been superseded in the database so is no longer reachable, so we can write a null indicator. + + if (SerializationPhase == SerializationPhase.SERIALIZED) + { + // If serialized state is cached, use that + var _serialized = serializedBytes; + if (_serialized != null) + { + WriteType(writer, isNull: false); + writer.Write(_serialized); + } + else + { + // Write null object to stream + WriteType(writer, isNull: true); + } + return; + } + + Thread.Yield(); + } + } + + /// + public void CacheSerializedObjectData(ref LogRecord srcLogRecord, ref LogRecord dstLogRecord) + { + // We'll want to clone the source object to the destination log record so PostCopyUpdater can modify it. + // Note that this does a shallow copy of the object's internal structures (e.g. List<>), which means subsequent modifications of newValue + // in the (v+1) version of the record will modify data seen from the 'this' in the (v) record. Normally this is OK because the (v) version + // of the record is not reachable once the (v+1) version is inserted, but if a checkpoint is ongoing, the (v) version is part of that. + // (If this was an Overflow instead of an Object, then PostCopyUpdater will follow the normal RCU logic, creating a new ValueSpan which will + // probably (but not necessarily) be another Overflow.) + Debug.Assert(ReferenceEquals(this, srcLogRecord.ValueObject), $"{GetCurrentMethodName()} must be called on the Source LogRecord's ValueObject."); + Debug.Assert(dstLogRecord.Info.ValueIsObject, $"{GetCurrentMethodName()} must be called for non-object {nameof(dstLogRecord)}."); + Debug.Assert(dstLogRecord.Info.IsInNewVersion, $"{GetCurrentMethodName()} must only be called when taking a checkpoint."); + _ = dstLogRecord.TrySetValueObject(srcLogRecord.ValueObject.Clone()); + + // Create a serialized version for checkpoint version (v). This is only done for CopyUpdate during a checkpoint, to preserve the (v) data + // of the object during a checkpoint while the (v+1) version of the record may modify the shallow-copied internal structures. + var oldValueObject = (HeapObjectBase)srcLogRecord.ValueObject; + while (true) + { + if (oldValueObject.SerializationPhase == (int)SerializationPhase.REST && oldValueObject.MakeTransition(SerializationPhase.REST, SerializationPhase.SERIALIZING)) + { + using var ms = new MemoryStream(); + using var writer = new BinaryWriter(ms, Encoding.UTF8); + oldValueObject.DoSerialize(writer); + oldValueObject.serializedBytes = ms.ToArray(); + + oldValueObject.SerializationPhase = SerializationPhase.SERIALIZED; // This is the only place .SERIALIZED is set + break; + } + + // If we're here, serializationState is one of the .SERIALIZ* states. CopyUpdate has a lock on the tag chain, so no other thread will + // be running CopyUpdate. Therefore there are two possibilities: + // 1. CopyUpdate has been called before and the state is .SERIALIZED and '_serialized' is created. We're done. + // 2. Serialize() is running (likely in a Flush()) and the state is .SERIALIZING. We will Yield and loop to wait for it to finish. + if (oldValueObject.SerializationPhase >= SerializationPhase.SERIALIZED) + break; + + _ = Thread.Yield(); + } + } + + /// + public void ClearSerializedObjectData() + { + // Clear the serialized data, so it can be GC'd + serializedBytes = null; + SerializationPhase = SerializationPhase.REST; // Reset to initial state + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs index fd2c3180a7d..c3d9dfbd4b9 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs @@ -16,10 +16,15 @@ public interface IAllocator : IAllocatorCallbacks GetBase() where TAllocator : IAllocator; - /// Initialize the value to span the address range. - /// The start of the record (address of its ). + /// Whether this allocator uses a separate object log + bool HasObjectLog { get; } + + /// Initialize the varbyte lengths to key length and a value that spans the address range, and the serialize the key into the record. + /// The key to be copied into the record + /// The logical address of the new record /// The record size info, which tells us the value size and whether that is overflow. - void InitializeValue(long physicalAddress, in RecordSizeInfo sizeInfo); + /// The new log record being initialized + void InitializeRecord(ReadOnlySpan key, long logicalAddress, in RecordSizeInfo sizeInfo, ref LogRecord logRecord); /// Get copy destination size for RMW, taking Input into account RecordSizeInfo GetRMWCopyRecordSize(in TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) @@ -56,18 +61,18 @@ RecordSizeInfo GetUpsertRecordSizeMark the page that contains as dirty atomically void MarkPageAtomic(long logicalAddress, long version); - /// Get segment offsets - long[] GetSegmentOffsets(); // TODO remove - - /// Serialize key to log - void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord); - /// Return the for the allocator page at LogRecord CreateLogRecord(long logicalAddress); /// Return the for the allocator page at LogRecord CreateLogRecord(long logicalAddress, long physicalAddress); + /// Return the for a transient (e.g. iterator or pending IO) page at + LogRecord CreateRemappedLogRecordOverTransientMemory(long logicalAddress, long physicalAddress); + + /// Return the for transient log records (e.g. iterator) + ObjectIdMap TranssientObjectIdMap { get; } + /// Dispose an in-memory log record void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs index 28f44c23be9..63e0506c09a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs @@ -4,37 +4,16 @@ namespace Tsavorite.core { /// - /// Interface for hybrid log memory allocator struct wrapper callbacks for inlining performance-path callbacks from - /// - /// to the fully derived allocator, including both record accessors and Scan calls. + /// Interface for hybrid log memory allocator struct wrapper callbacks for inlining performance-path callbacks from within + /// to the fully derived allocator, including both record accessors and Scan calls. /// /// This interface does not currently appear in type constraints, but the organization may prove useful. public interface IAllocatorCallbacks where TStoreFunctions : IStoreFunctions { - /// Get start logical address on - long GetStartLogicalAddress(long page); - - /// Get first valid logical address on - long GetFirstValidLogicalAddress(long page); - - /// Get physical address from - long GetPhysicalAddress(long logicalAddress); - /// Allocate the page in the circular buffer slot at void AllocatePage(int pageIndex); - /// Whether the page at is allocated - bool IsAllocated(int pageIndex); - - /// - /// Populate the page at from the pointer, which has bytes. - /// - unsafe void PopulatePage(byte* src, int required_bytes, long destinationPageIndex); - - /// Free the page at , starting at - void ClearPage(long pageIndex, int offset = 0); - /// Free the page at void FreePage(long pageIndex); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs index 1ba572a935e..8e10c67e4b1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.IO; namespace Tsavorite.core { @@ -11,13 +12,54 @@ namespace Tsavorite.core public interface IHeapObject : IDisposable { /// - /// Total size of the object in memory, including .NET object overheads. + /// The maximum length of a serialized Value object. /// - long MemorySize { get; set; } + public const long MaxSerializedObjectSize = 1L << 40; + + /// + /// Total estimated size of the object in heap memory, including .NET object overheads, for Overflow allocations and Objects. + /// + long HeapMemorySize { get; set; } /// /// Total serialized size of the object; the size it will take when written to disk or other storage. /// - long DiskSize { get; set; } + long SerializedSize { get; set; } + + /// + /// Total serialized size of the object as a byte stream; the size it will take when written to disk or other storage. + /// May be an estimate if + /// + public bool SerializedSizeIsExact { get; } + + /// + /// Create a cloned (shallow copy) of this object + /// + HeapObjectBase Clone(); + + /// + /// Top-level routine to Serialize to the binary writer; checks for cached checkpoint data and calls if needed. + /// + void Serialize(BinaryWriter binaryWriter); + + /// + /// Write the type of the object to the binary writer. + /// + void WriteType(BinaryWriter binaryWriter, bool isNull); + + /// + /// Internal routine to Serialize to the binary writer. + /// + void DoSerialize(BinaryWriter writer); + + /// + /// Copy the ValueObject from srcLogRecord to newLogRecord, cloning and caching serialized data if needed. + /// + void CacheSerializedObjectData(ref LogRecord srcLogRecord, ref LogRecord dstLogRecord); + + /// + /// Clear any serialized data from + /// + void ClearSerializedObjectData(); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs index b3f3b76b6fe..96e6e7cfeca 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs @@ -14,6 +14,15 @@ public unsafe interface ISourceLogRecord /// Fast access returning a copy of the record header RecordInfo Info { get; } + /// Type of the record. Should be set on creation of the and then immutable. + byte RecordType { get; } + + /// Namespace of the record. Should be set on creation of the and then immutable. + byte Namespace { get; } + + /// The for this instance. May be the allocator's or transient (for ). + ObjectIdMap ObjectIdMap { get; } + /// Whether there is actually a record here bool IsSet { get; } @@ -27,39 +36,48 @@ public unsafe interface ISourceLogRecord /// The pointer to the pinned memory if is true, else null. byte* PinnedKeyPointer { get; } + /// Get and set the if this Key is Overflow; an exception is thrown if it is a pinned pointer (e.g. to a . + OverflowByteArray KeyOverflow { get; set; } + /// The value , if this is a String LogRecord; an assertion is raised if it is an Object LogRecord. - /// Not a ref return as it cannot be changed directly; use instead. + /// Not a ref return as it cannot be changed directly; use instead. Span ValueSpan { get; } /// The value object, if the value in this record is an IHeapObject; an exception is thrown if it is a Span, either inline or overflow byte[]. IHeapObject ValueObject { get; } - /// The span of the entire record, if , else an exception is thrown. - ReadOnlySpan RecordSpan { get; } - /// Whether the record's value is pinned in memory, e.g. inline in the log vs an overflow byte[]. If this is true, is non-null. bool IsPinnedValue { get; } /// The pointer to the pinned memory if is true, else null. byte* PinnedValuePointer { get; } + /// Get and set the if this Value is not Overflow; an exception is thrown if it is a pinned pointer (e.g. to a . + OverflowByteArray ValueOverflow { get; set; } + /// The ETag of the record, if any (see ; 0 by default. long ETag { get; } /// The Expiration of the record, if any (see ; 0 by default. long Expiration { get; } - /// If requested by CopyUpdater, the source ValueObject will be cleared immediately (to manage object size tracking most effectively). + /// If requested by CopyUpdater or InPlaceDeleter, the source ValueObject or ValueOverflow will be cleared immediately (to manage object size tracking most effectively). + /// This is called after we have either ensured there is a newer record inserted at tail, or after we have tombstoned the record; either way, we won't be accessing its value. /// The disposer is not inlined, but this is called after object cloning, so the perf hit won't matter - void ClearValueObject(Action disposer); + /// True if we did clear a heap object or overflow, else false + void ClearValueIfHeap(Action disposer); + + /// Whether this is an instance of + bool IsMemoryLogRecord { get; } + + /// Return this as a ref , or throw if not + ref LogRecord AsMemoryLogRecordRef(); - /// A shim to "convert" a TSourceLogRecord generic type that is an instance of to a type. - /// True if this is a , with the output set; else false. - bool AsLogRecord(out LogRecord logRecord); + /// Whether this is an instance of + bool IsDiskLogRecord { get; } - /// A shim to "convert" a TSourceLogRecord generic type this is an instance of to a type. - /// True if this is a , with the output set; else false. - bool AsDiskLogRecord(out DiskLogRecord diskLogRecord); + /// Return this as a ref , or throw if not + ref DiskLogRecord AsDiskLogRecordRef(); /// Get the record's field info, for use in calculating required record size RecordFieldInfo GetRecordFieldInfo(); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs index c8278ea0b4d..3be3eb51d68 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs @@ -2,19 +2,18 @@ // Licensed under the MIT license. using System; +using System.Diagnostics; using System.Runtime.CompilerServices; -#pragma warning disable CS1591 // Missing XML comment for publicly visible type or member - namespace Tsavorite.core { /// /// Static class providing functions to operate on a Log field (Key Span, or Value Span or Object) at a certain address. Since (small) Objects can be represented /// as inline spans, this applies to those forms as well as the inline component of the Object, which is the ObjectId. The layout is: /// - /// Inline: [int Length][data bytes] - /// Overflow: an int ObjectId for a byte[] that is held in - /// Object: an int ObjectId for an IHeapObject that is held in + /// Varbyte indicator byte and lengths; see header comments for details + /// Key data: either the inline data or an int ObjectId for a byte[] that is held in + /// Value data: either the inline data or an int ObjectId for a byte[] that is held in /// /// /// Considerations regarding variable field sizes: @@ -24,106 +23,36 @@ namespace Tsavorite.core /// due to size changes altering whether the Value overflows is handled as part of normal Value-sizechange operations /// /// - public static unsafe class LogField + internal static unsafe class LogField { - /// This is the size of the length prefix on Span field. - public const int InlineLengthPrefixSize = sizeof(int); - - /// For an inline field, get a reference to the length field of the data. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static ref int GetInlineLengthRef(long fieldAddress) => ref *(int*)fieldAddress; - - /// For a field we have already verified is inline, get the address of the actual data (past the length prefix); this is the start of the stream of bytes. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetInlineDataAddress(long fieldAddress) => fieldAddress + InlineLengthPrefixSize; - - /// Gets a referemce to the ObjectId at address (which is ValueAddress). There is no length prefix. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static ref int GetObjectIdRef(long fieldAddress) => ref *(int*)fieldAddress; - - /// For a field we have already verified is inline, get the total inline size of the field: The length prefix plus the length of the byte stream - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int GetTotalSizeOfInlineField(long fieldAddress) => InlineLengthPrefixSize + GetInlineLengthRef(fieldAddress); - - /// The inline length of the key or value without any length prefix. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int GetInlineDataSizeOfField(long valueAddress, bool valueIsInline) => valueIsInline ? GetInlineLengthRef(valueAddress) : ObjectIdMap.ObjectIdSize; - - /// The inline length of the key or value including any length prefix. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int GetInlineTotalSizeOfField(long valueAddress, bool valueIsInline) => valueIsInline ? GetTotalSizeOfInlineField(valueAddress) : ObjectIdMap.ObjectIdSize; - - /// - /// Obtain a referencing the inline or overflow data and the datasize for this field. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span AsSpan(long fieldAddress, bool isInline, ObjectIdMap objectIdMap) - { - if (isInline) - return new((byte*)GetInlineDataAddress(fieldAddress), GetInlineLengthRef(fieldAddress)); - var objectId = GetObjectIdRef(fieldAddress); - if (objectId != ObjectIdMap.InvalidObjectId) - return new Span(Unsafe.As(objectIdMap.Get(objectId))); - return []; - } - - /// - /// Obtain a referencing the inline data and the datasize for this field; MUST be an inline field. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span AsInlineSpan(long fieldAddress) => new((byte*)GetInlineDataAddress(fieldAddress), GetInlineLengthRef(fieldAddress)); - - /// - /// Set all data within a portion of a field to zero. - /// - /// Address of the field - /// Starting position in the field to zero - /// Length of the data from to zero - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static void ZeroInlineData(long address, int dataOffset, int clearLength) - => ZeroData(GetInlineDataAddress(address) + dataOffset, clearLength); - - /// - /// Set all data within a portion of a field to zero. - /// - /// Address to start clearing at - /// Length of the data from to zero - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static void ZeroData(long clearStartAddress, int clearLength) - => new Span((byte*)clearStartAddress, clearLength).Clear(); - /// /// Convert a Span field from inline to overflow. /// /// /// Applies to Value-only during normal ops, and assumes any record size adjustment due to Value growth/shrinkage has already been handled - /// and that the field does not currently contain an overflow allocation. Applies to Keys as well during freelist revivification. + /// and that the field does not currently contain an overflow allocation. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span ConvertInlineToOverflow(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + internal static Span ConvertInlineToOverflow(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, long oldValueLength, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - // First copy the data - var array = GC.AllocateUninitializedArray(newLength); - var oldLength = GetInlineLengthRef(fieldAddress); - var copyLength = oldLength < newLength ? oldLength : newLength; + Debug.Assert(recordInfo.ValueIsInline); + + // First copy the data. We are converting to overflow so the length is limited to int. + var newLength = sizeInfo.FieldInfo.ValueSize; + var overflow = new OverflowByteArray(newLength, startOffset: 0, endOffset: 0, zeroInit: false); + var copyLength = oldValueLength < newLength ? oldValueLength : newLength; if (copyLength > 0) { - var oldSpan = new ReadOnlySpan((byte*)GetInlineDataAddress(fieldAddress), copyLength); - oldSpan.CopyTo(array); + var oldSpan = new ReadOnlySpan((byte*)valueAddress, (int)copyLength); + oldSpan.CopyTo(overflow.Span); } - // If the inline data length was > 0 we are "shrinking" because the overflow objectId replaces the inline field length (its size is the same - // as InlineLengthPrefixSize), so the entire previous data space must be zero-initialized. - // Note: We don't zeroinit data in the overflow allocation, just as we don't zeroinit data in the inline value within the length. - if (oldLength > 0) - ZeroInlineData(fieldAddress, 0, oldLength); - - recordInfo.SetValueIsOverflow(); var objectId = objectIdMap.Allocate(); - GetObjectIdRef(fieldAddress) = objectId; - objectIdMap.Set(objectId, array); - return array; + *(int*)valueAddress = objectId; + objectIdMap.Set(objectId, overflow); + recordInfo.SetValueIsOverflow(); + return overflow.Span; } /// @@ -135,20 +64,17 @@ internal static Span ConvertInlineToOverflow(ref RecordInfo recordInfo, lo /// prepared to convert from Object format to inline format. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span ConvertHeapObjectToOverflow(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + internal static Span ConvertValueObjectToOverflow(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - var array = GC.AllocateUninitializedArray(newLength); + Debug.Assert(recordInfo.ValueIsObject); + var overflow = new OverflowByteArray(sizeInfo.FieldInfo.ValueSize, startOffset: 0, endOffset: 0, zeroInit: false); - var objectId = GetObjectIdRef(fieldAddress); + var objectId = *(int*)valueAddress; if (objectId == ObjectIdMap.InvalidObjectId) - { - objectId = objectIdMap.Allocate(); - GetObjectIdRef(fieldAddress) = objectId; - } - objectIdMap.Set(objectId, array); - + *(int*)valueAddress = objectId = objectIdMap.Allocate(); + objectIdMap.Set(objectId, overflow); recordInfo.SetValueIsOverflow(); - return array; + return overflow.Span; } /// @@ -160,18 +86,13 @@ internal static Span ConvertHeapObjectToOverflow(ref RecordInfo recordInfo /// created an object that has converted from inline format to object format. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int ConvertInlineToHeapObject(ref RecordInfo recordInfo, long fieldAddress, ObjectIdMap objectIdMap) + internal static int ConvertInlineToValueObject(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - // Here we do not copy the data; we assume the caller will have already created an object that has converted from inline format to object format. + Debug.Assert(recordInfo.ValueIsInline); var objectId = objectIdMap.Allocate(); - var oldLength = GetInlineLengthRef(fieldAddress); - - // We must zeroinit the to-be-unused space. - if (oldLength > 0) - ZeroInlineData(fieldAddress, 0, oldLength); - recordInfo.SetValueIsObject(); - GetObjectIdRef(fieldAddress) = objectId; + + *(int*)valueAddress = objectId; return objectId; } @@ -184,85 +105,70 @@ internal static int ConvertInlineToHeapObject(ref RecordInfo recordInfo, long fi /// created an object that has converted from inline format to object format. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int ConvertOverflowToHeapObject(ref RecordInfo recordInfo, long fieldAddress, ObjectIdMap objectIdMap) + internal static int ConvertOverflowToValueObject(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - var objectId = GetObjectIdRef(fieldAddress); + Debug.Assert(recordInfo.ValueIsOverflow); + + var objectId = *(int*)valueAddress; if (objectId != ObjectIdMap.InvalidObjectId) - { - // Clear the byte[] from the existing slot - objectIdMap.Set(objectId, null); - } + objectIdMap.Set(objectId, null); // Clear the byte[] from the existing slot but do not free the slot; caller will put the HeapObject into the slot. else - { - objectId = objectIdMap.Allocate(); - GetObjectIdRef(fieldAddress) = objectId; - } + *(int*)valueAddress = objectId = objectIdMap.Allocate(); recordInfo.SetValueIsObject(); return objectId; } - /// - /// Utility function to set the overflow allocation at the given Span field's address. Assumes caller has ensured no existing overflow - /// allocation is there; e.g. SerializeKey and InitializeValue. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span SetOverflowAllocation(long fieldAddress, int newLength, ObjectIdMap objectIdMap) - { - // Assumes no object allocated for this field yet. - var objectId = objectIdMap.Allocate(); - GetObjectIdRef(fieldAddress) = objectId; - - var newArray = GC.AllocateUninitializedArray(newLength); - objectIdMap.Set(objectId, newArray); - return new Span(newArray); - } - /// /// Convert a Span field from overflow to inline. /// /// /// Applies to Value-only during normal ops, and assumes any record size adjustment due to Value growth/shrinkage has already been handled - /// and that the field currently contains an overflow allocation. Applies to Keys as well during freelist revivification. + /// and that the field currently contains an overflow allocation. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span ConvertOverflowToInline(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + internal static Span ConvertOverflowToInline(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { + Debug.Assert(recordInfo.ValueIsOverflow); + // First copy the data - var objectId = GetObjectIdRef(fieldAddress); + var objectId = *(int*)valueAddress; + + var newLength = sizeInfo.FieldInfo.ValueSize; + var newSpan = new Span((byte*)valueAddress, newLength); + if (objectId != ObjectIdMap.InvalidObjectId) { - var oldSpan = new Span((byte[])objectIdMap.Get(objectId)); + var overflow = objectIdMap.GetOverflowByteArray(objectId); + var oldSpan = overflow.Span; - // Sequencing here is important for zeroinit correctness var copyLength = oldSpan.Length < newLength ? oldSpan.Length : newLength; - var newSpan = SetInlineDataLength(fieldAddress, newLength); recordInfo.SetValueIsInline(); oldSpan.Slice(0, copyLength).CopyTo(newSpan); - objectIdMap.Set(objectId, null); - return newSpan; + objectIdMap.Free(objectId); } - return SetInlineDataLength(fieldAddress, newLength); + return newSpan; } /// /// Called when disposing a record, to free an Object or Overflow allocation and convert to inline so the lengths are set for record scanning or revivification. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static void FreeObjectIdAndConvertToInline(ref RecordInfo recordInfo, long fieldAddress, ObjectIdMap objectIdMap, bool isKey) + internal static void ClearObjectIdAndConvertToInline(ref RecordInfo recordInfo, long fieldAddress, ObjectIdMap objectIdMap, bool isKey, Action objectDisposer = null) { - // ObjectIdSize and InlineLengthPrefixSize are the same so we can just set the length to zero; there was no data associated with the objectId. This also - // means we don't have to adjust the filler length, since the field size here isn't changing. This method is called by record disposal, which also clears - // the optionals, which may adjust filler length). Consistency Note: LogRecord.InitializeForReuse also sets field lengths to zero and sets the filler length. - ref int objectIdRef = ref GetObjectIdRef(fieldAddress); - var objectId = objectIdRef; - objectIdRef = 0; + Debug.Assert(isKey ? !recordInfo.KeyIsInline : !recordInfo.ValueIsInline); + // We don't have to adjust the filler length, since the field size here isn't changing; we'll just have int-sized "data". This method is called by record disposal, which + // also clears the optionals, which may adjust filler length. Consistency Note: LogRecord.InitializeForReuse also sets field lengths to zero and sets the filler length. + // However, here we may be called after setting the IgnoreOptionals word, so we don't want to decode the indicator. + var objectId = *(int*)fieldAddress; if (objectId != ObjectIdMap.InvalidObjectId) - objectIdMap.Set(objectId, null); + { + objectIdMap.Free(objectId, objectDisposer); + *(int*)fieldAddress = ObjectIdMap.InvalidObjectId; + } - // Sequencing here is important for zeroinit correctness - GetInlineLengthRef(fieldAddress) = 0; + // We don't need to change the length; we'll keep the current length and just convert to inline. if (isKey) recordInfo.SetKeyIsInline(); else @@ -278,43 +184,16 @@ internal static void FreeObjectIdAndConvertToInline(ref RecordInfo recordInfo, l /// the caller will have already prepared to convert from Object format to inline format. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span ConvertHeapObjectToInline(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + internal static Span ConvertValueObjectToInline(ref RecordInfo recordInfo, long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - ref var objIdRef = ref GetObjectIdRef(fieldAddress); - objectIdMap.Free(objIdRef); - objIdRef = 0; + var objIdPtr = (int*)valueAddress; + var objectId = *objIdPtr; + if (objectId != ObjectIdMap.InvalidObjectId) + objectIdMap.Free(objectId); + *objIdPtr = 0; - // Sequencing here is important for zeroinit correctness - var newSpan = SetInlineDataLength(fieldAddress, newLength); recordInfo.SetValueIsInline(); - return newSpan; - } - - /// - /// Utility function to set the inline length of a Span field and return a to the data start (which may be an inline byte stream or a byte[]). - /// - internal static Span SetInlineDataLength(long fieldAddress, int newLength) - { - GetInlineLengthRef(fieldAddress) = newLength; // actual length (i.e. the inline data space used by this field) - return new Span((byte*)GetInlineDataAddress(fieldAddress), newLength); - } - - /// - /// Shrink an inline Span field in place. - /// - /// - /// Applies to Value-only during normal ops, and assumes any record size adjustment due to Value growth/shrinkage has already been handled - /// and that the field currently contains an overflow allocation. Applies to Keys as well during freelist revivification. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static byte* AdjustInlineLength(long fieldAddress, int newLength) - { - // Zeroinit the extra space. Here we are concerned about shrinkage leaving nonzero leftovers, so we clear those. - var clearLength = GetInlineLengthRef(fieldAddress) - newLength; - if (clearLength > 0) - ZeroInlineData(fieldAddress, newLength, clearLength); - GetInlineLengthRef(fieldAddress) = newLength; - return (byte*)GetInlineDataAddress(fieldAddress); + return new((byte*)valueAddress, sizeInfo.FieldInfo.ValueSize); } /// @@ -322,37 +201,40 @@ internal static Span SetInlineDataLength(long fieldAddress, int newLength) /// shrinkage is sufficient (given available space in the record) to convert the field in-place to inline. /// /// - /// Applies to Value-only during normal ops, and assumes any record size adjustment due to Value growth/shrinkage has already been handled - /// and that the field currently contains an overflow allocation. Applies to Keys as well during freelist revivification. + /// Applies to Value only, and assumes any record size adjustment due to Value growth/shrinkage has already been handled + /// and that the field currently contains an overflow allocation. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static Span ReallocateOverflow(long fieldAddress, int newLength, ObjectIdMap objectIdMap) + internal static Span ReallocateValueOverflow(long physicalAddress, long valueAddress, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - byte[] newArray; + OverflowByteArray newOverflow; + var newLength = sizeInfo.FieldInfo.ValueSize; - var objectId = GetObjectIdRef(fieldAddress); + var objectId = *(int*)valueAddress; if (objectId != ObjectIdMap.InvalidObjectId) { - var oldArray = (byte[])objectIdMap.Get(objectId); - if (oldArray.Length == newLength) - return new Span(oldArray); + var oldOverflow = objectIdMap.GetOverflowByteArray(objectId); + var oldSpan = oldOverflow.Span; + if (oldSpan.Length == newLength) + return oldSpan; + + // AllocateUninitialized and copy, and zeroinit any remainder + newOverflow = new(newLength, startOffset: 0, endOffset: 0, zeroInit: false); + var copyLength = oldSpan.Length < newLength ? oldSpan.Length : newLength; - // Allocate and copy - newArray = new byte[newLength]; - var copyLength = oldArray.Length < newLength ? oldArray.Length : newLength; - Array.Copy(oldArray, newArray, copyLength); + oldOverflow.Span.Slice(0, copyLength).CopyTo(newOverflow.Span); if (copyLength < newLength) - Array.Clear(newArray, copyLength, newLength - copyLength); + newOverflow.Span.Slice(copyLength, newLength - copyLength).Clear(); } else { - // Allocate; nothing to copy - newArray = new byte[newLength]; + // Allocate; nothing to copy, so allocate with zero initialization + newOverflow = new(newLength, startOffset: 0, endOffset: 0, zeroInit: false); objectId = objectIdMap.Allocate(); - GetObjectIdRef(fieldAddress) = objectId; + *(int*)valueAddress = objectId; } - objectIdMap.Set(objectId, newArray); - return new Span(newArray); + objectIdMap.Set(objectId, newOverflow); + return newOverflow.Span; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/LogRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/LogRecord.cs index a0b88a30942..9b60cd463ca 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/LogRecord.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/LogRecord.cs @@ -4,28 +4,28 @@ using System; using System.Diagnostics; using System.Runtime.CompilerServices; -using static Tsavorite.core.Utility; #pragma warning disable CS1591 // Missing XML comment for publicly visible type or member namespace Tsavorite.core { - /// The in-memory record on the log: header, key, value, and optional fields - /// until some other things have been done that will allow clean separation. - /// - /// The space is laid out as: - /// - /// [RecordInfo][Key (Span or ObjectId)][Value (Span or ObjectId)][ETag?][Expiration?][FillerLen?][Filler bytes] - ///
Where Value Span is one of:
- /// - /// ObjectId: If this is a object record, this is the ID into the - /// Span data: See for details - /// - ///
- ///
- /// This lets us get to the key without intermediate computations to account for the optional fields. +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + using static VarbyteLengthUtility; + + /// The in-memory record on the log. The space is laid out as: + /// + /// header + /// Varbyte indicator bytes (including RecordType and Namespace) and lengths; see header comments for details + /// Key data: either the inline data or an int ObjectId for a byte[] that is held in + /// Value data: either the inline data or an int ObjectId for a byte[] that is held in + /// Optional data (may or may not be present): ETag, Expiration + /// Pseudo-optional ObjectLogPosition indicating the position in the object log file, if the record is not fully inline. + /// Optional filler length: Extra space in the record, due to record-alignment round-up or Value shrinkage + /// + /// This lets us get to the key without intermediate computations having to account for the optional fields. /// Some methods have both member and static versions for ease of access and possibly performance gains. - /// + /// public unsafe partial struct LogRecord : ISourceLogRecord { /// The physicalAddress in the log. @@ -40,9 +40,8 @@ public unsafe partial struct LogRecord : ISourceLogRecord public const int NoETag = 0; /// Number of bytes required to store an Expiration public const int ExpirationSize = sizeof(long); - /// Miminal length metadata size: one byte each for indicator byte, key length, value length - public const int MinLengthMetadataSize = 3; - + /// Number of bytes required to the object log position + public const int ObjectLogPositionSize = sizeof(long); /// Number of bytes required to store the FillerLen internal const int FillerLengthSize = sizeof(int); @@ -51,6 +50,12 @@ public unsafe partial struct LogRecord : ISourceLogRecord [MethodImpl(MethodImplOptions.AggressiveInlining)] internal LogRecord(long physicalAddress) => this.physicalAddress = physicalAddress; + internal readonly long IndicatorAddress => physicalAddress + RecordInfo.Size; + private readonly long RecordTypeAddress => physicalAddress + RecordInfo.Size + 1; + private readonly long NamespaceAddress => physicalAddress + RecordInfo.Size + 2; + + public readonly byte IndicatorByte => *(byte*)IndicatorAddress; + /// This ctor is primarily used for internal record-creation operations for the ObjectAllocator, and is passed to IObjectSessionFunctions callbacks. [MethodImpl(MethodImplOptions.AggressiveInlining)] internal LogRecord(long physicalAddress, ObjectIdMap objectIdMap) @@ -59,35 +64,110 @@ internal LogRecord(long physicalAddress, ObjectIdMap objectIdMap) this.objectIdMap = objectIdMap; } + /// This ctor is used construct a transient copy of an in-memory LogRecord that remaps the object Ids in to the transient map. + /// is must be a pointer to transient memory that contains a copy of the in-memory allocator page's record span, including the objectIds + /// in Key and Value data. This is used for iteration. Note that the objects are not removed from the allocator-page map, so for iteration they may temporarily be in both. + /// + /// This is ONLY to be done for transient log records, not records on the main log. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static LogRecord CreateRemappedOverTransientMemory(long physicalAddress, ObjectIdMap allocatorMap, ObjectIdMap transientMap) + { + var logRecord = new LogRecord(physicalAddress, transientMap); + if (logRecord.Info.KeyIsOverflow) + { + var (length, dataAddress) = GetKeyFieldInfo(logRecord.IndicatorAddress); + var overflow = allocatorMap.GetOverflowByteArray(*(int*)dataAddress); + *(int*)dataAddress = transientMap.AllocateAndSet(overflow); + } + + if (logRecord.Info.ValueIsOverflow) + { + var (length, dataAddress) = GetValueFieldInfo(logRecord.IndicatorAddress); + var overflow = allocatorMap.GetOverflowByteArray(*(int*)dataAddress); + *(int*)dataAddress = transientMap.AllocateAndSet(overflow); + } + else if (logRecord.Info.ValueIsObject) + { + var (length, dataAddress) = GetValueFieldInfo(logRecord.IndicatorAddress); + var heapObj = allocatorMap.GetHeapObject(*(int*)dataAddress); + *(int*)dataAddress = transientMap.AllocateAndSet(heapObj); + } + return logRecord; + } + #region ISourceLogRecord /// - public readonly bool IsSet => physicalAddress != 0; - /// - public readonly ref RecordInfo InfoRef + public byte RecordType { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get { return ref GetInfoRef(physicalAddress); } + get => *(byte*)RecordTypeAddress; + set => *(byte*)RecordTypeAddress = value; } /// - public readonly RecordInfo Info + public byte Namespace { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get { return GetInfo(physicalAddress); } + get => *(byte*)NamespaceAddress; + set => *(byte*)NamespaceAddress = value; } + /// + public readonly ObjectIdMap ObjectIdMap => objectIdMap; + + /// + public readonly bool IsSet => physicalAddress != 0; + + /// + public readonly ref RecordInfo InfoRef => ref *(RecordInfo*)physicalAddress; + + /// + public readonly RecordInfo Info => *(RecordInfo*)physicalAddress; + /// public readonly ReadOnlySpan Key { [MethodImpl(MethodImplOptions.AggressiveInlining)] - get => GetKey(physicalAddress, objectIdMap); + get + { + var (length, dataAddress) = GetKeyFieldInfo(IndicatorAddress); + return Info.KeyIsInline ? new((byte*)dataAddress, length) : objectIdMap.GetOverflowByteArray(*(int*)dataAddress).ReadOnlySpan; + } } /// public readonly bool IsPinnedKey => Info.KeyIsInline; /// - public byte* PinnedKeyPointer => IsPinnedKey ? (byte*)LogField.GetInlineDataAddress(KeyAddress) : null; + public readonly byte* PinnedKeyPointer + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + if (!IsPinnedKey) + throw new TsavoriteException("PinnedKeyPointer is unavailable when Key is not pinned; use IsPinnedKey"); + (_ /*length*/, var dataAddress) = GetKeyFieldInfo(IndicatorAddress); + return (byte*)dataAddress; + } + } + + /// Get and set the if this Key is not pinned; an exception is thrown if it is a pinned pointer (e.g. to a . + public readonly OverflowByteArray KeyOverflow + { + get + { + if (Info.KeyIsInline) + throw new TsavoriteException("get_Overflow is unavailable when Key is inline"); + var (length, dataAddress) = GetKeyFieldInfo(IndicatorAddress); + return objectIdMap.GetOverflowByteArray(*(int*)dataAddress); + } + set + { + + var (length, dataAddress) = GetKeyFieldInfo(IndicatorAddress); + if (!Info.KeyIsOverflow || length != ObjectIdMap.ObjectIdSize) + throw new TsavoriteException("set_KeyOverflow should only be called when transferring into a new record with KeyIsInline==false and key.Length==ObjectIdSize"); + *(int*)dataAddress = objectIdMap.AllocateAndSet(value); + } + } /// public readonly Span ValueSpan @@ -96,7 +176,8 @@ public readonly Span ValueSpan get { Debug.Assert(!Info.ValueIsObject, "ValueSpan is not valid for Object values"); - return LogField.AsSpan(ValueAddress, Info.ValueIsInline, objectIdMap); + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + return Info.ValueIsInline ? new((byte*)dataAddress, (int)length) : objectIdMap.GetOverflowByteArray(*(int*)dataAddress).Span; } } @@ -109,83 +190,165 @@ public readonly IHeapObject ValueObject Debug.Assert(Info.ValueIsObject, "ValueObject is not valid for Span values"); if (!Info.ValueIsObject) return default; - var objectId = *ValueObjectIdAddress; - if (objectId == ObjectIdMap.InvalidObjectId) - return default; - var heapObj = objectIdMap.Get(objectId); - return Unsafe.As(ref heapObj); + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + return objectIdMap.GetHeapObject(*(int*)dataAddress); + } + internal set + { + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + + // We cannot verify that value.Length==ObjectIdSize because we have reused the varbyte length as the high byte of the 5-byte length. + if (!Info.ValueIsObject) + throw new TsavoriteException("SetValueObject should only be called by DiskLogRecord or Deserialization with ValueIsObject==true"); + *(int*)dataAddress = objectIdMap.AllocateAndSet(value); + + // We reused the varbyte length as the high byte of the 5-byte length, so reset it now to ObjectIdSize. + UpdateVarbyteValueLengthByteInWord(IndicatorAddress, ObjectIdMap.ObjectIdSize); } } + /// The span of the entire record, including the ObjectId space if the record has objects. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly ReadOnlySpan AsReadOnlySpan() => new((byte*)physicalAddress, GetInlineRecordSizes().actualSize); + /// - public readonly ReadOnlySpan RecordSpan + public readonly bool IsPinnedValue => Info.ValueIsInline; + + /// + public readonly byte* PinnedValuePointer { [MethodImpl(MethodImplOptions.AggressiveInlining)] get { - if (!Info.RecordIsInline) - throw new TsavoriteException("RecordSpan is not valid for non-inline records"); - return new((byte*)physicalAddress, GetInlineRecordSizes().actualSize); + if (!IsPinnedValue) + throw new TsavoriteException("PinnedValuePointer is unavailable when Key is not pinned; use IsPinnedKey"); + (_ /*length*/, var dataAddress) = GetValueFieldInfo(IndicatorAddress); + return (byte*)dataAddress; } } - /// - public bool IsPinnedValue => Info.ValueIsInline; + /// Get and set the if this Value is not pinned; an exception is thrown if it is a pinned pointer (e.g. to a . + public readonly OverflowByteArray ValueOverflow + { + get + { + if (!Info.ValueIsOverflow) + throw new TsavoriteException("get_Overflow is unavailable when Value is not overflow"); + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + return objectIdMap.GetOverflowByteArray(*(int*)dataAddress); + } + set + { + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + if (!Info.ValueIsOverflow || length != ObjectIdMap.ObjectIdSize) + throw new TsavoriteException("SetValueObject should only be called when trnasferring into a new record with ValueIsOverflow == true and value.Length==ObjectIdSize"); + *(int*)dataAddress = objectIdMap.AllocateAndSet(value); + } + } - /// - public byte* PinnedValuePointer => IsPinnedValue ? (byte*)LogField.GetInlineDataAddress(ValueAddress) : null; + public static int GetOptionalLength(RecordInfo info) => (info.HasETag ? ETagSize : 0) + (info.HasExpiration ? ExpirationSize : 0) + (info.RecordHasObjects ? sizeof(long) : 0); /// public readonly long ETag => Info.HasETag ? *(long*)GetETagAddress() : NoETag; /// public readonly long Expiration => Info.HasExpiration ? *(long*)GetExpirationAddress() : 0; + /// + public readonly bool IsMemoryLogRecord => true; + + /// + public readonly unsafe ref LogRecord AsMemoryLogRecordRef() => ref Unsafe.AsRef(in this); + + /// + public readonly bool IsDiskLogRecord => false; + + /// + public readonly unsafe ref DiskLogRecord AsDiskLogRecordRef() => throw new InvalidOperationException("Cannot cast a memory LogRecord to a DiskLogRecord."); + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': not doing so because it modifies internal state - public void ClearValueObject(Action disposer) -#pragma warning restore IDE0251 + public readonly RecordFieldInfo GetRecordFieldInfo() { - Debug.Assert(Info.ValueIsObject, "ClearValueObject() is not valid for Span Values"); - if (Info.ValueIsObject) + var (keyLength, valueLength, _ /*offsetToKeyStart*/) = GetInlineKeyAndValueSizes(IndicatorAddress); + return new() { - objectIdMap.ClearAt(ValueObjectId, disposer); - if (!Info.ValueIsInline) - *ValueObjectIdAddress = ObjectIdMap.InvalidObjectId; - } + KeySize = keyLength, + ValueSize = valueLength, + ValueIsObject = Info.ValueIsObject, + HasETag = Info.HasETag, + HasExpiration = Info.HasExpiration + }; } + #endregion // ISourceLogRecord - /// + /// + /// Initialize record for --includes Overflow option for Key and Overflow and Object option for Value + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly bool AsLogRecord(out LogRecord logRecord) + public readonly void InitializeRecord(ReadOnlySpan key, in RecordSizeInfo sizeInfo, ObjectIdMap objectIdMap) { - logRecord = this; - return true; - } + // Set varbyte lengths + *(long*)IndicatorAddress = sizeInfo.IndicatorWord; - /// - public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) - { - diskLogRecord = default; - return false; + var keyAddress = sizeInfo.GetKeyAddress(physicalAddress); + var keySize = sizeInfo.FieldInfo.KeySize; + var valueAddress = keyAddress + sizeInfo.InlineKeySize; + + // Serialize Key + if (sizeInfo.KeyIsInline) + { + InfoRef.SetKeyIsInline(); + key.CopyTo(new Span((byte*)keyAddress, keySize)); + } + else + { + InfoRef.SetKeyIsOverflow(); + var overflow = new OverflowByteArray(key.Length, startOffset: 0, endOffset: 0, zeroInit: false); + key.CopyTo(overflow.Span); + + // This is record initialization so no object has been allocated for this field yet. + var objectId = objectIdMap.Allocate(); + *(int*)keyAddress = objectId; + objectIdMap.Set(objectId, overflow); + } + + // Initialize Value metadata + if (sizeInfo.ValueIsInline) + InfoRef.SetValueIsInline(); + else + { + // Unlike for Keys, we do not set the objectId here; we wait for the UMD operation to do that. + *(int*)valueAddress = ObjectIdMap.InvalidObjectId; + if (sizeInfo.ValueIsObject) + { + Debug.Assert(sizeInfo.FieldInfo.ValueSize == ObjectIdMap.ObjectIdSize, $"Expected object size ({ObjectIdMap.ObjectIdSize}) for Object ValueSize but was {sizeInfo.FieldInfo.ValueSize}"); + InfoRef.SetValueIsObject(); + } + else + InfoRef.SetValueIsOverflow(); + } + + // The rest is considered filler + InitializeFillerLength(in sizeInfo); } - /// + /// + /// Initialize record for --does not include Overflow/Object options so is streamlined + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly RecordFieldInfo GetRecordFieldInfo() => new() + public readonly void InitializeRecord(ReadOnlySpan key, in RecordSizeInfo sizeInfo) { - KeyDataSize = LogField.GetInlineDataSizeOfField(KeyAddress, Info.KeyIsInline), - ValueDataSize = LogField.GetInlineDataSizeOfField(ValueAddress, Info.ValueIsInline), - ValueIsObject = Info.ValueIsObject, - HasETag = Info.HasETag, - HasExpiration = Info.HasExpiration - }; - #endregion // ISourceLogRecord + // Set varbyte lengths + *(long*)IndicatorAddress = sizeInfo.IndicatorWord; - /// - /// The record is directly copyable if it is entirely inline; otherwise it must be Serialized to linear format - /// - public readonly bool IsDirectlyCopyable => Info.RecordIsInline; + InfoRef.SetKeyAndValueInline(); + + // Serialize Key + key.CopyTo(new Span((byte*)sizeInfo.GetKeyAddress(physicalAddress), sizeInfo.FieldInfo.KeySize)); + + // The rest is considered filler + InitializeFillerLength(in sizeInfo); + } /// A ref to the record header [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -195,131 +358,118 @@ public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) [MethodImpl(MethodImplOptions.AggressiveInlining)] public static RecordInfo GetInfo(long physicalAddress) => *(RecordInfo*)physicalAddress; - /// The address of the key - public readonly long KeyAddress => GetKeyAddress(physicalAddress); - /// The address of the key - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetKeyAddress(long physicalAddress) => physicalAddress + RecordInfo.GetLength(); - - /// A representing the record Key - /// Not a ref return as it cannot be changed - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ReadOnlySpan GetKey(long physicalAddress, ObjectIdMap objectIdMap) => LogField.AsSpan(GetKeyAddress(physicalAddress), GetInfo(physicalAddress).KeyIsInline, objectIdMap); - - /// A representing the record Key *if* the key is inline; this must be tested before the call - /// Not a ref return as it cannot be changed - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ReadOnlySpan GetInlineKey(long physicalAddress) => LogField.AsInlineSpan(GetKeyAddress(physicalAddress)); - - /// The address of the value - public readonly long ValueAddress => GetValueAddress(physicalAddress); - /// The address of the value. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetValueAddress(long physicalAddress) + internal static ReadOnlySpan GetInlineKey(long physicalAddress) { - var keyAddress = GetKeyAddress(physicalAddress); - return keyAddress + LogField.GetInlineTotalSizeOfField(keyAddress, GetInfo(physicalAddress).KeyIsInline); + Debug.Assert((*(RecordInfo*)physicalAddress).KeyIsInline, "Key must be inline"); + var (length, dataAddress) = GetKeyFieldInfo(physicalAddress + RecordInfo.Size); + return new((byte*)dataAddress, length); } - internal readonly int* ValueObjectIdAddress => (int*)ValueAddress; - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int* GetValueObjectIdAddress(long physicalAddress) => (int*)GetValueAddress(physicalAddress); - - /// The value object id (index into the object values array) - internal readonly int ValueObjectId + /// The actual size of the main-log (inline) portion of the record; for in-memory records it does not include filler length. + public readonly int ActualRecordSize { + [MethodImpl(MethodImplOptions.AggressiveInlining)] get { - Debug.Assert(Info.ValueIsObject, "Cannot get ValueObjectId for String LogRecord"); - Debug.Assert(!Info.ValueIsInline, "Cannot get ValueObjectId for inline values"); - return Info.ValueIsInline ? ObjectIdMap.InvalidObjectId : *ValueObjectIdAddress; + var (length, dataAddress) = GetValueFieldInfo(IndicatorAddress); + return (int)(dataAddress - physicalAddress + length + OptionalLength); } } - /// The actual size of the main-log (inline) portion of the record; for in-memory records it does not include filler length. - public readonly int ActualRecordSize + public readonly Span RecordSpan { [MethodImpl(MethodImplOptions.AggressiveInlining)] get { - var valueAddress = ValueAddress; - var valueSize = LogField.GetInlineTotalSizeOfField(valueAddress, GetInfo(physicalAddress).ValueIsInline); - return (int)(valueAddress - physicalAddress + valueSize + OptionalLength); + return Info.RecordIsInline + ? new((byte*)physicalAddress, GetInlineRecordSizes().actualSize) + : throw new TsavoriteException("RecordSpan is not valid for non-inline records"); } } /// /// Asserts that is the same size as the value data size in the before setting the length. /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TrySetValueLength(int newValueSize, in RecordSizeInfo sizeInfo) + public readonly bool TrySetValueLength(int newValueSize, in RecordSizeInfo sizeInfo, bool zeroInit = false) { - Debug.Assert(newValueSize == sizeInfo.FieldInfo.ValueDataSize, $"Mismatched value size; expected {sizeInfo.FieldInfo.ValueDataSize}, actual {newValueSize}"); - return TrySetValueLength(in sizeInfo); + Debug.Assert(newValueSize == sizeInfo.FieldInfo.ValueSize, $"Mismatched value size; expected {sizeInfo.FieldInfo.ValueSize}, actual {newValueSize}"); + return TrySetValueLength(in sizeInfo, zeroInit); } /// - /// Tries to set the length of the value field, with consideration to whether there is also space for the optionals (ETag and Expiration). + /// Tries to set the length of the value field, with consideration to whether there is also space for the optionals (ETag, Expiration, ObjectLogPosition). /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TrySetValueLength(in RecordSizeInfo sizeInfo) + public readonly bool TrySetValueLength(in RecordSizeInfo sizeInfo, bool zeroInit = false) => TrySetValueLength(in sizeInfo, zeroInit, out _ /*valueAddress*/); + + private readonly bool TrySetValueLength(in RecordSizeInfo sizeInfo, bool zeroInit, out long valueAddress) { - var valueAddress = ValueAddress; - var oldTotalInlineValueSize = LogField.GetInlineTotalSizeOfField(valueAddress, Info.ValueIsInline); - var newTotalInlineValueSize = sizeInfo.InlineTotalValueSize; + // Get the number of bytes in existing key and value lengths. + var (keyLengthBytes, valueLengthBytes, _ /*hasFiller*/) = DeconstructIndicatorByte(*(byte*)IndicatorAddress); + var oldInlineValueSize = ReadVarbyteLength(valueLengthBytes, (byte*)(IndicatorAddress + NumIndicatorBytes + keyLengthBytes)); + var newInlineValueSize = sizeInfo.InlineValueSize; + + // Calculate this with our own valueLengthBytes, as the new value may result in a different byte count, which would throw things off. + // Key does not change, so its size and size byte count remain the same. + valueAddress = physicalAddress + RecordInfo.Size + NumIndicatorBytes + keyLengthBytes + valueLengthBytes + sizeInfo.InlineKeySize; + + // We don't need to change the size if value size hasn't changed (ignore optionalSize changes; we're not changing the data for that, only shifting them). + // For this quick check, just check for inline differences; we'll examine overflow size changes and conversions later. + if (Info.RecordIsInline && sizeInfo.KeyIsInline && sizeInfo.ValueIsInline && oldInlineValueSize == newInlineValueSize) + return true; + + // It is OK if the value is shrinking so we don't need all the old valueLengthBytes, but we cannot grow past the old valueLengthBytes. + // (If we are converting from inline to overflow that will already be accounted for because sizeInfo will be set for the ObjectId length.) + if (sizeInfo.ValueLengthBytes > valueLengthBytes) + return false; // Growth and fillerLen may be negative if shrinking. - var inlineValueGrowth = newTotalInlineValueSize - oldTotalInlineValueSize; - var newOptionalSize = sizeInfo.OptionalSize; + var inlineValueGrowth = (int)(newInlineValueSize - oldInlineValueSize); var oldOptionalSize = OptionalLength; - var inlineTotalGrowth = inlineValueGrowth + (newOptionalSize - oldOptionalSize); + var newOptionalSize = sizeInfo.OptionalSize; - var optionalStartAddress = valueAddress + oldTotalInlineValueSize; + var optionalStartAddress = valueAddress + oldInlineValueSize; var fillerLenAddress = optionalStartAddress + oldOptionalSize; var fillerLen = GetFillerLength(fillerLenAddress); - // See if we have enough room for the inline data. Note: We don't consider here things like moving inline data that is less than - // overflow length into overflow to free up inline space; we calculate the inline size required for the new value (including whether - // it is overflow) and optionals, and success is based on whether that can fit into the allocated record space. - if (fillerLen < inlineTotalGrowth) + // See if we have enough room for the change in Value inline data. Note: This includes things like moving inline data that is less than + // overflow length into overflow, which frees up inline space > ObjectIdMap.ObjectIsSize. We calculate the inline size required for the + // new value (including whether it is overflow) and the existing optionals, and success is based on whether that can fit into the allocated + // record space. We do not change the presence of optionals h ere; we just ensure there is enough for the larger of (current optionals, + // new optionals) and a later operation will actually read/update the optional(s), including setting/clearing the flag(s). + if (fillerLen < inlineValueGrowth + (newOptionalSize - oldOptionalSize)) return false; - // We know we have enough space for the changed value *and* for whatever changes are coming in for the optionals. However, we aren't - // changing the data of the optionals here, so don't adjust fillerLen for them; only adjust it for value length change. - fillerLen -= inlineValueGrowth; - - var newInlineDataLength = sizeInfo.FieldInfo.ValueDataSize; - - // See if we need to shift/zero the optionals and Filler. - var shiftOptionals = inlineValueGrowth != 0; + // Update record part 1: Set varbyte value length to the full length of the value, including filler but NOT optionalSize (which is calculated + // directly from RecordInfo's HasETag and HasExpiration bits, which we do not change here). Scan will now be able to navigate to the end of + // the record via GetInlineRecordSizes().allocatedSize. + var oldValueAndFillerSize = (int)(oldInlineValueSize + fillerLen); + *(long*)IndicatorAddress = CreateIgnoreOptionalsVarbyteWord(*(long*)IndicatorAddress, keyLengthBytes, valueLengthBytes, oldValueAndFillerSize); - // We have enough space to handle the changed value size, including changing between inline and out-of-line or vice-versa, and the new - // optional space. But we do not count the change in optional space here; we just ensure there is enough, and a later operation will - // actually add/remove/update the optional(s), including setting the flag. So only adjust offsets for valueGrowth, not totalGrowth. + // Update record part 2: Save the optionals if shifting is needed. We can't just shift now because we may be e.g. converting from inline to + // overflow and they'd overwrite needed data. + var shiftOptionals = inlineValueGrowth != 0 && oldOptionalSize > 0; + var optionalFields = new OptionalFieldsShift(); + if (shiftOptionals) + optionalFields.Save(optionalStartAddress, Info); + // Update record part 3: Set value length, which includes converting between Inline, Overflow, and Object. This may allocate or free Heap Objects. // Evaluate in order of most common (i.e. most perf-critical) cases first. if (Info.ValueIsInline && sizeInfo.ValueIsInline) { - // Both are inline, so resize in place (and adjust filler) if needed. - if (inlineValueGrowth != 0) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.AdjustInlineLength(ValueAddress, newInlineDataLength); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - goto Done; + // Both are inline, so nothing to do here; we will set the new size into the varbyte indicator word below. } else if (Info.ValueIsOverflow && sizeInfo.ValueIsOverflow) { - // Both are out-of-line, so reallocate in place if needed. Object records will do what they need to after this call returns; - // we're only here to set up inline lengths, and that hasn't changed here. - _ = LogField.ReallocateOverflow(valueAddress, newInlineDataLength, objectIdMap); - goto Done; + // Both are out-of-line, so reallocate in place if needed; the caller will operate on that space after we return. + _ = LogField.ReallocateValueOverflow(physicalAddress, valueAddress, in sizeInfo, objectIdMap); } else if (Info.ValueIsObject && sizeInfo.ValueIsObject) { - // Both are object records, so nothing to change. - goto Done; + // Both are object records, so nothing to change; the caller will operate on the object after we return. } else { @@ -329,60 +479,24 @@ public bool TrySetValueLength(in RecordSizeInfo sizeInfo) { if (sizeInfo.ValueIsOverflow) { - // Convert from inline to overflow. This replaces the InlineLengthPrefix with the ObjectId. - Debug.Assert(inlineValueGrowth == ObjectIdMap.ObjectIdSize - oldTotalInlineValueSize, - $"ValueGrowth {inlineValueGrowth} does not equal expected {oldTotalInlineValueSize - ObjectIdMap.ObjectIdSize}"); - - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertInlineToOverflow(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertInlineToOverflow(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); + Debug.Assert(inlineValueGrowth == ObjectIdMap.ObjectIdSize - oldInlineValueSize, + $"ValueGrowth {inlineValueGrowth} does not equal expected {oldInlineValueSize - ObjectIdMap.ObjectIdSize}"); + _ = LogField.ConvertInlineToOverflow(ref InfoRef, physicalAddress, valueAddress, oldInlineValueSize, in sizeInfo, objectIdMap); } else { Debug.Assert(sizeInfo.ValueIsObject, "Expected ValueIsObject to be set, pt 1"); - - // If there is no change in inline size (it is already the out-of-line inline-portion size), we don't need the optional-shift. - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertInlineToHeapObject(ref InfoRef, (long)ValueObjectIdAddress, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertInlineToHeapObject(ref InfoRef, (long)ValueObjectIdAddress, objectIdMap); + _ = LogField.ConvertInlineToValueObject(ref InfoRef, physicalAddress, valueAddress, in sizeInfo, objectIdMap); } } else if (Info.ValueIsOverflow) { if (sizeInfo.ValueIsInline) - { - // Convert from overflow to inline. We have already exited if it won't fit. - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertOverflowToInline(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertOverflowToInline(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - } + _ = LogField.ConvertOverflowToInline(ref InfoRef, physicalAddress, valueAddress, in sizeInfo, objectIdMap); else { Debug.Assert(sizeInfo.ValueIsObject, "Expected ValueIsObject to be set, pt 2"); - - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertOverflowToHeapObject(ref InfoRef, valueAddress, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertOverflowToHeapObject(ref InfoRef, valueAddress, objectIdMap); + _ = LogField.ConvertOverflowToValueObject(ref InfoRef, physicalAddress, valueAddress, in sizeInfo, objectIdMap); } } else @@ -390,70 +504,90 @@ public bool TrySetValueLength(in RecordSizeInfo sizeInfo) Debug.Assert(Info.ValueIsObject, "Expected ValueIsObject to be set, pt 3"); if (sizeInfo.ValueIsInline) - { - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertHeapObjectToInline(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertHeapObjectToInline(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - } + _ = LogField.ConvertValueObjectToInline(ref InfoRef, physicalAddress, valueAddress, in sizeInfo, objectIdMap); else { Debug.Assert(sizeInfo.ValueIsOverflow, "Expected ValueIsOverflow to be true"); - - if (shiftOptionals) - { - var optionalFields = OptionalFieldsShift.SaveAndClear(optionalStartAddress, ref InfoRef); - _ = LogField.ConvertHeapObjectToOverflow(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); - optionalFields.Restore(optionalStartAddress + inlineValueGrowth, ref InfoRef, fillerLen); - } - else - _ = LogField.ConvertHeapObjectToOverflow(ref InfoRef, valueAddress, newInlineDataLength, objectIdMap); + _ = LogField.ConvertValueObjectToOverflow(ref InfoRef, physicalAddress, valueAddress, in sizeInfo, objectIdMap); } } } - Done: + // Update record part 4: Restore optionals to their new location. + optionalStartAddress += inlineValueGrowth; + if (shiftOptionals) + optionalFields.Restore(optionalStartAddress, Info); + + // Update record part 5: Update Filler length in its new location. We don't want to change the optional presence or values here, + // so don't adjust fillerLen for that; only adjust it for value length change. Because (TrySet|Remove)(ETag|Expiration) update + // RecordInfo (affecting OptionalSize), we cannot use the same "set valuelength to full value space minus optional size" as we + // can't change both RecordInfo and the varbyte word atomically. Therefore we must zeroinit from the end of the current "filler + // space" (either the address, if it was within the less-than-FillerLengthSize bytes at the end of the record, or the end of the + // int value) if we have shrunk the value. + fillerLen -= inlineValueGrowth; // optional data is unchanged even if newOptionalSize != oldOptionalSize + var hasFillerBit = 0L; + var newFillerLenAddress = optionalStartAddress + oldOptionalSize; // optional data is unchanged even if newOptionalSize != oldOptionalSize + var endOfNewFillerSpace = newFillerLenAddress; + if (fillerLen >= FillerLengthSize) + { + *(int*)newFillerLenAddress = fillerLen; + hasFillerBit = kHasFillerBitMask; + endOfNewFillerSpace += FillerLengthSize; + } + if (inlineValueGrowth < 0) + { + // Zeroinit any empty space we opened up by shrinking. + var endOfOldFillerSpace = fillerLenAddress + (fillerLen >= FillerLengthSize ? FillerLengthSize : 0); + var clearLength = (int)(endOfOldFillerSpace - endOfNewFillerSpace); + // If old filler space was < FillerLengthSize and we only shrank by a couple bytes, we may have written FillerLengthSize leaving clearLength <= 0 + if (clearLength > 0) + new Span((byte*)endOfNewFillerSpace, clearLength).Clear(); + } + else if (zeroInit) + { + // Zeroinit any space we grew by. For example, if we grew by one byte we might have a stale fillerLength in that byte. + new Span((byte*)(valueAddress + oldInlineValueSize), (int)(newInlineValueSize - oldInlineValueSize)).Clear(); + } + + // Update record part 6: Finally, set varbyte value length to the actual new value length, with filler bit if we have it. + // We'll be consistent internally and Scan will be able to navigate to the end of the record with GetInlineRecordSizes().allocatedSize. + *(long*)IndicatorAddress = CreateUpdatedInlineVarbyteLengthWord(*(long*)IndicatorAddress, keyLengthBytes, valueLengthBytes, newInlineValueSize, hasFillerBit); + Debug.Assert(Info.ValueIsInline == sizeInfo.ValueIsInline, "Final ValueIsInline is inconsistent"); - Debug.Assert(!Info.ValueIsInline || ValueSpan.Length <= sizeInfo.MaxInlineValueSpanSize, $"Inline ValueSpan.Length {ValueSpan.Length} is greater than sizeInfo.MaxInlineValueSpanSize {sizeInfo.MaxInlineValueSpanSize}"); + Debug.Assert(!Info.ValueIsInline || ValueSpan.Length <= sizeInfo.MaxInlineValueSize, $"Inline ValueSpan.Length {ValueSpan.Length} is greater than sizeInfo.MaxInlineValueSpanSize {sizeInfo.MaxInlineValueSize}"); return true; } + /// + /// Set the value span, checking for conversion from inline and for space for optionals (ETag, Expiration). + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TrySetValueSpan(ReadOnlySpan value, in RecordSizeInfo sizeInfo) + public readonly bool TrySetValueSpan(ReadOnlySpan value, in RecordSizeInfo sizeInfo, bool zeroInit = false) { RecordSizeInfo.AssertValueDataLength(value.Length, in sizeInfo); - - if (!TrySetValueLength(in sizeInfo)) + if (!TrySetValueLength(in sizeInfo, zeroInit, out var valueAddress)) return false; - value.CopyTo(LogField.AsSpan(ValueAddress, Info.ValueIsInline, objectIdMap)); + var valueSpan = sizeInfo.ValueIsInline ? new((byte*)valueAddress, sizeInfo.FieldInfo.ValueSize) : objectIdMap.GetOverflowByteArray(*(int*)valueAddress).Span; + value.CopyTo(valueSpan); return true; } /// /// Set the object, checking for conversion from inline and for space for optionals (ETag, Expiration). /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state - public bool TrySetValueObject(IHeapObject value, in RecordSizeInfo sizeInfo) -#pragma warning restore IDE0251 - { - return TrySetValueLength(in sizeInfo) && TrySetValueObject(value); - } - + public readonly bool TrySetValueObject(IHeapObject value, in RecordSizeInfo sizeInfo) => TrySetValueLength(in sizeInfo) && TrySetValueObject(value); /// /// This overload must be called only when it is known the LogRecord's Value is not inline, and there is no need to check /// optionals (ETag or Expiration). In that case it is faster to just set the object. /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state - public bool TrySetValueObject(IHeapObject value) -#pragma warning restore IDE0251 + public readonly bool TrySetValueObject(IHeapObject value) { Debug.Assert(Info.ValueIsObject, $"Cannot call this overload of {GetCurrentMethodName()} for non-object Value"); @@ -463,22 +597,27 @@ public bool TrySetValueObject(IHeapObject value) return false; } - var objectId = *ValueObjectIdAddress; + var (valueLength, valueAddress) = GetValueFieldInfo(IndicatorAddress); + + // If there is no object there yet, allocate a slot + var objectId = *(int*)valueAddress; if (objectId == ObjectIdMap.InvalidObjectId) - objectId = *ValueObjectIdAddress = objectIdMap.Allocate(); + objectId = *(int*)valueAddress = objectIdMap.Allocate(); + // Set the new object into the slot objectIdMap.Set(objectId, value); return true; } private readonly int ETagLen => Info.HasETag ? ETagSize : 0; private readonly int ExpirationLen => Info.HasExpiration ? ExpirationSize : 0; + private readonly int ObjectLogPositionLen => Info.RecordHasObjects ? ObjectLogPositionSize : 0; /// A tuple of the total size of the main-log (inline) portion of the record, with and without filler length. public readonly (int actualSize, int allocatedSize) GetInlineRecordSizes() { if (Info.IsNull) - return (RecordInfo.GetLength(), RecordInfo.GetLength()); + return (RecordInfo.Size, RecordInfo.Size); var actualSize = ActualRecordSize; return (actualSize, actualSize + GetFillerLength()); } @@ -486,11 +625,14 @@ public readonly (int actualSize, int allocatedSize) GetInlineRecordSizes() [MethodImpl(MethodImplOptions.AggressiveInlining)] internal readonly long GetOptionalStartAddress() { - var valueAddress = ValueAddress; - return ValueAddress + LogField.GetInlineTotalSizeOfField(valueAddress, Info.ValueIsInline); + var (valueLength, valueAddress) = GetValueFieldInfo(IndicatorAddress); + return valueAddress + valueLength; } - public readonly int OptionalLength => ETagLen + ExpirationLen; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly ReadOnlySpan GetOptionalFieldsSpan() => new((byte*)GetOptionalStartAddress(), OptionalLength); + + public readonly int OptionalLength => ETagLen + ExpirationLen + ObjectLogPositionLen; [MethodImpl(MethodImplOptions.AggressiveInlining)] internal readonly long GetETagAddress() => GetOptionalStartAddress(); @@ -505,7 +647,7 @@ internal readonly long GetOptionalStartAddress() [MethodImpl(MethodImplOptions.AggressiveInlining)] internal readonly int GetFillerLength(long fillerLenAddress) { - if (Info.HasFiller) + if (HasFiller(IndicatorByte)) return *(int*)fillerLenAddress; // Filler includes Filler space opened up by removing ETag or Expiration. If there is no Filler, we may still have a couple bytes (< Constants.FillerLenSize) @@ -514,51 +656,53 @@ internal readonly int GetFillerLength(long fillerLenAddress) return RoundUp(recSize, Constants.kRecordAlignment) - recSize; } + /// + /// Set the filler length (the extra data length, if any). + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies the object list - public void SetFillerLength(int allocatedSize) -#pragma warning restore IDE0251 + public readonly void InitializeFillerLength(in RecordSizeInfo sizeInfo) { // This assumes Key and Value lengths have been set. It is called when we have initialized a record, or reinitialized due to revivification etc. - // Therefore optionals (ETag, Expiration) are not considered here. - var valueAddress = ValueAddress; - var fillerAddress = valueAddress + LogField.GetInlineTotalSizeOfField(valueAddress, Info.ValueIsInline); - var usedSize = (int)(fillerAddress - physicalAddress); - var fillerSize = allocatedSize - usedSize; + // Therefore the Optional ETag and Expiration flags have not yet been set in this LogRecord, so their space is considered filler here until we + // actually set them. However ObjectLogPositionSize is necessary if the ValueIsObject flag is set, so just subtract ETag and Expiration space. + Debug.Assert(!Info.HasOptionalFields, "Expected no optional flags in RecordInfo in InitializeFillerLength"); + var usedSize = sizeInfo.ActualInlineRecordSize - sizeInfo.ETagSize - sizeInfo.ExpirationSize; + var fillerSize = sizeInfo.AllocatedInlineRecordSize - usedSize; if (fillerSize >= FillerLengthSize) { - InfoRef.SetHasFiller(); // must do this first, for zero-init - *(int*)fillerAddress = fillerSize; + SetHasFiller(IndicatorAddress); // must do this first, for zero-init + *(int*)(physicalAddress + usedSize) = fillerSize; } } /// /// Called during cleanup of a record allocation, before the key was copied. /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeForReuse(in RecordSizeInfo sizeInfo) + public readonly void InitializeForReuse(in RecordSizeInfo sizeInfo) { Debug.Assert(!Info.HasETag && !Info.HasExpiration, "Record should not have ETag or Expiration here"); // This assumes the record has just been allocated, so it's at the tail (or very close to it). The Key and Value have not been set. - // The record does not need to be zeroinitialized if we are not doing that on initial allocation; the zero-length key and value - // combined with lack of optionals will yield the correct location of filler length (or it will be within the less-than-int-size - // roundup to allocation size). Consistency Note: LogField.FreeObjectIdAndConvertToInline also sets the field length to 0, then - // its caller sets the filler length to the remaining record size. - InfoRef.SetKeyIsInline(); - _ = LogField.SetInlineDataLength(KeyAddress, 0); - InfoRef.SetValueIsInline(); - _ = LogField.SetInlineDataLength(ValueAddress, 0); + // Calculate fillerSize to see if we have enough to set the filler bit, then create and set the indicator word. + var fillerSize = sizeInfo.AllocatedInlineRecordSize - sizeInfo.ActualInlineRecordSize; + var hasFillerBit = fillerSize >= FillerLengthSize ? kHasFillerBitMask : 0; + *(long*)IndicatorAddress = ConstructInlineVarbyteLengthWord(sizeInfo.FieldInfo.KeySize, sizeInfo.FieldInfo.ValueSize, hasFillerBit, out _ /*keyLengthBytes*/, out _ /*valueLengthBytes*/); - // Anything remaining is filler. - SetFillerLength(sizeInfo.AllocatedInlineRecordSize); + // If we have enough space, set the filler. + if (fillerSize >= FillerLengthSize) + *(int*)(physicalAddress + sizeInfo.AllocatedInlineRecordSize) = fillerSize; } + /// + /// Set the ETag, checking for space for optionals. + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state - public bool TrySetETag(long eTag) -#pragma warning restore IDE0251 + public readonly bool TrySetETag(long eTag) { if (Info.HasETag) { @@ -574,15 +718,19 @@ public bool TrySetETag(long eTag) if (fillerLen < 0) return false; - // Start at FillerLen address and back up, for speed + // Start at FillerLen address and back up, for speed. var address = fillerLenAddress; // Preserve zero-init by: // - Zeroing out FillerLen (this will leave only zeroes all the way to the next record, as there is nothing past FillerLen in this record). // - We must do this here in case there is not enough room for filler after the growth. - if (Info.HasFiller) + if (HasFiller(IndicatorByte)) *(int*)address = 0; + // We don't preserve the ObjectLogPosition field; that's only for serialization. We'll set it to 0 below. + if (Info.RecordHasObjects) + address -= ObjectLogPositionSize; + // - Preserve Expiration if present; set ETag; re-enter Expiration if present var expiration = 0L; if (Info.HasExpiration) @@ -597,22 +745,31 @@ public bool TrySetETag(long eTag) if (Info.HasExpiration) { - *(long*)address = expiration; // will be 0 or a valid expiration - address += ExpirationSize; // repositions to fillerAddress + *(long*)address = expiration; // will be 0 or a valid expiration + address += ExpirationSize; // repositions to fillerAddress + } + + // ObjectLogPosition is not preserved (it's only for serialization) but we set it to zero and adjust address for its space. + if (Info.RecordHasObjects) + { + *(long*)address = 0; + address += ObjectLogPositionSize; } // - Set the new (reduced) FillerLength if there is still space for it. if (fillerLen >= FillerLengthSize) *(int*)address = fillerLen; else - InfoRef.ClearHasFiller(); + ClearHasFiller(IndicatorAddress); return true; } + /// + /// Remove the ETag. + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state - public bool RemoveETag() -#pragma warning restore IDE0251 + public readonly bool RemoveETag() { if (!Info.HasETag) return true; @@ -627,8 +784,16 @@ public bool RemoveETag() // Preserve zero-init by: // - Zeroing out FillerLen (this will leave only zeroes all the way to the next record, as there is nothing past FillerLen in this record). - if (Info.HasFiller) + if (HasFiller(IndicatorByte)) *(int*)address = 0; + + // We don't preserve the ObjectLogPosition field; that's only for serialization. Just set it to 0 here. + if (Info.RecordHasObjects) + { + address -= ObjectLogPositionSize; + *(long*)address = 0; + } + // - Move Expiration, if present, up to cover ETag; then clear the ETag bit var expiration = 0L; var expirationSize = 0; @@ -646,17 +811,28 @@ public bool RemoveETag() address += expirationSize; // repositions to fillerAddress if expirationSize is nonzero InfoRef.ClearHasETag(); + // ObjectLogPosition is not preserved (it's only for serialization) but we set it to zero and adjust address for its space. + if (Info.RecordHasObjects) + { + *(long*)address = 0; + address += ObjectLogPositionSize; + } + // - Set the new (increased) FillerLength if there is space for it. if (fillerLen >= FillerLengthSize) { - InfoRef.SetHasFiller(); // May already be set, but will definitely now be true since we opened up more than FillerLengthSize bytes + SetHasFiller(IndicatorAddress); // May already be set, but will definitely now be true since we opened up more than FillerLengthSize bytes *(int*)address = fillerLen; } return true; } + /// + /// Set the Expiration, checking for space for optionals. + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TrySetExpiration(long expiration) + public readonly bool TrySetExpiration(long expiration) { if (expiration == 0) return RemoveExpiration(); @@ -680,26 +856,39 @@ public bool TrySetExpiration(long expiration) // Preserve zero-init by: // - Zeroing out FillerLen (this will leave only zeroes all the way to the next record, as there is nothing past FillerLen in this record). // - We must do this here in case there is not enough room for filler after the growth. - if (Info.HasFiller) + if (HasFiller(IndicatorByte)) *(int*)fillerLenAddress = 0; + // We don't preserve the ObjectLogPosition field; that's only for serialization. We'll set it to 0 below. + if (Info.RecordHasObjects) + fillerLenAddress -= ObjectLogPositionSize; + // - Set Expiration where filler space used to be InfoRef.SetHasExpiration(); *(long*)fillerLenAddress = expiration; fillerLenAddress += ExpirationSize; + // ObjectLogPosition is not preserved (it's only for serialization) but we set it to zero and adjust address for its space. + if (Info.RecordHasObjects) + { + *(long*)fillerLenAddress = 0; + fillerLenAddress += ObjectLogPositionSize; + } + // - Set the new (reduced) FillerLength if there is still space for it. if (fillerLen >= FillerLengthSize) *(int*)fillerLenAddress = fillerLen; else - InfoRef.ClearHasFiller(); + ClearHasFiller(IndicatorAddress); return true; } + /// + /// Remove the expiration + /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] -#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state - public bool RemoveExpiration() -#pragma warning restore IDE0251 + public readonly bool RemoveExpiration() { if (!Info.HasExpiration) return true; @@ -712,32 +901,48 @@ public bool RemoveExpiration() // Start at FillerLen address and back up, for speed // Preserve zero-init by: // - Zeroing out FillerLen (this will leave only zeroes all the way to the next record, as there is nothing past FillerLen in this record). - if (Info.HasFiller) + if (HasFiller(IndicatorByte)) *(int*)fillerLenAddress = 0; + // We don't preserve the ObjectLogPosition field; that's only for serialization. Just set it to 0 here and remove the space. + if (Info.RecordHasObjects) + { + fillerLenAddress -= ObjectLogPositionSize; + *(long*)fillerLenAddress = 0; + } + // - Remove Expiration and clear the Expiration bit; this will be the new fillerLenAddress fillerLenAddress -= ExpirationSize; *(long*)fillerLenAddress = 0; InfoRef.ClearHasExpiration(); + // ObjectLogPosition is not preserved (it's only for serialization) but we set it to zero and adjust address for its space. + if (Info.RecordHasObjects) + { + *(long*)fillerLenAddress = 0; + fillerLenAddress += ObjectLogPositionSize; + } + // - Set the new (increased) FillerLength if there is space for it. if (fillerLen >= FillerLengthSize) { - InfoRef.SetHasFiller(); // May already be set, but will definitely now be true since we opened up more than FillerLengthSize bytes + SetHasFiller(IndicatorAddress); // May already be set, but will definitely now be true since we opened up more than FillerLengthSize bytes *(int*)fillerLenAddress = fillerLen; } return true; } /// - /// Copy the entire record values: Value and optionals (ETag, Expiration) + /// Copy the entire record values: Value and optionals (ETag, Expiration). /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryCopyFrom(in TSourceLogRecord srcLogRecord, in RecordSizeInfo sizeInfo) + public readonly bool TryCopyFrom(in TSourceLogRecord srcLogRecord, in RecordSizeInfo sizeInfo) where TSourceLogRecord : ISourceLogRecord { - // This assumes the Key has been set and is not changed - if (!srcLogRecord.Info.ValueIsObject) + // TODOnow: For RENAME, add a key param to this and copy it in. Reflect this in RecordFieldInfo's KeyLength etc. (including whether it's overflow) + // For now, this assumes the Key has been set and is not changed + if (srcLogRecord.Info.ValueIsInline) { if (!TrySetValueLength(in sizeInfo)) return false; @@ -745,19 +950,28 @@ public bool TryCopyFrom(in TSourceLogRecord srcLogRecord, in R } else { - Debug.Assert(srcLogRecord.ValueObject is not null, "Expected srcLogRecord.ValueObject to be set (or deserialized) already"); - if (!TrySetValueObject(srcLogRecord.ValueObject, in sizeInfo)) - return false; + if (srcLogRecord.Info.ValueIsOverflow) + { + Debug.Assert(Info.ValueIsOverflow, "Expected this.Info.ValueIsOverflow to be set already"); + ValueOverflow = srcLogRecord.ValueOverflow; + } + else + { + // TODOnow: make sure Object isn't disposed by the source, to avoid use-after-Dispose. Maybe this (and DiskLogRecord remapping to TransientOIDMap) needs Clone() + Debug.Assert(srcLogRecord.ValueObject is not null, "Expected srcLogRecord.ValueObject to be set (or deserialized) already"); + if (!TrySetValueObject(srcLogRecord.ValueObject, in sizeInfo)) + return false; + } } - return TryCopyOptionals(in srcLogRecord, in sizeInfo); } /// /// Copy the record optional values (ETag, Expiration) /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryCopyOptionals(in TSourceLogRecord srcLogRecord, in RecordSizeInfo sizeInfo) + public readonly bool TryCopyOptionals(in TSourceLogRecord srcLogRecord, in RecordSizeInfo sizeInfo) where TSourceLogRecord : ISourceLogRecord { var srcRecordInfo = srcLogRecord.Info; @@ -776,73 +990,206 @@ public bool TryCopyOptionals(in TSourceLogRecord srcLogRecord, } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ClearOptionals() + public readonly void ClearOptionals() { _ = RemoveExpiration(); _ = RemoveETag(); } + /// + /// Clears any heap-allocated Value: Object or Overflow. Does not clear key (if it is Overflow). + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool FreeKeyOverflow() + public readonly void ClearValueIfHeap(Action objectDisposer) { - if (!Info.KeyIsOverflow) - return false; - LogField.FreeObjectIdAndConvertToInline(ref InfoRef, KeyAddress, objectIdMap, isKey: true); - return true; + if (Info.ValueIsInline) + return; + + if (!Info.KeyIsInline) + { + // The key is overflow and will remain that way, so we won't be changing ObjectLogPosition or filler. + // Just call this directly. + var (_ /*valueLength*/, valueAddress) = GetValueFieldInfo(IndicatorAddress); + LogField.ClearObjectIdAndConvertToInline(ref InfoRef, valueAddress, objectIdMap, isKey: false, objectDisposer); + return; + } + + // The key is not overflow so we must remove ObjectLogPosition and update filler. + ClearHeapFields(clearKey: false, objectDisposer); } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool FreeValueOverflow() + /// + /// Clears any heap-allocated field, Object or Overflow, in the Value and optionally the Key. If we go from + /// being false to true, then we need to adjust filler as well. + /// + public readonly void ClearHeapFields(bool clearKey, Action objectDisposer) { - if (!Info.ValueIsOverflow) - return false; - LogField.FreeObjectIdAndConvertToInline(ref InfoRef, ValueAddress, objectIdMap, isKey: false); - return true; + if (Info.RecordIsInline) + return; + var (valueLength, valueAddress) = GetValueFieldInfo(IndicatorAddress); + + // If we're not clearing the key and it's Heap, then we don't need the operations below to change ObjectLogPosition and filler. + if (!clearKey && !Info.KeyIsInline) + { + if (!Info.ValueIsInline) + LogField.ClearObjectIdAndConvertToInline(ref InfoRef, valueAddress, objectIdMap, isKey: false, objectDisposer); + return; + } + + // First create a varbyte word that uses the value length to be the full space from end of key to end of record. + var (keyLengthBytes, valueLengthBytes, hasFillerBit) = DeconstructIndicatorByte(*(byte*)IndicatorAddress); + var allocatedSize = GetInlineRecordSizes().allocatedSize; + var fullValueLength = (int)(physicalAddress + allocatedSize - valueAddress); + *(long*)IndicatorAddress = CreateIgnoreOptionalsVarbyteWord(*(long*)IndicatorAddress, keyLengthBytes, valueLengthBytes, fullValueLength); + + // Clear the filler if we have it set. Note that this offset still includes ObjectLogPosition. + var usedLength = (int)(valueAddress - physicalAddress + valueLength + ETagLen + ExpirationLen); + if (hasFillerBit) + *(int*)(physicalAddress + usedLength + ObjectLogPositionSize) = 0; + + // If we're here and the key is overflow we're clearing it. + if (!Info.KeyIsInline) + { + var keyAddress = IndicatorAddress + NumIndicatorBytes + keyLengthBytes + valueLengthBytes; + LogField.ClearObjectIdAndConvertToInline(ref InfoRef, keyAddress, objectIdMap, isKey: true); + } + if (!Info.ValueIsInline) + LogField.ClearObjectIdAndConvertToInline(ref InfoRef, valueAddress, objectIdMap, isKey: false, objectDisposer); + + // We know we have cleared out the ObjectLogPosition field, so we also know we have at least sizeof(int) bytes of filler. + // Set the current number of filler bytes. + *(int*)(physicalAddress + usedLength) = allocatedSize - usedLength; + + // Set varbyte value length to the actual new value length, with filler bit if we have it. valueLength has not changed. + // We'll be consistent internally and Scan will be able to navigate to the end of the record with GetInlineRecordSizes().allocatedSize. + *(long*)IndicatorAddress = CreateUpdatedInlineVarbyteLengthWord(*(long*)IndicatorAddress, keyLengthBytes, valueLengthBytes, (int)valueLength, kHasFillerBitMask); } /// /// For revivification or reuse: prepare the current record to be passed to initial updaters, based upon the sizeInfo's key and value lengths. /// + /// This is 'readonly' because it does not alter the fields of this object, only what they point to. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void PrepareForRevivification(in RecordSizeInfo sizeInfo, int allocatedSize) + public readonly void PrepareForRevivification(ref RecordSizeInfo sizeInfo, int allocatedSize) { - var keyAddress = KeyAddress; - var newKeySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; - var newValueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; - var fillerAddress = GetFillerLengthAddress(); - - // We expect that the key and value are and there are no optionals, per LogRecord.InitalizeForReuse and LogField.FreeObjectIdAndConvertToInline. + // We expect that the key and value are inline and there are no optionals, per LogRecord.InitalizeForReuse and LogField.ClearObjectIdAndConvertToInline. Debug.Assert(Info.KeyIsInline, "Expected Key to be inline in PrepareForRevivification"); Debug.Assert(Info.ValueIsInline, "Expected Value to be inline in PrepareForRevivification"); Debug.Assert(!Info.HasETag && !Info.HasExpiration, "Expected no optionals in PrepareForRevivification"); + Debug.Assert(allocatedSize >= sizeInfo.AllocatedInlineRecordSize, $"Expected allocatedSize {allocatedSize} >= sizeInfo.AllocatedInlineRecordSize {sizeInfo.AllocatedInlineRecordSize}"); + + // We can now set the new Key and value lengths directly. This record may have come from revivification so we must use the actual allocated length, + // not the requested sizeInfo.AllocatedInlineRecordSize, in the calculation of value length bytes. However, we will still use only the requested + // length as the valueLength, with the rest as filler. + + // Get new record layout part 1: Key bytes + var keyLength = sizeInfo.InlineKeySize; + var valueLength = sizeInfo.InlineValueSize; + Debug.Assert(keyLength + valueLength <= allocatedSize, "Insufficient new record size"); // Should not happen as we passed sizeInfo to BlockAllocate + var keyLengthBytes = GetByteCount(keyLength); + + // Get new record layout part 2: Full value space, using 1 byte for the estimated value length bytes, to ensure we don't underestimate the value length by a byte or two + // (there is thus a boundary case where we will thus overestimate the value space by a byte or two and allocate an extra length varbyte, but after 255 that's a tiny + // percentage of the record size). + var spaceAfterKey = allocatedSize - RecordInfo.Size - 1 /*indicatorByte*/ - keyLengthBytes - 1 /*valueLengthBytes*/; + var valueLengthBytes = GetByteCount(spaceAfterKey); + + // Get new record layout part 3: Determine if we have a filler; that is the space after the value and any optionals (eTag, expiration). + // If we do, set the value of that filler into the proper location before we set the new varbyte info. + var fillerSpace = spaceAfterKey - valueLength - sizeInfo.OptionalSize; + long hasFillerBit = 0; + if (fillerSpace > sizeof(int)) + { + hasFillerBit = kHasFillerBitMask; + *(int*)(physicalAddress + allocatedSize - fillerSpace) = fillerSpace; + } - var keyLength = LogField.GetInlineLengthRef(keyAddress); - var valueLength = LogField.GetInlineLengthRef(ValueAddress); + // Finally, set the new record layout and update sizeInfo. + *(long*)IndicatorAddress = ConstructInlineVarbyteLengthWord(keyLengthBytes, keyLength, valueLengthBytes, valueLength, hasFillerBit); + sizeInfo.AllocatedInlineRecordSize = allocatedSize; + } - Debug.Assert(newKeySize + newValueSize <= allocatedSize, "Insufficient new record size"); // Should not happen as we passed sizeInfo to BlockAllocate + /// + /// Sets the lengths of Overflow Keys and Values and Object values into the disk-image copy of the log record before the main-log page is flushed. + /// + /// + /// IMPORTANT: This is only to be called in the disk image copy of the log record, not in the actual log record itself. + /// + internal readonly void SetObjectLogRecordStartPositionAndLength(in ObjectLogFilePositionInfo objectLogFilePosition, ulong valueObjectLength) + { + if (Info.KeyIsOverflow) + { + var (keyLength, keyAddress) = GetKeyFieldInfo(IndicatorAddress); + var overflow = objectIdMap.GetOverflowByteArray(*(int*)keyAddress); + *(int*)keyAddress = overflow.Length; + } - // First zero out the filler. - if (Info.HasFiller) + var (valueLength, valueAddress) = GetValueFieldInfo(IndicatorAddress); + if (Info.ValueIsOverflow) + { + var overflow = objectIdMap.GetOverflowByteArray(*(int*)valueAddress); + *(int*)valueAddress = overflow.Length; + } + else if (Info.ValueIsObject) + { + *(uint*)valueAddress = (uint)(valueObjectLength & 0xFFFFFFFF); + // Update the high byte in the value length metadata (it is combined with the length that is stored in the ObjectId field data of the record). + UpdateVarbyteValueLengthByteInWord(IndicatorAddress, (byte)((valueObjectLength >> 32) & 0xFF)); + } + else if (Info.RecordIsInline) // ValueIsInline is true; if the record is fully inline, we should not be called here { - *(int*)fillerAddress = 0; - InfoRef.ClearHasFiller(); + Debug.Fail("Cannot call SetObjectLogRecordStartPositionAndLength for an inline record"); + return; } - // The key is already inline. If the new key length will be greater, then zero the value data and length and set the key length to the new size. - // Otherwise leave things as they are. When AllocatorBase.SerializeKey and allocator.InitializeValue run, they will just write the same lengths - // (including filler) over anything we're setting here. - var keyGrowth = sizeInfo.InlineTotalKeySize - (keyLength + LogField.InlineLengthPrefixSize); - if (keyGrowth > 0) + // If we've replaced the varbyte valueLength with the upper byte of valueObjectLength, the usual optional accessors (e.g.GetExpirationAddress()) + // won't work, so we add all the pieces together here. + *(ulong*)(valueAddress + valueLength + ETagLen + ExpirationLen) = objectLogFilePosition.word; + } + + /// + /// Returns the object log position for the start of the key (if any) and value (if any). + /// + /// Outputs key length; will always be for overflow + /// Outputs key length; will be for overflow or object + /// The object log position for this record + internal readonly ulong GetObjectLogRecordStartPositionAndLengths(out int keyLength, out ulong valueObjectLength) + { + if (Info.KeyIsOverflow) { - // Zero out the old value data. Note: We could minimize this length by seeing how far the key grows (careful of it partially overwriting value length), - // what the new value length is, etc. - new Span((byte*)ValueAddress, valueLength).Clear(); - LogField.GetInlineLengthRef(ValueAddress) = 0; - LogField.GetInlineLengthRef(keyAddress) += keyGrowth; + (keyLength, var keyAddress) = GetKeyFieldInfo(IndicatorAddress); + keyLength = *(int*)keyAddress; } + else // KeyIsInline is true; keyLength will be ignored + keyLength = 0; - // Anything remaining is filler. - SetFillerLength(allocatedSize); + var (valueLength, valueAddress) = GetValueFieldInfo(IndicatorAddress); + if (Info.ValueIsOverflow) + valueObjectLength = (ulong)*(int*)valueAddress; + else if (Info.ValueIsObject) + { + valueObjectLength = *(uint*)valueAddress | (((ulong)valueLength & 0xFF) << 32); + valueLength = ObjectIdMap.ObjectIdSize; // locally only, restore this. We will restore it for real later, when we read the objects. + } + else // ValueIsInline is true; valueLength will be ignored + { + valueObjectLength = 0; + if (Info.RecordIsInline) // If the record is fully inline, we should not be called here + { + Debug.Fail("Cannot call GetObjectLogRecordStartPositionAndLength for an inline record"); + return 0; + } + } + + // If we've replaced the varbyte valueLength with the upper byte of valueObjectLength, the usual optional accessors (e.g.GetExpirationAddress()) + // won't work, so we add all the pieces together here. + return *(ulong*)(valueAddress + valueLength + ETagLen + ExpirationLen); + } + + public void Dispose(Action objectDisposer) + { + if (IsSet) + ClearHeapFields(clearKey: true, objectDisposer); } public override readonly string ToString() @@ -854,4 +1201,4 @@ public override readonly string ToString() return $"ri {Info} | key {Key.ToShortString(20)} | val {valueString} | HasETag {bstr(Info.HasETag)}:{ETag} | HasExpiration {bstr(Info.HasExpiration)}:{Expiration}"; } } -} \ No newline at end of file +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocator.cs index 31a80df7e89..ce1b638d48b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocator.cs @@ -32,27 +32,13 @@ public readonly AllocatorBase GetBase() where TAllocator : IAllocator => (AllocatorBase)(object)_this; - /// - public readonly bool IsFixedLength => true; - /// public readonly bool HasObjectLog => true; /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetStartLogicalAddress(long page) => _this.GetStartLogicalAddressOfPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddressOnPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void InitializeValue(long physicalAddress, in RecordSizeInfo sizeInfo) => ObjectAllocatorImpl.InitializeValue(physicalAddress, in sizeInfo); + public readonly void InitializeRecord(ReadOnlySpan key, long logicalAddress, in RecordSizeInfo sizeInfo, ref LogRecord logRecord) + => _this.InitializeRecord(key, logicalAddress, in sizeInfo, ref logRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -96,14 +82,6 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.AllocatePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly bool IsAllocated(int pageIndex) => _this.IsAllocated(pageIndex); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe void PopulatePage(byte* src, int required_bytes, long destinationPageIndex) => _this.PopulatePage(src, required_bytes, destinationPageIndex); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void MarkPage(long logicalAddress, long version) => _this.MarkPage(logicalAddress, version); @@ -112,25 +90,13 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.MarkPageAtomic(logicalAddress, version); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void ClearPage(long page, int offset = 0) => _this.ClearPage(page, offset); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void FreePage(long pageIndex) => _this.FreePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() => _this.GetSegmentOffsets(); - /// public readonly int OverflowPageCount => _this.OverflowPageCount; - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => _this.SerializeKey(key, logicalAddress, ref logRecord); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly LogRecord CreateLogRecord(long logicalAddress) => _this.CreateLogRecord(logicalAddress); @@ -139,6 +105,12 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.CreateLogRecord(logicalAddress, physicalAddress); + /// + public readonly LogRecord CreateRemappedLogRecordOverTransientMemory(long logicalAddress, long physicalAddress) => _this.CreateRemappedLogRecordOverTransientMemory(logicalAddress, physicalAddress); + + /// + public readonly ObjectIdMap TranssientObjectIdMap => _this.transientObjectIdMap; + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs index e5db01c69d1..59824f58acf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs @@ -5,7 +5,6 @@ using System; using System.Diagnostics; -using System.IO; using System.Runtime.CompilerServices; using System.Runtime.InteropServices; using System.Threading; @@ -13,16 +12,12 @@ namespace Tsavorite.core { - using static LogAddress; +#pragma warning disable IDE0065 // Misplaced using directive using static Utility; internal sealed unsafe class ObjectAllocatorImpl : AllocatorBase> where TStoreFunctions : IStoreFunctions { - /// Circular buffer definition, in parallel with - /// The long is actually a byte*, but storing as 'long' makes going through logicalAddress/physicalAddress translation more easily - long* pagePointers; - /// For each in-memory page of this allocator we have an for keys that are too large to fit inline into the main log /// and become overflow byte[], or are Object values; this is needed to root the objects for GC. internal struct ObjectPage @@ -32,38 +27,53 @@ internal struct ObjectPage public ObjectPage() => objectIdMap = new(); internal readonly void Clear() => objectIdMap?.Clear(); // TODO: Ensure we have already called the RecordDisposer + + public override readonly string ToString() => $"oidMap {objectIdMap}"; } - internal ObjectPage[] values; + /// The pages of the log, containing object storage. In parallel with AllocatorBase.pagePointers + internal ObjectPage[] pages; + + /// The position information for the next write to the object log. + ObjectLogFilePositionInfo objectLogNextRecordStartPosition; + + // Default to max sizes so testing a size as "greater than" will always be false + readonly int maxInlineKeySize = LogSettings.kMaxInlineKeySize; + readonly int maxInlineValueSize = int.MaxValue; - readonly int maxInlineKeySize; - readonly int maxInlineValueSize; + readonly int numberOfFlushBuffers; + readonly int numberOfDeserializationBuffers; - // Size of object chunks being written to storage - // TODO: private readonly int objectBlockSize = 100 * (1 << 20); + private readonly IDevice objectLogDevice; + /// The free pages of the log private readonly OverflowPool> freePagePool; public ObjectAllocatorImpl(AllocatorSettings settings, TStoreFunctions storeFunctions, Func> wrapperCreator) - : base(settings.LogSettings, storeFunctions, wrapperCreator, settings.evictCallback, settings.epoch, settings.flushCallback, settings.logger) + : base(settings.LogSettings, storeFunctions, wrapperCreator, settings.evictCallback, settings.epoch, settings.flushCallback, settings.logger, transientObjectIdMap: new ObjectIdMap()) { - IsObjectAllocator = true; + objectLogDevice = settings.LogSettings.ObjectLogDevice; maxInlineKeySize = 1 << settings.LogSettings.MaxInlineKeySizeBits; maxInlineValueSize = 1 << settings.LogSettings.MaxInlineValueSizeBits; freePagePool = new OverflowPool>(4, static p => { }); - var bufferSizeInBytes = (nuint)RoundUp(sizeof(long*) * BufferSize, Constants.kCacheLineBytes); - pagePointers = (long*)NativeMemory.AlignedAlloc(bufferSizeInBytes, Constants.kCacheLineBytes); - NativeMemory.Clear(pagePointers, bufferSizeInBytes); + if (settings.LogSettings.NumberOfFlushBuffers < LogSettings.kMinFlushBuffers || settings.LogSettings.NumberOfFlushBuffers > LogSettings.kMaxFlushBuffers || !IsPowerOfTwo(settings.LogSettings.NumberOfFlushBuffers)) + throw new TsavoriteException($"{nameof(settings.LogSettings.NumberOfFlushBuffers)} must be between {LogSettings.kMinFlushBuffers} and {LogSettings.kMaxFlushBuffers - 1} and a power of 2"); + numberOfFlushBuffers = settings.LogSettings.NumberOfFlushBuffers; - values = new ObjectPage[BufferSize]; - for (var ii = 0; ii < BufferSize; ii++) - values[ii] = new(); + if (settings.LogSettings.NumberOfDeserializationBuffers < LogSettings.kMinDeserializationBuffers || settings.LogSettings.NumberOfDeserializationBuffers > LogSettings.kMaxDeserializationBuffers || !IsPowerOfTwo(settings.LogSettings.NumberOfDeserializationBuffers)) + throw new TsavoriteException($"{nameof(settings.LogSettings.NumberOfDeserializationBuffers)} must be between {LogSettings.kMinDeserializationBuffers} and {LogSettings.kMaxDeserializationBuffers - 1} and a power of 2"); + numberOfDeserializationBuffers = settings.LogSettings.NumberOfDeserializationBuffers; + + if (settings.LogSettings.ObjectLogSegmentSizeBits is < LogSettings.kMinObjectLogSegmentSizeBits or > LogSettings.kMaxSegmentSizeBits) + throw new TsavoriteException($"{nameof(settings.LogSettings.ObjectLogSegmentSizeBits)} must be between {LogSettings.kMinObjectLogSegmentSizeBits} and {LogSettings.kMaxSegmentSizeBits}"); + objectLogNextRecordStartPosition.SegmentSizeBits = settings.LogSettings.ObjectLogSegmentSizeBits; - // For LogField conversions between inline and heap fields, we assume the inline field size prefix is the same size as objectId size - Debug.Assert(LogField.InlineLengthPrefixSize == ObjectIdMap.ObjectIdSize, "InlineLengthPrefixSize must be equal to ObjectIdMap.ObjectIdSize"); + pages = new ObjectPage[BufferSize]; + for (var ii = 0; ii < BufferSize; ii++) + pages[ii] = new(); } internal int OverflowPageCount => freePagePool.Count; @@ -87,15 +97,16 @@ internal void AllocatePage(int index) if (freePagePool.TryGet(out var item)) { pagePointers[index] = item.pointer; - values[index] = item.value; - // TODO resize the values[index] arrays smaller if they are above a certain point - return; + pages[index] = item.value; } - - // No free pages are available so allocate new - pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); - NativeMemory.Clear((void*)pagePointers[index], (nuint)PageSize); - values[index] = new(); + else + { + // No free pages are available so allocate new + pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); + NativeMemory.Clear((void*)pagePointers[index], (nuint)PageSize); + pages[index] = new(); + } + PageHeader.Initialize(pagePointers[index]); } void ReturnPage(int index) @@ -106,7 +117,7 @@ void ReturnPage(int index) _ = freePagePool.TryAdd(new() { pointer = pagePointers[index], - value = values[index] + value = pages[index] }); pagePointers[index] = default; _ = Interlocked.Decrement(ref AllocatedPageCount); @@ -117,38 +128,18 @@ void ReturnPage(int index) internal LogRecord CreateLogRecord(long logicalAddress) => CreateLogRecord(logicalAddress, GetPhysicalAddress(logicalAddress)); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new(physicalAddress, values[GetPageIndexForAddress(logicalAddress)].objectIdMap); + internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new(physicalAddress, pages[GetPageIndexForAddress(logicalAddress)].objectIdMap); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal ObjectIdMap GetObjectIdMap(long logicalAddress) => values[GetPageIndexForAddress(logicalAddress)].objectIdMap; + internal LogRecord CreateRemappedLogRecordOverTransientMemory(long logicalAddress, long physicalAddress) + => LogRecord.CreateRemappedOverTransientMemory(physicalAddress, pages[GetPageIndexForAddress(logicalAddress)].objectIdMap, transientObjectIdMap); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => SerializeKey(key, logicalAddress, ref logRecord, maxInlineKeySize, GetObjectIdMap(logicalAddress)); - - public override void Initialize() => Initialize(FirstValidAddress); + internal ObjectIdMap GetObjectIdMap(long logicalAddress) => pages[GetPageIndexForAddress(logicalAddress)].objectIdMap; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static void InitializeValue(long physicalAddress, in RecordSizeInfo sizeInfo) - { - var valueAddress = LogRecord.GetValueAddress(physicalAddress); - if (sizeInfo.ValueIsInline) - { - // Set the actual length indicator for inline. - LogRecord.GetInfoRef(physicalAddress).SetValueIsInline(); - _ = LogField.SetInlineDataLength(valueAddress, sizeInfo.FieldInfo.ValueDataSize); - } - else - { - // If it's a revivified record, it may have ValueIsInline set, so clear that. - LogRecord.GetInfoRef(physicalAddress).ClearValueIsInline(); - - if (sizeInfo.ValueIsObject) - LogRecord.GetInfoRef(physicalAddress).SetValueIsObject(); - - // Either an IHeapObject or a byte[] - *LogRecord.GetValueObjectIdAddress(physicalAddress) = ObjectIdMap.InvalidObjectId; - } - } + public void InitializeRecord(ReadOnlySpan key, long logicalAddress, in RecordSizeInfo sizeInfo, ref LogRecord logRecord) + => logRecord.InitializeRecord(key, in sizeInfo, GetObjectIdMap(logicalAddress)); [MethodImpl(MethodImplOptions.AggressiveInlining)] public RecordSizeInfo GetRMWCopyRecordSize(in TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) @@ -210,8 +201,8 @@ public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) { FieldInfo = new() { - KeyDataSize = key.Length, - ValueDataSize = 0, // This will be inline, and with the length prefix and possible space when rounding up to kRecordAlignment, allows the possibility revivification can reuse the record for a Heap Field + KeySize = key.Length, + ValueSize = 0, // This will be inline, and with the length prefix and possible space when rounding up to kRecordAlignment, allows the possibility revivification can reuse the record for a Heap Field HasETag = false, HasExpiration = false } @@ -223,36 +214,30 @@ public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) [MethodImpl(MethodImplOptions.AggressiveInlining)] public void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) { + // Object allocator may have Inline or Overflow Keys or Values; additionally, Values may be Object. Both non-inline cases are an objectId in the record. // Key - sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeyDataSize <= maxInlineKeySize; - var keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeySize <= maxInlineKeySize; + var keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeySize : ObjectIdMap.ObjectIdSize; // Value - sizeInfo.MaxInlineValueSpanSize = maxInlineValueSize; - sizeInfo.ValueIsInline = !sizeInfo.ValueIsObject && sizeInfo.FieldInfo.ValueDataSize <= sizeInfo.MaxInlineValueSpanSize; - var valueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + sizeInfo.MaxInlineValueSize = maxInlineValueSize; + sizeInfo.ValueIsInline = !sizeInfo.ValueIsObject && sizeInfo.FieldInfo.ValueSize <= sizeInfo.MaxInlineValueSize; + var valueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueSize : ObjectIdMap.ObjectIdSize; // Record - sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; - sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); + sizeInfo.CalculateSizes(keySize, valueSize); } + /// + /// Dispose an in-memory + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) { - logRecord.ClearOptionals(); - if (disposeReason != DisposeReason.Deleted) - _ = logRecord.FreeKeyOverflow(); - - if (!logRecord.Info.ValueIsInline) + if (logRecord.IsSet) { - if (!logRecord.FreeValueOverflow() && logRecord.ValueObjectId != ObjectIdMap.InvalidObjectId) - { - var heapObj = logRecord.ValueObject; - if (heapObj is not null) - storeFunctions.DisposeValueObject(heapObj, disposeReason); - logRecord.objectIdMap.Free(logRecord.ValueObjectId); - } + logRecord.ClearHeapFields(disposeReason != DisposeReason.Deleted, obj => storeFunctions.DisposeValueObject(obj, disposeReason)); + logRecord.ClearOptionals(); } } @@ -260,7 +245,7 @@ internal void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason internal void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) { // Clear the IHeapObject if we deserialized it - if (logRecord.Info.ValueIsObject && logRecord.ValueObject is not null) + if (logRecord.IsSet && logRecord.Info.ValueIsObject && logRecord.ValueObject is not null) storeFunctions.DisposeValueObject(logRecord.ValueObject, disposeReason); } @@ -269,90 +254,54 @@ internal void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeRe ///
public override void Dispose() { - var localValues = Interlocked.Exchange(ref values, null); + var localValues = Interlocked.Exchange(ref pages, null); if (localValues != null) { - base.Dispose(); freePagePool.Dispose(); foreach (var value in localValues) value.Clear(); - - if (pagePointers is not null) - { - for (var ii = 0; ii < BufferSize; ii++) - { - if (pagePointers[ii] != 0) - NativeMemory.AlignedFree((void*)pagePointers[ii]); - } - NativeMemory.AlignedFree((void*)pagePointers); - pagePointers = null; - } + base.Dispose(); } } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetPhysicalAddress(long logicalAddress) - { - // Index of page within the circular buffer, and offset on the page - var pageIndex = GetPageIndexForAddress(logicalAddress); - var offset = GetOffsetOnPage(logicalAddress); - return *(pagePointers + pageIndex) + offset; - } - - internal bool IsAllocated(int pageIndex) => pagePointers[pageIndex] != 0; - - protected override void TruncateUntilAddress(long toAddress) // TODO: ObjectAllocator specifics if any + protected override void TruncateUntilAddress(long toAddress) { base.TruncateUntilAddress(toAddress); } - protected override void TruncateUntilAddressBlocking(long toAddress) // TODO: ObjectAllocator specifics if any + protected override void TruncateUntilAddressBlocking(long toAddress) { base.TruncateUntilAddressBlocking(toAddress); } - protected override void RemoveSegment(int segment) // TODO: ObjectAllocator specifics if any + protected override void RemoveSegment(int segment) { + //TODOnow("Get the object log segment information from this main-log segment's last PageHeader"); base.RemoveSegment(segment); } - protected override void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) - { - WriteAsync(flushPage, - (ulong)(AlignedPageSizeBytes * flushPage), - (uint)PageSize, - callback, - asyncResult, device); - } + protected override void WriteAsync(CircularDiskWriteBuffer flushBuffers, long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) + => WriteAsync(flushBuffers, flushPage, (ulong)(AlignedPageSizeBytes * flushPage), (uint)PageSize, callback, asyncResult, device, objectLogDevice); - protected override void WriteAsyncToDevice - (long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, - PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long[] localSegmentOffsets, long fuzzyStartLogicalAddress) + protected override void WriteAsyncToDevice(CircularDiskWriteBuffer flushBuffers, long startPage, long flushPage, int possiblyPartialPageSize, DeviceIOCompletionCallback callback, + PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long fuzzyStartLogicalAddress) { -#if READ_WRITE VerifyCompatibleSectorSize(device); VerifyCompatibleSectorSize(objectLogDevice); - var epochTaken = false; - if (!epoch.ThisInstanceProtected()) - { - epochTaken = true; - epoch.Resume(); - } + var epochTaken = epoch.ResumeIfNotProtected(); try { - if (HeadAddress >= (flushPage << LogPageSizeBits) + pageSize) + if (HeadAddress >= GetLogicalAddressOfStartOfPage(flushPage) + possiblyPartialPageSize) { // Requested page is unavailable in memory, ignore callback(0, 0, asyncResult); } else { - // We are writing to separate device, so use fresh segment offsets - WriteAsync(flushPage, - (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), - (uint)pageSize, callback, asyncResult, - device, objectLogDevice, flushPage, localSegmentOffsets, fuzzyStartLogicalAddress); + // We are writing to a separate device which starts at "startPage" + WriteAsync(flushBuffers, flushPage, (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), (uint)possiblyPartialPageSize, + callback, asyncResult, device, objectLogDevice, fuzzyStartLogicalAddress); } } finally @@ -360,30 +309,13 @@ protected override void WriteAsyncToDevice if (epochTaken) epoch.Suspend(); } -#endif // READ_WRITE - } - - internal void ClearPage(long page, int offset) - { - // This is called during recovery, not as part of normal operations, so there is no need to walk pages starting at offset to Free() ObjectIds - NativeMemory.Clear((byte*)pagePointers[page] + offset, (nuint)(PageSize - offset)); } internal void FreePage(long page) { - ClearPage(page, 0); - - // Close segments - var thisCloseSegment = page >> (LogSegmentSizeBits - LogPageSizeBits); - var nextCloseSegment = (page + 1) >> (LogSegmentSizeBits - LogPageSizeBits); + pages[page % BufferSize].objectIdMap.Clear(); -#if READ_WRITE - if (thisCloseSegment != nextCloseSegment) - { - // We are clearing the last page in current segment - segmentOffsets[thisCloseSegment % SegmentBufferSize] = 0; - } -#endif // READ_WRITE + ClearPage(page, 0); // If all pages are being used (i.e. EmptyPageCount == 0), nothing to re-utilize by adding // to overflow pool. @@ -391,11 +323,23 @@ internal void FreePage(long page) ReturnPage((int)(page % BufferSize)); } - private void WriteAsync(long flushPage, ulong alignedDestinationAddress, uint numBytesToWrite, + /// Create the flush buffer (for only) + internal override CircularDiskWriteBuffer CreateCircularFlushBuffers(IDevice objectLogDevice, ILogger logger) + => new(bufferPool, IStreamBuffer.BufferSize, numberOfFlushBuffers, objectLogDevice ?? this.objectLogDevice, logger); + + /// Create the flush buffer (for only) + internal override CircularDiskReadBuffer CreateCircularReadBuffers(IDevice objectLogDevice, ILogger logger) + => new(bufferPool, IStreamBuffer.BufferSize, numberOfDeserializationBuffers, objectLogDevice ?? this.objectLogDevice, logger); + + private CircularDiskReadBuffer CreateCircularReadBuffers() + => new(bufferPool, IStreamBuffer.BufferSize, numberOfDeserializationBuffers, objectLogDevice, logger); + + private void WriteAsync(CircularDiskWriteBuffer flushBuffers, long flushPage, ulong alignedMainLogFlushPageAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult, - IDevice device, long intendedDestinationPage = -1, long[] localSegmentOffsets = null, long fuzzyStartLogicalAddress = long.MaxValue) + IDevice device, IDevice objectLogDevice, long fuzzyStartLogicalAddress = long.MaxValue) { -#if READ_WRITE + // We flush within the DiskStreamWriteBuffer, so we do not use the asyncResult here for IO (until the final callback), but it has necessary fields. + // Short circuit if we are using a null device if ((device as NullDevice) != null) { @@ -403,254 +347,145 @@ private void WriteAsync(long flushPage, ulong alignedDestinationAddres return; } - int start = 0, aligned_start = 0, end = (int)numBytesToWrite; - if (asyncResult.partial) - { - // We're writing only a subset of the page - start = (int)(asyncResult.fromAddress - (asyncResult.page << LogPageSizeBits)); - aligned_start = (start / sectorSize) * sectorSize; - end = (int)(asyncResult.untilAddress - (asyncResult.page << LogPageSizeBits)); - } - - // Check if user did not override with special segment offsets - localSegmentOffsets ??= segmentOffsets; + Debug.Assert(asyncResult.page == flushPage, $"asyncResult.page {asyncResult.page} should equal flushPage {flushPage}"); + var allocatorPage = pages[flushPage % BufferSize]; - // This is the in-memory buffer page to be written - var src = values[flushPage % BufferSize]; + // numBytesToWrite is calculated from start and end logical addresses, either for the full page or a subset of records (aligned to start and end of record boundaries), + // in the allocator page (including the objectId space for Overflow and Heap Objects). Note: "Aligned" in this discussion refers to sector (as opposed to record) alignment. - // We create a shadow copy of the page if we are under epoch protection. - // This copy ensures that object references are kept valid even if the original page is reclaimed. - // We suspend epoch during the actual flush as that can take a long time. - var epochProtected = false; - if (epoch.ThisInstanceProtected()) + // Initialize offsets into the allocator page based on full-page (including the page header), then override them if partial. + // asyncResult.fromAddress is either start of page or start of a record past the page header + var pageStart = GetLogicalAddressOfStartOfPage(asyncResult.page); + Debug.Assert(asyncResult.fromAddress - pageStart is >= PageHeader.Size or 0, $"fromAddress ({asyncResult.fromAddress}, offset {asyncResult.fromAddress - pageStart}) must be 0 or after the PageHeader"); + Debug.Assert(asyncResult.untilAddress - pageStart >= PageHeader.Size, $"untilAddress ({asyncResult.untilAddress}, offset {asyncResult.untilAddress - pageStart}) must be past PageHeader {flushPage}"); + int startOffset = (int)(asyncResult.fromAddress - pageStart), endOffset = startOffset + (int)numBytesToWrite; + if (asyncResult.partial) { - epochProtected = true; - src = new AllocatorRecord[values[flushPage % BufferSize].Length]; - Array.Copy(values[flushPage % BufferSize], src, values[flushPage % BufferSize].Length); - epoch.Suspend(); + // We're writing only a subset of the page. + endOffset = (int)(asyncResult.untilAddress - pageStart); + numBytesToWrite = (uint)(endOffset - startOffset); } - try - { - // Temporary storage to hold the image "template" we'll write to disk: It will have RecordInfos and object pointers that will be overwritten by addresses - // when writing to the main log (both object pointers and addresses are 8 bytes). - var buffer = bufferPool.Get((int)numBytesToWrite); - if (aligned_start < start && (KeyHasObjects() || ValueHasObjects())) + // Adjust so the first record on the page includes the page header. We've already asserted fromAddress such that startOffset is either 0 or >= PageHeader. + var logicalAddress = asyncResult.fromAddress; + var isFirstRecordOnPage = startOffset <= PageHeader.Size; + var firstRecordOffset = startOffset; + if (isFirstRecordOnPage) + { + if (startOffset == 0) { - // Do not read back the invalid header of page 0 - if ((flushPage > 0) || (start > GetFirstValidLogicalAddress(flushPage))) - { - // Get the overlapping HLOG from disk as we wrote it with object pointers previously. This avoids object reserialization - PageAsyncReadResult result = new() - { - handle = new CountdownEvent(1) - }; - device.ReadAsync(alignedDestinationAddress + (ulong)aligned_start, (IntPtr)buffer.aligned_pointer + aligned_start, - (uint)sectorSize, AsyncReadPageCallback, result); - result.handle.Wait(); - } - fixed (RecordInfo* pin = &src[0].info) - { - // Write all the RecordInfos on one operation. This also includes object pointers, but for valid records we will overwrite those below. - Debug.Assert(buffer.aligned_pointer + numBytesToWrite <= (byte*)Unsafe.AsPointer(ref buffer.buffer[0]) + buffer.buffer.Length); - - Buffer.MemoryCopy((void*)((long)Unsafe.AsPointer(ref src[0]) + start), buffer.aligned_pointer + start, - numBytesToWrite - start, numBytesToWrite - start); - } + // For the first record on the page the caller may have passed the address of the start of the page rather than the offset at the end of the PageHeader. + firstRecordOffset = PageHeader.Size; + logicalAddress += firstRecordOffset; } else { - fixed (RecordInfo* pin = &src[0].info) - { - // Write all the RecordInfos on one operation. This also includes object pointers, but for valid records we will overwrite those below. - Debug.Assert(buffer.aligned_pointer + numBytesToWrite <= (byte*)Unsafe.AsPointer(ref buffer.buffer[0]) + buffer.buffer.Length); - - Buffer.MemoryCopy((void*)((long)Unsafe.AsPointer(ref src[0]) + aligned_start), buffer.aligned_pointer + aligned_start, - numBytesToWrite - aligned_start, numBytesToWrite - aligned_start); - } + startOffset = 0; // Include the PageHeader + numBytesToWrite = (uint)(endOffset - startOffset); } + } - // In the main log, we write addresses to pages in the object log. This array saves the addresses of the key and/or value fields in 'buffer', - // which again is the image we're building from the 'values' "page" for this write. The "addresses into 'buffer'" are cast below to AddressInfo - // structures and stored in the sequence we'll write them: alternating series of key then value if both are object types, else keys or values only. - var addr = new List(); - asyncResult.freeBuffer1 = buffer; - - // Object keys and values are serialized into this MemoryStream. - MemoryStream ms = new(); - var keySerializer = KeyHasObjects() ? _storeFunctions.BeginSerializeKey(ms) : null; - var valueSerializer = ValueHasObjects() ? _storeFunctions.BeginSerializeValue(ms) : null; + var alignedStartOffset = RoundDown(startOffset, (int)device.SectorSize); + var startPadding = startOffset - alignedStartOffset; + var alignedBufferSize = RoundUp(startPadding + (int)numBytesToWrite, (int)device.SectorSize); - // Track the size to be written to the object log. - long endPosition = 0; + // We suspend epoch during the time-consuming actual flush. Note: The ShiftHeadAddress check to always remain below FlushedUntilAddress + // means the actual log page, inluding ObjectIdMap, will remain valid until we complete this partial flush. + var epochWasProtected = epoch.SuspendIfProtected(); - for (int i = start / RecordSize; i < end / RecordSize; i++) + // Do everything below here in the try{} to be sure the epoch is Resumed()d if we Suspended it. + SectorAlignedMemory srcBuffer = default; + try + { + // Create a local copy of the main-log page inline data. Space for ObjectIds and the ObjectLogPosition will be updated as we go + // (ObjectId space and a byte of the length-metadata space will combine for 5 bytes or 1TB of object size, which is our max). This does + // not change record sizes, so the logicalAddress space is unchanged. Also, we will not advance HeadAddress until this flush is complete + // and has updated FlushedUntilAddress, so we don't have to worry about the page being yanked out from underneath us (and Objects + // won't be disposed before we're done). TODO: Loop on successive subsets of the page's records to make this initial copy buffer smaller. + var objectIdMap = pages[flushPage % BufferSize].objectIdMap; + srcBuffer = bufferPool.Get(alignedBufferSize); + asyncResult.freeBuffer1 = srcBuffer; + + // Read back the first sector if the start is not aligned (this means we already wrote a partially-filled sector with ObjectLog fields set). + if (startPadding > 0) { - byte* recordPtr = buffer.aligned_pointer + i * RecordSize; + // TODO: This will potentially overwrite partial sectors if this is a partial flush; a workaround would be difficult. + // TODO: Cache the last sector flushed in readBuffers so we can avoid this Read. + PageAsyncReadResult result = new() { handle = new CountdownEvent(1) }; + device.ReadAsync(alignedMainLogFlushPageAddress + (ulong)alignedStartOffset, (IntPtr)srcBuffer.aligned_pointer, (uint)sectorSize, AsyncReadPageCallback, result); + result.handle.Wait(); + result.DisposeHandle(); + } - // Retrieve reference to record struct - ref var record = ref Unsafe.AsRef(recordPtr); - AddressInfo* key_address = null, value_address = null; + // Copy from the record start position (startOffset) in the main log page to the src buffer starting at its offset in the first sector (startPadding). + var allocatorPageSpan = new Span((byte*)pagePointers[flushPage % BufferSize] + startOffset, (int)numBytesToWrite); + allocatorPageSpan.CopyTo(srcBuffer.TotalValidSpan.Slice(startPadding)); + srcBuffer.available_bytes = (int)numBytesToWrite + startPadding; - // Zero out object reference addresses (AddressInfo) in the planned disk image - if (KeyHasObjects()) - { - key_address = GetKeyAddressInfo((long)recordPtr); - *key_address = default; - } - if (ValueHasObjects()) - { - value_address = GetValueAddressInfo((long)recordPtr); - *value_address = default; - } + // Overflow Keys and Values are written to, and Object values are serialized to, this Stream. + var logWriter = new ObjectLogWriter(device, flushBuffers, storeFunctions); + _ = logWriter.OnBeginPartialFlush(objectLogNextRecordStartPosition); - // Now fill in AddressInfo data for the valid records - if (!record.info.Invalid) - { - // Calculate the logical address of the 'values' page currently being written. - var address = (flushPage << LogPageSizeBits) + i * RecordSize; + // Include page header when calculating end address. + var endPhysicalAddress = (long)srcBuffer.GetValidPointer() + startPadding + numBytesToWrite; + var physicalAddress = (long)srcBuffer.GetValidPointer() + firstRecordOffset - alignedStartOffset; + while (physicalAddress < endPhysicalAddress) + { + // LogRecord is in the *copy of* the log buffer. We will update it (for objectIds) without affecting the actual record in the log. + var logRecord = new LogRecord(physicalAddress, objectIdMap); + + // Use allocatedSize here because that is what LogicalAddress is based on. + var logRecordSize = logRecord.GetInlineRecordSizes().allocatedSize; + // Do not write Invalid records. This includes IsNull records. + if (!logRecord.Info.Invalid) + { // Do not write v+1 records (e.g. during a checkpoint) - if (address < fuzzyStartLogicalAddress || !record.info.IsInNewVersion) + if (logicalAddress < fuzzyStartLogicalAddress || !logRecord.Info.IsInNewVersion) { - if (KeyHasObjects()) - { - long pos = ms.Position; - keySerializer.Serialize(ref src[i].key); - - // Store the key address into the 'buffer' AddressInfo image as an offset into 'ms'. - key_address->Address = pos; - key_address->Size = (int)(ms.Position - pos); - addr.Add((long)key_address); - endPosition = pos + key_address->Size; - } - - if (ValueHasObjects() && !record.info.Tombstone) + // Do not write objects for fully-inline records + if (logRecord.Info.RecordHasObjects) { - long pos = ms.Position; - try + var recordStartPosition = logWriter.GetNextRecordStartPosition(); + if (isFirstRecordOnPage) { - valueSerializer.Serialize(ref src[i].value); + ((PageHeader*)srcBuffer.GetValidPointer())->SetLowestObjectLogPosition(recordStartPosition); + isFirstRecordOnPage = false; } - catch (Exception ex) - { - logger?.LogError(ex, "Failed to serialize value"); - ms.Position = pos; - TValue defaultValue = default; - valueSerializer.Serialize(ref defaultValue); - } - - // Store the value address into the 'buffer' AddressInfo image as an offset into 'ms'. - value_address->Address = pos; - value_address->Size = (int)(ms.Position - pos); - addr.Add((long)value_address); - endPosition = pos + value_address->Size; + var valueObjectLength = logWriter.WriteRecordObjects(in logRecord); + logRecord.SetObjectLogRecordStartPositionAndLength(recordStartPosition, valueObjectLength); } } else { // Mark v+1 records as invalid to avoid deserializing them on recovery - record.info.SetInvalid(); + logRecord.InfoRef.SetInvalid(); } } - // If this record's serialized size surpassed ObjectBlockSize or it's the last record to be written, write to the object log. - if (endPosition > objectBlockSize || i == (end / RecordSize) - 1) - { - var memoryStreamActualLength = ms.Position; - var memoryStreamTotalLength = (int)endPosition; - endPosition = 0; - - if (KeyHasObjects()) - keySerializer.EndSerialize(); - if (ValueHasObjects()) - valueSerializer.EndSerialize(); - ms.Close(); - - // Get the total serialized length rounded up to sectorSize - var _alignedLength = (memoryStreamTotalLength + (sectorSize - 1)) & ~(sectorSize - 1); - - // Reserve the current address in the object log segment offsets for this chunk's write operation. - var _objAddr = Interlocked.Add(ref localSegmentOffsets[(long)(alignedDestinationAddress >> LogSegmentSizeBits) % SegmentBufferSize], _alignedLength) - _alignedLength; - - // Allocate the object-log buffer to build the image we'll write to disk, then copy to it from the memory stream. - SectorAlignedMemory _objBuffer = null; - if (memoryStreamTotalLength > 0) - { - _objBuffer = bufferPool.Get(memoryStreamTotalLength); - - fixed (void* src_ = ms.GetBuffer()) - Buffer.MemoryCopy(src_, _objBuffer.aligned_pointer, memoryStreamTotalLength, memoryStreamActualLength); - } - - // Each address we calculated above is now an offset to objAddr; convert to the actual address. - foreach (var address in addr) - ((AddressInfo*)address)->Address += _objAddr; - - // If we have not written all records, prepare for the next chunk of records to be written. - if (i < (end / RecordSize) - 1) - { - // Create a new MemoryStream for the next chunk of records to be written. - ms = new MemoryStream(); - if (KeyHasObjects()) - keySerializer.BeginSerialize(ms); - if (ValueHasObjects()) - valueSerializer.BeginSerialize(ms); - - // Reset address list for the next chunk of records to be written. - addr = new List(); - - // Write this chunk of records to the object log device. - asyncResult.done = new AutoResetEvent(false); - Debug.Assert(memoryStreamTotalLength > 0); - objlogDevice.WriteAsync( - (IntPtr)_objBuffer.aligned_pointer, - (int)(alignedDestinationAddress >> LogSegmentSizeBits), - (ulong)_objAddr, (uint)_alignedLength, AsyncFlushPartialObjectLogCallback, asyncResult); - - // Wait for write to complete before resuming next write - _ = asyncResult.done.WaitOne(); - _objBuffer.Return(); - } - else - { - // We have written all records in this 'values' "page". - if (memoryStreamTotalLength > 0) - { - // Increment the count because we need to write both page and object cache. - _ = Interlocked.Increment(ref asyncResult.count); - - asyncResult.freeBuffer2 = _objBuffer; - objlogDevice.WriteAsync( - (IntPtr)_objBuffer.aligned_pointer, - (int)(alignedDestinationAddress >> LogSegmentSizeBits), - (ulong)_objAddr, (uint)_alignedLength, callback, asyncResult); - } - } - } + logicalAddress += logRecordSize; // advance in main log + physicalAddress += logRecordSize; // advance in source buffer } + // We are done with the per-record objectlog flushes and we've updated the copy of the allocator page. Now write that updated page + // to the main log file. if (asyncResult.partial) { // We're writing only a subset of the page, so update our count of bytes to write. - var aligned_end = (int)(asyncResult.untilAddress - (asyncResult.page << LogPageSizeBits)); - aligned_end = (aligned_end + (sectorSize - 1)) & ~(sectorSize - 1); - numBytesToWrite = (uint)(aligned_end - aligned_start); + var aligned_end = (int)RoundUp(asyncResult.untilAddress - alignedStartOffset, (int)device.SectorSize); + numBytesToWrite = (uint)(aligned_end - alignedStartOffset); } - // Round up the number of byte to write to sector alignment. - var alignedNumBytesToWrite = (uint)((numBytesToWrite + (sectorSize - 1)) & ~(sectorSize - 1)); - - // Finally write the hlog page - device.WriteAsync((IntPtr)buffer.aligned_pointer + aligned_start, alignedDestinationAddress + (ulong)aligned_start, - alignedNumBytesToWrite, callback, asyncResult); + // Finally write the main log page as part of OnPartialFlushComplete. + // TODO: This will potentially overwrite partial sectors if this is a partial flush; a workaround would be difficult. + logWriter.OnPartialFlushComplete(srcBuffer.GetValidPointer(), alignedBufferSize, device, alignedMainLogFlushPageAddress + (uint)alignedStartOffset, + callback, asyncResult, out objectLogNextRecordStartPosition); } finally { - if (epochProtected) + if (epochWasProtected) epoch.Resume(); } -#endif // READ_WRITE } private void AsyncReadPageCallback(uint errorCode, uint numBytes, object context) @@ -663,327 +498,129 @@ private void AsyncReadPageCallback(uint errorCode, uint numBytes, object context _ = result.handle.Signal(); } - protected override void ReadAsync( - ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, - DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device, IDevice objlogDevice) + /// + /// This override of the base function reads Overflow keys or values, or Object values. + private protected override bool VerifyRecordFromDiskCallback(ref AsyncIOContext ctx, out long prevAddressToRead, out int prevLengthToRead) { -#if READ_WRITE - asyncResult.freeBuffer1 = bufferPool.Get((int)aligned_read_length); - asyncResult.freeBuffer1.required_bytes = (int)aligned_read_length; + // If this fails it is either too-short main-log record or a key mismatch. Let the top-level retry handle it. + if (!base.VerifyRecordFromDiskCallback(ref ctx, out prevAddressToRead, out prevLengthToRead)) + return false; - if (!(KeyHasObjects() || ValueHasObjects())) - { - device.ReadAsync(alignedSourceAddress, (IntPtr)asyncResult.freeBuffer1.aligned_pointer, - aligned_read_length, callback, asyncResult); - return; - } + // If the record is inline, we have no Overflow or Objects to retrieve. + ref var diskLogRecord = ref ctx.diskLogRecord; + if (diskLogRecord.Info.RecordIsInline) + return true; - asyncResult.callback = callback; + var startPosition = new ObjectLogFilePositionInfo(ctx.diskLogRecord.logRecord.GetObjectLogRecordStartPositionAndLengths(out var keyLength, out var valueLength), + objectLogNextRecordStartPosition.SegmentSizeBits); + var totalBytesToRead = (ulong)keyLength + valueLength; + + using var readBuffers = CreateCircularReadBuffers(objectLogDevice, logger); - if (objlogDevice == null) + var logReader = new ObjectLogReader(readBuffers, storeFunctions); + logReader.OnBeginReadRecords(startPosition, totalBytesToRead); + if (logReader.ReadRecordObjects(ref diskLogRecord.logRecord, ctx.request_key, transientObjectIdMap, startPosition.SegmentSizeBits)) { - Debug.Assert(objectLogDevice != null); - objlogDevice = objectLogDevice; + // Success; set the DiskLogRecord objectDisposer. We dispose the object here because it is read from the disk, unless we transfer it such as by CopyToTail. + ctx.diskLogRecord.objectDisposer = obj => storeFunctions.DisposeValueObject(obj, DisposeReason.DeserializedFromDisk); + + // Default the output arguments for reading a previous record. + prevAddressToRead = 0; + return true; } - asyncResult.objlogDevice = objlogDevice; - device.ReadAsync(alignedSourceAddress, (IntPtr)asyncResult.freeBuffer1.aligned_pointer, - aligned_read_length, AsyncReadPageWithObjectsCallback, asyncResult); -#endif // READ_WRITE + // If readBuffer.Read returned false it was due to an Overflow key mismatch or an Invalid record, so get the previous record. + prevAddressToRead = (*(RecordInfo*)ctx.record.GetValidPointer()).PreviousAddress; + return false; } - - /// - /// IOCompletion callback for page flush - /// - /// - /// - /// - private void AsyncFlushPartialObjectLogCallback(uint errorCode, uint numBytes, object context) + protected override void ReadAsync(CircularDiskReadBuffer readBuffers, ulong alignedSourceAddress, IntPtr destinationPtr, uint aligned_read_length, + DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device) { - if (errorCode != 0) - logger?.LogError($"{nameof(AsyncFlushPartialObjectLogCallback)} error: {{errorCode}}", errorCode); + //TODOnow("Add CancellationToken to the ReadAsync path"); - // Set the page status to flushed - var result = (PageAsyncFlushResult)context; - _ = result.done.Set(); + asyncResult.callback = callback; + asyncResult.destinationPtr = destinationPtr; + asyncResult.readBuffers = readBuffers; + asyncResult.maxPtr = aligned_read_length; + + device.ReadAsync(alignedSourceAddress, destinationPtr, aligned_read_length, AsyncReadPageWithObjectsCallback, asyncResult); } private void AsyncReadPageWithObjectsCallback(uint errorCode, uint numBytes, object context) { -#if READ_WRITE if (errorCode != 0) logger?.LogError($"{nameof(AsyncReadPageWithObjectsCallback)} error: {{errorCode}}", errorCode); var result = (PageAsyncReadResult)context; - - AllocatorRecord[] src; - - // We are reading into a frame - if (result.frame != null) + var pageStartAddress = (long)result.destinationPtr; + result.maxPtr = numBytes; + + // Iterate all records in range to determine how many bytes we need to read from objlog. + ObjectLogFilePositionInfo startPosition = new(); + ulong totalBytesToRead = 0; + var recordAddress = pageStartAddress + PageHeader.Size; + while (true) { - var frame = (GenericFrame)result.frame; - src = frame.GetPage(result.page % frame.frameSize); - } - else - src = values[result.page % BufferSize]; - - - // Deserialize all objects until untilptr - if (result.resumePtr < result.untilPtr) - { - MemoryStream ms = new(result.freeBuffer2.buffer); - ms.Seek(result.freeBuffer2.offset, SeekOrigin.Begin); - // We do not track deserialization size changes if we are deserializing to a frame - Deserialize(result.freeBuffer1.GetValidPointer(), result.resumePtr, result.untilPtr, src, ms, result.frame != null); - ms.Dispose(); - - result.freeBuffer2.Return(); - result.freeBuffer2 = null; - result.resumePtr = result.untilPtr; - } - - // If we have processed entire page, return - if (result.untilPtr >= result.maxPtr) - { - result.Free(); - - // Call the "real" page read callback - result.callback(errorCode, numBytes, context); - return; - } - - // We will now be able to process all records until (but not including) untilPtr - GetObjectInfo(result.freeBuffer1.GetValidPointer(), ref result.untilPtr, result.maxPtr, objectBlockSize, out long startptr, out long alignedLength); - - // Object log fragment should be aligned by construction - Debug.Assert(startptr % sectorSize == 0); - Debug.Assert(alignedLength % sectorSize == 0); - - if (alignedLength > int.MaxValue) - throw new TsavoriteException("Unable to read object page, total size greater than 2GB: " + alignedLength); - - var objBuffer = bufferPool.Get((int)alignedLength); - result.freeBuffer2 = objBuffer; - - // Request objects from objlog - result.objlogDevice.ReadAsync( - (int)((result.page - result.offset) >> (LogSegmentSizeBits - LogPageSizeBits)), - (ulong)startptr, - (IntPtr)objBuffer.aligned_pointer, (uint)alignedLength, AsyncReadPageWithObjectsCallback, result); -#endif // READ_WRITE - } - - /// - /// Invoked by users to obtain a record from disk. It uses sector aligned memory to read - /// the record efficiently into memory. - /// - /// - /// - /// - /// - /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) - { -#if READ_WRITE - var fileOffset = (ulong)(AlignedPageSizeBytes * (fromLogical >> LogPageSizeBits) + (fromLogical & PageSizeMask)); - var alignedFileOffset = (ulong)(((long)fileOffset / sectorSize) * sectorSize); - - var alignedReadLength = (uint)((long)fileOffset + numBytes - (long)alignedFileOffset); - alignedReadLength = (uint)((alignedReadLength + (sectorSize - 1)) & ~(sectorSize - 1)); - - var record = bufferPool.Get((int)alignedReadLength); - record.valid_offset = (int)(fileOffset - alignedFileOffset); - record.available_bytes = (int)(alignedReadLength - (fileOffset - alignedFileOffset)); - record.required_bytes = numBytes; - - var asyncResult = default(AsyncGetFromDiskResult); - asyncResult.context = context; - asyncResult.context.record = result; - asyncResult.context.objBuffer = record; - objectLogDevice.ReadAsync( - (int)(context.logicalAddress >> LogSegmentSizeBits), - alignedFileOffset, - (IntPtr)asyncResult.context.objBuffer.aligned_pointer, - alignedReadLength, - callback, - asyncResult); -#endif // READ_WRITE - } - - public struct AllocatorRecord // TODO remove - { - public RecordInfo info; - public byte[] key; - public byte[] value; - } - - #region Page handlers for objects - /// - /// Deseialize part of page from stream - /// - /// - /// From pointer - /// Until pointer - /// - /// Stream - public void Deserialize(byte* raw, long ptr, long untilptr, AllocatorRecord[] src, Stream stream) - { -#if READ_WRITE - long streamStartPos = stream.Position; - long start_addr = -1; - int start_offset = -1, end_offset = -1; + var logRecord = new LogRecord(recordAddress); - var keySerializer = KeyHasObjects() ? _storeFunctions.BeginDeserializeKey(stream) : null; - var valueSerializer = ValueHasObjects() ? _storeFunctions.BeginDeserializeValue(stream) : null; + // Use allocatedSize here because that is what LogicalAddress is based on. + var logRecordSize = logRecord.GetInlineRecordSizes().allocatedSize; + recordAddress += logRecordSize; - while (ptr < untilptr) - { - ref var record = ref Unsafe.AsRef(raw + ptr); - src[ptr / RecordSize].info = record.info; - if (start_offset == -1) - start_offset = (int)(ptr / RecordSize); - - end_offset = (int)(ptr / RecordSize) + 1; + if (logRecord.Info.Invalid || logRecord.Info.RecordIsInline) + continue; - if (!record.info.Invalid) + if (!startPosition.IsSet) { - if (KeyHasObjects()) - { - var key_addr = GetKeyAddressInfo((long)raw + ptr); - if (start_addr == -1) start_addr = key_addr->Address & ~((long)sectorSize - 1); - if (stream.Position != streamStartPos + key_addr->Address - start_addr) - _ = stream.Seek(streamStartPos + key_addr->Address - start_addr, SeekOrigin.Begin); - - keySerializer.Deserialize(out src[ptr / RecordSize].key); - } - else - src[ptr / RecordSize].key = record.key; - - if (!record.info.Tombstone) - { - if (ValueHasObjects()) - { - var value_addr = GetValueAddressInfo((long)raw + ptr); - if (start_addr == -1) start_addr = value_addr->Address & ~((long)sectorSize - 1); - if (stream.Position != streamStartPos + value_addr->Address - start_addr) - stream.Seek(streamStartPos + value_addr->Address - start_addr, SeekOrigin.Begin); - - valueSerializer.Deserialize(out src[ptr / RecordSize].value); - } - else - src[ptr / RecordSize].value = record.value; - } + startPosition = new(logRecord.GetObjectLogRecordStartPositionAndLengths(out _, out _), objectLogNextRecordStartPosition.SegmentSizeBits); + continue; } - ptr += GetRecordSize(ptr).Item2; - } - if (KeyHasObjects()) - keySerializer.EndDeserialize(); - if (ValueHasObjects()) - valueSerializer.EndDeserialize(); - - if (OnDeserializationObserver != null && start_offset != -1 && end_offset != -1 && !doNotObserve) - { - using var iter = new MemoryPageScanIterator(src, start_offset, end_offset, -1, RecordSize); - OnDeserializationObserver.OnNext(iter); - } -#endif // READ_WRITE - } - /// - /// Get location and range of object log addresses for specified log page - /// - /// - /// - /// - /// - /// - /// - public void GetObjectInfo(byte* raw, ref long ptr, long untilptr, int objectBlockSize, out long startptr, out long size) - { -#if READ_WRITE - var minObjAddress = long.MaxValue; - var maxObjAddress = long.MinValue; - var done = false; - - while (!done && (ptr < untilptr)) - { - ref var record = ref Unsafe.AsRef(raw + ptr); - - if (!record.info.Invalid) + // We have already incremented record address to get to the next record; if it is at or beyond the maxPtr, we have processed all records. + if (recordAddress >= pageStartAddress + result.maxPtr) { - if (KeyHasObjects()) - { - var key_addr = GetKeyAddressInfo((long)raw + ptr); - var addr = key_addr->Address; - - if (addr < minObjAddress) minObjAddress = addr; - addr += key_addr->Size; - if (addr > maxObjAddress) maxObjAddress = addr; - - // If object pointer is greater than kObjectSize from starting object pointer - if (minObjAddress != long.MaxValue && (addr - minObjAddress > objectBlockSize)) - done = true; - } - - - if (ValueHasObjects() && !record.info.Tombstone) - { - var value_addr = GetValueAddressInfo((long)raw + ptr); - var addr = value_addr->Address; - - if (addr < minObjAddress) minObjAddress = addr; - addr += value_addr->Size; - if (addr > maxObjAddress) maxObjAddress = addr; - - // If object pointer is greater than kObjectSize from starting object pointer - if (minObjAddress != long.MaxValue && (addr - minObjAddress > objectBlockSize)) - done = true; - } + ObjectLogFilePositionInfo endPosition = new(logRecord.GetObjectLogRecordStartPositionAndLengths(out var keyLength, out var valueLength), + objectLogNextRecordStartPosition.SegmentSizeBits); + endPosition.Advance((ulong)keyLength + valueLength); + totalBytesToRead = endPosition - startPosition; + break; } - ptr += GetRecordSize(ptr).allocatedSize; } - // Handle the case where no objects are to be written - if (minObjAddress == long.MaxValue && maxObjAddress == long.MinValue) + // The page may not have contained any records with objects + if (startPosition.IsSet) { - minObjAddress = 0; - maxObjAddress = 0; - } - - // Align start pointer for retrieval - minObjAddress &= ~((long)sectorSize - 1); - - // Align max address as well - maxObjAddress = (maxObjAddress + (sectorSize - 1)) & ~((long)sectorSize - 1); + // Iterate all records again to actually do the deserialization. + result.readBuffers.nextReadFilePosition = startPosition; + recordAddress = pageStartAddress + PageHeader.Size; + ReadOnlySpan noKey = default; + var logReader = new ObjectLogReader(result.readBuffers, storeFunctions); + logReader.OnBeginReadRecords(startPosition, totalBytesToRead); + + do + { + var logRecord = new LogRecord(recordAddress); - startptr = minObjAddress; - size = maxObjAddress - minObjAddress; -#else - startptr = 0; - size = 0; -#endif // READ_WRITE - } - #endregion + // Use allocatedSize here because that is what LogicalAddress is based on. + var logRecordSize = logRecord.GetInlineRecordSizes().allocatedSize; + recordAddress += logRecordSize; - public long[] GetSegmentOffsets() => null; + if (logRecord.Info.Invalid || logRecord.Info.RecordIsInline) + continue; - internal void PopulatePage(byte* src, int required_bytes, long destinationPage) - { -#if READ_WRITE - PopulatePage(src, required_bytes, ref values[destinationPage % BufferSize]); -#endif // READ_WRITE - } + // We don't need the DiskLogRecord here; we're either iterating (and will create it in GetNext()) or recovering + // (and do not need one; we're just populating the record ObjectIds and ObjectIdMap). objectLogDevice is in readBuffers. + _ = logReader.ReadRecordObjects(pageStartAddress, logRecordSize, noKey, transientObjectIdMap, startPosition.SegmentSizeBits, out _ /*diskLogRecord*/); - internal void PopulatePage(byte* src, int required_bytes, ref AllocatorRecord[] destinationPage) - { -#if READ_WRITE - fixed (RecordInfo* pin = &destinationPage[0].info) - { - Debug.Assert(required_bytes <= RecordSize * destinationPage.Length); - Buffer.MemoryCopy(src, Unsafe.AsPointer(ref destinationPage[0]), required_bytes, required_bytes); + // If the incremented record address is at or beyond the maxPtr, we have processed all records. + } while (recordAddress < pageStartAddress + result.maxPtr); } -#endif // READ_WRITE + + // Call the "real" page read callback + result.callback(errorCode, numBytes, context); + result.Free(); + return; } /// @@ -992,7 +629,7 @@ internal void PopulatePage(byte* src, int required_bytes, ref AllocatorRecord[] /// public override ITsavoriteScanIterator Scan(TsavoriteKV> store, long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, bool includeClosedRecords) - => new RecordScanIterator>(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeClosedRecords: includeClosedRecords); + => new ObjectScanIterator>(CreateCircularReadBuffers(), store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeClosedRecords: includeClosedRecords); /// /// Implementation for push-scanning Tsavorite log, called from LogAccessor @@ -1000,7 +637,7 @@ public override ITsavoriteScanIterator Scan(TsavoriteKV(TsavoriteKV> store, long beginAddress, long endAddress, ref TScanFunctions scanFunctions, DiskScanBufferingMode scanBufferingMode) { - using RecordScanIterator> iter = new(store, this, beginAddress, endAddress, epoch, scanBufferingMode, includeClosedRecords: false, logger: logger); + using ObjectScanIterator> iter = new(CreateCircularReadBuffers(), store, this, beginAddress, endAddress, epoch, scanBufferingMode, includeClosedRecords: false, logger: logger); return PushScanImpl(beginAddress, endAddress, ref scanFunctions, iter); } @@ -1011,9 +648,9 @@ internal override bool ScanCursor(TsavoriteKV> iter = new(store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, + using ObjectScanIterator> iter = new(CreateCircularReadBuffers(), store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: maxAddress < long.MaxValue, logger: logger); - return ScanLookup>>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, + return ScanLookup>>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress, resetCursor: resetCursor, includeTombstones: includeTombstones); } @@ -1023,7 +660,7 @@ internal override bool ScanCursor(TsavoriteKV(TsavoriteKV> store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) { - using RecordScanIterator> iter = new(store, this, beginAddress, epoch, logger: logger); + using ObjectScanIterator> iter = new(CreateCircularReadBuffers(), store, this, beginAddress, epoch, logger: logger); return IterateHashChain(store, key, beginAddress, ref scanFunctions, iter); } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs index ce26d5f5af3..fcade411e5a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs @@ -39,46 +39,70 @@ internal ObjectIdMap() public int Allocate() => freeSlots.TryPop(out var objectId) ? objectId : objectArray.Allocate(); + /// Reserve a slot, place the Overflow into it, and return the slot's ID. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public int AllocateAndSet(OverflowByteArray element) + { + var id = Allocate(); + Set(id, element); + return id; + } + + /// Reserve a slot, place the Object into it, and return the slot's ID. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public int AllocateAndSet(IHeapObject element) + { + var id = Allocate(); + Set(id, element); + return id; + } + /// Free a slot for reuse by another record on this page (e.g. when sending a record to the revivification freelist, or on a failed CAS, etc.). [MethodImpl(MethodImplOptions.AggressiveInlining)] public void Free(int objectId) { if (objectId != InvalidObjectId) { - Set(objectId, default); + objectArray.Set(objectId, default); freeSlots.Push(objectId); } } - /// Free a slot for reuse by another record on this page (e.g. when sending a record to the revivification freelist, or on a failed CAS, etc.). + /// Clear a specific slot of the array. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void Free(ref int objectIdRef) + public void Free(int objectId, Action disposer) { - var objectId = objectIdRef; - objectIdRef = InvalidObjectId; - Free(objectId); + if (objectId != InvalidObjectId) + { + if (disposer is not null) + { + var element = objectArray.Get(objectId); + disposer(Unsafe.As(ref element)); + } + objectArray.Set(objectId, default); + freeSlots.Push(objectId); + } } - /// Returns the slot's object. + /// Returns the slot's object as an IHeapObject. [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal object Get(int objectId) => objectArray.Get(objectId); + internal IHeapObject GetHeapObject(int objectId) => Unsafe.As(objectArray.Get(objectId)); - /// Returns the slot's object. + /// Returns the slot's object as an . [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Set(int objectId, object element) => objectArray.Set(objectId, element); + internal OverflowByteArray GetOverflowByteArray(int objectId) => new(Unsafe.As(objectArray.Get(objectId))); - /// Clear the array. + /// Sets the slot's object. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void Clear() => objectArray?.Clear(); // TODO reduce allocated chapter count also? + internal void Set(int objectId, IHeapObject element) => objectArray.Set(objectId, element); - /// Clear a specific slot of the array. + /// Sets the slot's object. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ClearAt(int objectId, Action disposer) - { - var element = Get(objectId); - disposer(Unsafe.As(ref element)); - Set(objectId, default); - } + internal void Set(int objectId, OverflowByteArray element) => objectArray.Set(objectId, element.Array); + + /// Clear the array. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Clear() => objectArray?.Clear(); // TODO reduce allocated chapter count also? /// public override string ToString() => $"tail: {(objectArray is not null ? objectArray.tail.ToString() : "")}"; diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectScanIterator.cs new file mode 100644 index 00000000000..7ab6c03f1be --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectScanIterator.cs @@ -0,0 +1,418 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Threading; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ + /// + /// Scan iterator for hybrid log + /// + internal sealed unsafe class ObjectScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + private readonly TsavoriteKV store; + private readonly AllocatorBase hlogBase; + private readonly BlittableFrame frame; + + private SectorAlignedMemory recordBuffer; + private readonly bool assumeInMemory; + + private DiskLogRecord diskLogRecord; + + /// + /// Constructor + /// + /// + /// The fully derived log implementation + /// + /// + /// + /// + /// + /// Epoch to use for protection; may be null if is true. + /// Provided address range is known by caller to be in memory, even if less than HeadAddress + /// + internal ObjectScanIterator(CircularDiskReadBuffer readBuffers, TsavoriteKV store, AllocatorBase hlogBase, + long beginAddress, long endAddress, LightEpoch epoch, DiskScanBufferingMode diskScanBufferingMode, + InMemoryScanBufferingMode memScanBufferingMode = InMemoryScanBufferingMode.NoBuffering, + bool includeClosedRecords = false, bool assumeInMemory = false, ILogger logger = null) + : base(readBuffers, beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, endAddress, diskScanBufferingMode, memScanBufferingMode, includeClosedRecords, epoch, hlogBase.LogPageSizeBits, logger: logger) + { + this.store = store; + this.hlogBase = hlogBase; + this.assumeInMemory = assumeInMemory; + if (frameSize > 0) + frame = new BlittableFrame(frameSize, hlogBase.PageSize, hlogBase.GetDeviceSectorSize()); + } + + /// + /// Constructor for use with tail-to-head push iteration of the passed key's record versions + /// + internal ObjectScanIterator(CircularDiskReadBuffer readBuffers, TsavoriteKV store, AllocatorBase hlogBase, + long beginAddress, LightEpoch epoch, ILogger logger = null) + : base(readBuffers, beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, hlogBase.GetTailAddress(), + DiskScanBufferingMode.SinglePageBuffering, InMemoryScanBufferingMode.NoBuffering, false, epoch, hlogBase.LogPageSizeBits, logger: logger) + { + this.store = store; + this.hlogBase = hlogBase; + assumeInMemory = false; + if (frameSize > 0) + frame = new BlittableFrame(frameSize, hlogBase.PageSize, hlogBase.GetDeviceSectorSize()); + } + + #region TODO Unify with SpanByteScanIterator + /// + public bool SnapCursorToLogicalAddress(ref long cursor) + { + Debug.Assert(currentAddress == -1, "SnapCursorToLogicalAddress must be called before GetNext()"); + Debug.Assert(nextAddress == cursor, "SnapCursorToLogicalAddress should have nextAddress == cursor"); + + if (!InitializeGetNextAndAcquireEpoch(out var stopAddress)) + return false; + try + { + if (!LoadPageIfNeeded(out var headAddress, out var currentPage, stopAddress)) + return false; + beginAddress = nextAddress = SnapToLogicalAddressBoundary(ref cursor, headAddress, currentPage); + } + catch + { + epoch?.Suspend(); + throw; + } + + return true; + } + + private bool InitializeGetNextAndAcquireEpoch(out long stopAddress) + { + if (diskLogRecord.IsSet) + { + hlogBase._wrapper.DisposeRecord(ref diskLogRecord, DisposeReason.DeserializedFromDisk); + diskLogRecord.Dispose(); + diskLogRecord = default; + } + diskLogRecord = default; + currentAddress = nextAddress; + stopAddress = endAddress < hlogBase.GetTailAddress() ? endAddress : hlogBase.GetTailAddress(); + if (currentAddress >= stopAddress) + return false; + + // Success; acquire the epoch. Caller will suspend the epoch as needed. + epoch?.Resume(); + return true; + } + + private bool LoadPageIfNeeded(out long headAddress, out long currentPage, long stopAddress) + { + headAddress = hlogBase.HeadAddress; + + if (currentAddress < hlogBase.BeginAddress && !assumeInMemory) + currentAddress = hlogBase.BeginAddress; + + // If currentAddress < headAddress and we're not buffering and not guaranteeing the records are in memory, fail. + if (frameSize == 0 && currentAddress < headAddress && !assumeInMemory) + { + // Caller will suspend the epoch. + throw new TsavoriteException("Iterator address is less than log HeadAddress in memory-scan mode"); + } + + currentPage = hlogBase.GetPage(currentAddress); + if (currentAddress < headAddress && !assumeInMemory) + _ = BufferAndLoad(currentAddress, currentPage, currentPage % frameSize, headAddress, stopAddress); + + // Success; keep the epoch held for GetNext (SnapCursorToLogicalAddress will Suspend()). + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal long SnapToLogicalAddressBoundary(ref long logicalAddress, long headAddress, long currentPage) + { + var offset = hlogBase.GetOffsetOnPage(logicalAddress); + + // Subtracting offset means this physicalAddress is at the start of the page. + var physicalAddress = GetPhysicalAddress(logicalAddress, headAddress, currentPage, offset) - offset; + long totalSizes = 0; + if (currentPage == 0) + { + if (logicalAddress < hlogBase.BeginAddress) + return logicalAddress = hlogBase.BeginAddress; + + // Bump past the FirstValidAddress offset + physicalAddress += hlogBase.BeginAddress; + totalSizes = (int)hlogBase.BeginAddress; + } + + while (totalSizes <= offset) + { + var allocatedSize = new LogRecord(physicalAddress).GetInlineRecordSizes().allocatedSize; + if (totalSizes + allocatedSize > offset) + break; + totalSizes += allocatedSize; + physicalAddress += allocatedSize; + } + + return logicalAddress += totalSizes - offset; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + long GetPhysicalAddress(long currentAddress, long headAddress, long currentPage, long offset) + => currentAddress >= headAddress || assumeInMemory + ? hlogBase.GetPhysicalAddress(currentAddress) + : frame.GetPhysicalAddress(currentPage, offset); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, long currentPage, long offset, out long allocatedSize) + { + var physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); + + // We are just getting sizes so no need for ObjectIdMap + var logRecord = new LogRecord(physicalAddress); + (var _, allocatedSize) = logRecord.GetInlineRecordSizes(); + return logRecord.physicalAddress; + } + #endregion TODO Unify with SpanByteScanIterator + + /// + /// Get next record in iterator + /// + /// True if record found, false if end of scan + public unsafe bool GetNext() + { + while (true) + { + if (!InitializeGetNextAndAcquireEpoch(out var stopAddress)) + return false; + + try + { + if (!LoadPageIfNeeded(out var headAddress, out var currentPage, stopAddress)) + return false; + + var offset = hlogBase.GetOffsetOnPage(currentAddress); + var physicalAddress = GetPhysicalAddressAndAllocatedSize(currentAddress, headAddress, currentPage, offset, out var allocatedSize); + var recordInfo = LogRecord.GetInfo(physicalAddress); + + // If record does not fit on page, skip to the next page. Offset should always be at least PageHeader.Size; if it's zero, it means + // our record size aligned perfectly with end of page, so we must move to the next page (skipping its PageHeader). + if (offset == 0 || offset + allocatedSize > hlogBase.PageSize) + { + var nextPage = hlogBase.GetPage(currentAddress); + nextAddress = hlogBase.GetFirstValidLogicalAddressOnPage(offset == 0 ? nextPage : nextPage + 1); + continue; + } + + nextAddress = currentAddress + allocatedSize; + + var skipOnScan = !includeClosedRecords && recordInfo.SkipOnScan; + if (skipOnScan || recordInfo.IsNull) + continue; + + if (currentAddress >= headAddress || assumeInMemory) + { + // TODO: for this PR we always buffer the in-memory records; pull iterators require it, and currently push iterators are implemented on top of pull. + // Copy the entire record into bufferPool memory so we don't have a ref to log data outside epoch protection. + OperationStackContext stackCtx = default; + try + { + // Lock to ensure no value tearing while copying to temp storage. + if (currentAddress >= headAddress && store is not null) + { + var logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress, physicalAddress); + store.LockForScan(ref stackCtx, logRecord.Key); + } + + if (recordBuffer == null) + recordBuffer = hlogBase.bufferPool.Get((int)allocatedSize); + else if (recordBuffer.AlignedTotalCapacity < (int)allocatedSize) + { + recordBuffer.Return(); + recordBuffer = hlogBase.bufferPool.Get((int)allocatedSize); + } + + // These objects are still alive in the log, so do not dispose the value object if any. + // Don't pass the recordBuffer to diskLogRecord; we reuse that here. + var remapPtr = recordBuffer.GetValidPointer(); + Buffer.MemoryCopy((byte*)physicalAddress, remapPtr, allocatedSize, allocatedSize); + var memoryLogRecord = hlogBase._wrapper.CreateRemappedLogRecordOverTransientMemory(currentAddress, (long)remapPtr); + diskLogRecord = new DiskLogRecord(in memoryLogRecord, obj => { }); + } + finally + { + if (stackCtx.recSrc.HasLock) + store.UnlockForScan(ref stackCtx); + } + } + else + { + // We advance a record at a time in the IO frame so set the diskLogRecord to the current frame offset and advance nextAddress. + // We dispose the object here because it is read from the disk, unless we transfer it such as by CopyToTail. + diskLogRecord = new(new LogRecord(physicalAddress, hlogBase._wrapper.TranssientObjectIdMap), + obj => store.storeFunctions.DisposeValueObject(obj, DisposeReason.DeserializedFromDisk)); + } + } + finally + { + // Success + epoch?.Suspend(); + } + + return true; + } + } + + /// + /// Get previous record and keep the epoch held while we call the user's scan functions + /// + /// True if record found, false if end of scan + bool IPushScanIterator.BeginGetPrevInMemory(ReadOnlySpan key, out LogRecord logRecord, out bool continueOnDisk) + { + while (true) + { + // "nextAddress" is reused as "previous address" for this operation. + currentAddress = nextAddress; + var headAddress = hlogBase.HeadAddress; + if (currentAddress < headAddress) + { + logRecord = default; + continueOnDisk = currentAddress >= hlogBase.BeginAddress; + return false; + } + + epoch?.Resume(); + + logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress); + nextAddress = logRecord.Info.PreviousAddress; + var skipOnScan = !includeClosedRecords && logRecord.Info.SkipOnScan; + if (skipOnScan || logRecord.Info.IsNull || !hlogBase.storeFunctions.KeysEqual(logRecord.Key, key)) + { + epoch?.Suspend(); + continue; + } + + // Success; defer epoch?.Suspend(); to EndGet + continueOnDisk = false; + return true; + } + } + + void IPushScanIterator.EndGetPrevInMemory() => epoch?.Suspend(); + + #region ISourceLogRecord + /// + public ref RecordInfo InfoRef => ref diskLogRecord.InfoRef; + /// + public RecordInfo Info => diskLogRecord.Info; + + /// + public byte RecordType => diskLogRecord.RecordType; + + /// + public byte Namespace => diskLogRecord.Namespace; + + /// + public ObjectIdMap ObjectIdMap => diskLogRecord.ObjectIdMap; + + /// + public bool IsSet => diskLogRecord.IsSet; + + /// + public ReadOnlySpan Key => diskLogRecord.Key; + + /// + public bool IsPinnedKey => diskLogRecord.IsPinnedKey; + + /// + public byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; + + /// + public OverflowByteArray KeyOverflow + { + get => diskLogRecord.KeyOverflow; + set => diskLogRecord.KeyOverflow = value; + } + + /// + public Span ValueSpan => diskLogRecord.ValueSpan; + + /// + public IHeapObject ValueObject => diskLogRecord.ValueObject; + + /// + public bool IsPinnedValue => diskLogRecord.IsPinnedValue; + + /// + public byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; + + /// + public OverflowByteArray ValueOverflow + { + get => diskLogRecord.ValueOverflow; + set => diskLogRecord.ValueOverflow = value; + } + + /// + public long ETag => diskLogRecord.ETag; + + /// + public long Expiration => diskLogRecord.Expiration; + + /// + public void ClearValueIfHeap(Action disposer) { } // Not relevant for "iterator as logrecord" + + /// + public bool IsMemoryLogRecord => false; + + /// + public unsafe ref LogRecord AsMemoryLogRecordRef() => throw new InvalidOperationException("Cannot cast a DiskLogRecord to a memory LogRecord."); + + /// + public bool IsDiskLogRecord => true; + + /// + public unsafe ref DiskLogRecord AsDiskLogRecordRef() => ref Unsafe.AsRef(in diskLogRecord); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordFieldInfo GetRecordFieldInfo() => diskLogRecord.GetRecordFieldInfo(); + #endregion // ISourceLogRecord + + /// + /// Dispose iterator + /// + public override void Dispose() + { + base.Dispose(); + if (diskLogRecord.IsSet) + hlogBase._wrapper.DisposeRecord(ref diskLogRecord, DisposeReason.DeserializedFromDisk); + recordBuffer?.Return(); + recordBuffer = null; + //TODOnow("Dispose objects in frame"); + frame?.Dispose(); + } + + internal override void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, + long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) + => hlogBase.AsyncReadPagesFromDeviceToFrame(readBuffers, readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice, cts); + + private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) + { + var result = (PageAsyncReadResult)context; + + if (errorCode == 0) + _ = result.handle?.Signal(); + else + { + logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); + result.cts?.Cancel(); + } + Interlocked.MemoryBarrier(); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskReadBuffer.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskReadBuffer.cs new file mode 100644 index 00000000000..f14ab35717c --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskReadBuffer.cs @@ -0,0 +1,208 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + + /// + /// This class drives object-deserialization reading from the disk. It has multiple buffers and reads buffers ahead of the current one + /// ahead while deserialization logic is running. + public class CircularDiskReadBuffer : IDisposable + { + internal readonly SectorAlignedBufferPool bufferPool; + internal readonly int bufferSize; + internal readonly IDevice objectLogDevice; + internal readonly ILogger logger; + + readonly DiskReadBuffer[] buffers; + int currentIndex; + + /// Device address to do the next read from (segment and offset); set at the start of a record by + /// and incremented with each buffer read; all of these should be aligned to sector size, so this address remains sector-aligned. + internal ObjectLogFilePositionInfo nextReadFilePosition; + + /// Track the remaining length to be read for one or more records for Object values, and we can also read some or all of Overflow values into the buffer. + ulong unreadLengthRemaining; + + internal CircularDiskReadBuffer(SectorAlignedBufferPool bufferPool, int bufferSize, int numBuffers, IDevice objectLogDevice, ILogger logger) + { + this.bufferPool = bufferPool; + this.bufferSize = bufferSize; + this.objectLogDevice = objectLogDevice; + this.logger = logger; + + buffers = new DiskReadBuffer[numBuffers]; + currentIndex = 0; + } + + internal DiskReadBuffer GetCurrentBuffer() => buffers[currentIndex]; + + int GetNextBufferIndex(int curIndex) + { + var index = curIndex + 1; + return index >= buffers.Length ? 0 : index; + } + + /// + /// Prepare the and local variables to read the next buffer (or as much of it as we need). This is called + /// by OnBeginReadRecords and when we are leaving a buffer with more data, to fill that buffer so it is available when we wrap around + /// to it again. For both of these, we do not have to worry that there is pending IO in the buffer. + /// + /// The index into of the that will do the reading + /// Start position on the page (relative to start of page) + private void ReadBuffer(int bufferIndex, int unalignedRecordStartPosition) + { + var buffer = buffers[bufferIndex]; + if (buffer is null) + { + buffer = new(bufferPool.Get(bufferSize), objectLogDevice, logger); + buffers[bufferIndex] = buffer; + } + else + { + Debug.Assert(buffer.countdownEvent.CurrentCount == 0, $"Unexpected countdownEvent.CurrentCount ({buffer.countdownEvent.CurrentCount}) when preparing to read into buffer"); + buffer.Initialize(); + } + + var alignedReadStartPosition = RoundDown(unalignedRecordStartPosition, (int)objectLogDevice.SectorSize); + var recordStartPosition = unalignedRecordStartPosition - alignedReadStartPosition; + + // See how much to read. We have two limits: the total size requested for this ReadAsync operation, and the segment size. + var unalignedReadLength = bufferSize - alignedReadStartPosition; + if ((ulong)unalignedReadLength > unreadLengthRemaining) + unalignedReadLength = (int)unreadLengthRemaining; + + Debug.Assert(IsAligned(nextReadFilePosition.Offset, (int)objectLogDevice.SectorSize), $"filePosition.Offset ({nextReadFilePosition.Offset}) is not sector-aligned"); + if (nextReadFilePosition.Offset + (ulong)unalignedReadLength > nextReadFilePosition.SegmentSize) + unalignedReadLength = (int)(nextReadFilePosition.SegmentSize - nextReadFilePosition.Offset); + + // We may not have had a sector-aligned amount of remaining unread data. + var alignedReadLength = RoundUp(unalignedReadLength, (int)objectLogDevice.SectorSize); + buffer.ReadFromDevice(nextReadFilePosition, recordStartPosition, (uint)alignedReadLength, ReadFromDeviceCallback); + + // Advance the filePosition. This used aligned read length so may advance it past end of record but that's OK because + // filePosition is for the "read buffer-sized chunks" logic while data transfer via Read() uses buffer.currentPosition. + nextReadFilePosition.Offset += (uint)alignedReadLength; + + Debug.Assert(nextReadFilePosition.Offset <= nextReadFilePosition.SegmentSize, $"filePosition.Offset ({nextReadFilePosition.Offset}) must be <= filePosition.SegmentSize ({nextReadFilePosition.SegmentSize})"); + if (nextReadFilePosition.Offset == nextReadFilePosition.SegmentSize) + nextReadFilePosition.AdvanceToNextSegment(); + + unreadLengthRemaining -= (uint)unalignedReadLength; + } + + /// + /// Called when one or more records are to be read via ReadAsync. + /// + /// The initial file position to read + /// The cumulative length of all object-log entries for the span of records to be read. We read ahead for all record + /// in the ReadAsync call. + internal void OnBeginReadRecords(ObjectLogFilePositionInfo startFilePosition, ulong totalLength) + { + Debug.Assert(totalLength > 0, "TotalLength cannot be 0"); + nextReadFilePosition = startFilePosition; + unreadLengthRemaining = totalLength; + + // Initialize all buffers + for (var ii = 0; ii < buffers.Length; ii++) + buffers[ii]?.Initialize(); + currentIndex = 0; + + // Do an initial read to fill the buffers, at least as much as we have. Again, totalLength applies to all records in the ReadAsync range, + // whether one or many. First align the initial read. recordStartPosition is the padding between rounded-down-to-align-readStart and recordStart. + var alignedReadPosition = RoundDown(nextReadFilePosition.Offset, (int)objectLogDevice.SectorSize); + var recordStartPosition = (int)(nextReadFilePosition.Offset - alignedReadPosition); + unreadLengthRemaining += (uint)recordStartPosition; + nextReadFilePosition.Offset -= (uint)recordStartPosition; + + // Load all the buffers as long as we have more unread data. Leave currentIndex at 0. + for (var ii = 0; ii < buffers.Length; ii++) + { + if (unreadLengthRemaining == 0) + break; + ReadBuffer(ii, recordStartPosition); + recordStartPosition = 0; // After the first read, subsequent reads start on an aligned address + } + } + + internal void OnBeginRecord(ObjectLogFilePositionInfo recordFilePosition) + { + var buffer = buffers[currentIndex] ?? throw new TsavoriteException($"Internal error in read buffer sequencing; empty buffer[{currentIndex}] encountered with unreadLengthRemaining {unreadLengthRemaining}"); + + // Because each partial flush ends with a sector-aligning write, we may have a record start position greater than our ongoing buffer.currentPosition + // incrementing. It should never be less. recordFilePosition is only guaranteed to be sector-aligned if it's the first record after a partial flush. + var bufferFilePosition = buffer.GetCurrentFilePosition(); + if (!buffer.HasData) + _ = buffer.WaitForDataAvailable(); + Debug.Assert(recordFilePosition.word >= bufferFilePosition.word, $"Record file position ({recordFilePosition}) should be >= ongoing position {bufferFilePosition}"); + Debug.Assert(recordFilePosition.SegmentId == bufferFilePosition.SegmentId, $"Record file segment ({recordFilePosition.SegmentId}) should == ongoing position {bufferFilePosition.SegmentId}"); + var increment = recordFilePosition - bufferFilePosition; + + Debug.Assert(increment < objectLogDevice.SectorSize, $"Increment {increment} is more than SectorSize ({objectLogDevice.SectorSize})"); + Debug.Assert(buffer.currentPosition + (int)increment < buffer.endPosition, $"Increment {increment} overflows buffer (curPos {buffer.currentPosition}, endPos {buffer.endPosition})"); + buffer.currentPosition += (int)increment; + } + + /// + /// Begin the deserialization process for a single record. + /// + internal void OnBeginDeserialize() + { + // Currently nothing + } + + /// + /// Move to the next buffer and see if it has data. + /// + /// The next buffer + /// + internal unsafe bool MoveToNextBuffer(out DiskReadBuffer nextBuffer) + { + // If we have more data to read, "backfill" this buffer with a read before departing it, else initialize it. + if (unreadLengthRemaining > 0) + ReadBuffer(currentIndex, unalignedRecordStartPosition: 0); + else + buffers[currentIndex].Initialize(); + + // Move to the next buffer and wait for any in-flight read to complete. If there is no pending IO and the buffer is + // empty, we are done with this read op. + currentIndex = GetNextBufferIndex(currentIndex); + nextBuffer = buffers[currentIndex]; + if (nextBuffer.WaitForDataAvailable()) + return true; + + Debug.Assert(unreadLengthRemaining == 0, $"unreadLengthRemaining ({unreadLengthRemaining}) was not 0 when WaitForDataAvailable returned false"); + return false; + } + + internal unsafe void ReadFromDeviceCallback(uint errorCode, uint numBytes, object context) + { + if (errorCode != 0) + logger?.LogError($"{nameof(ReadFromDeviceCallback)} error: {{errorCode}}", errorCode); + + // Finish setting up the buffer, and extract optionals if this was the last buffer. + var buffer = (DiskReadBuffer)context; + buffer.endPosition += (int)numBytes; + Debug.Assert(buffer.endPosition > buffer.currentPosition, $"buffer.endPosition ({buffer.endPosition}) must be >= buffer.currentPosition ({buffer.currentPosition})"); + + // Signal the buffer's event to indicate the data is available. + _ = buffer.countdownEvent.Signal(); + } + + public void Dispose() + { + for (var ii = 0; ii < buffers.Length; ii++) + buffers[ii]?.Dispose(); + } + + /// + public override string ToString() + => $"currIdx {currentIndex}; bufSize {bufferSize}; filePosition {nextReadFilePosition}; SecSize {(int)objectLogDevice.SectorSize}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskWriteBuffer.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskWriteBuffer.cs new file mode 100644 index 00000000000..49de0528769 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/CircularDiskWriteBuffer.cs @@ -0,0 +1,274 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using System.Threading; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + + /// + /// This class drives object-serialization writing to the disk. It is reused by multiple "partial flushes": ranges on a single page (rare) or + /// full pages. We create one instance for all ranges of a top-level Flush() call; each partial range will call , + /// do its flushes, and call . This reuse makes the most efficient use of the buffer allocations. It also + /// requires tracking of "in-flight" device writes at multiple levels: + /// + /// : A tracks how many in-flight device writes are associated with that buffer. + /// : A separate instance tracks total in-flight device writes + /// for each and pair, both at the level + /// and without buffer association, such as direct writes from pinned byte[] spans. This lets us call the main-page callback once the main-page + /// and all associated object log writes are complete. + /// + /// This also contains a counter of how many instances are active (i.e. how many partial flush + /// completion write batches are in-flight); when this hits 0, we can call . + /// + /// + /// + public class CircularDiskWriteBuffer : IDisposable + { + internal readonly SectorAlignedBufferPool bufferPool; + internal readonly int bufferSize; + internal readonly IDevice device; + internal readonly ILogger logger; + + DiskWriteBuffer[] buffers; + + /// Index of the current buffer + int currentIndex; + + /// Device address to write to (segment and offset); incremented with each buffer flush or out-of-line write by the caller; all of these should be aligned to sector size, + /// so this address remains sector-aligned. + internal ObjectLogFilePositionInfo filePosition; + + /// Countdown event for global count of all buffers and all direct writes. Also triggers the external callback of a partial-flush sequence. + /// This is passed to all disk-write operations; multiple pending flushes may be in-flight with the callback unset; when the final flush (which may be a buffer-span, a direct write, or the + /// final sector-aligning partial-flush completion flush), it allows the final pending flush to complete to know it *is* the final one and the callback can be called. + internal CountdownCallbackAndContext countdownCallbackAndContext; + + /// If true, has been called. Coordinates with to indicate when we can call . + bool disposed; + + /// Tracks the number of in-flight partial flush completion writes. Coordinates with to indicate when we can call . + long numInFlightWrites; + + internal CircularDiskWriteBuffer(SectorAlignedBufferPool bufferPool, int bufferSize, int numBuffers, IDevice device, ILogger logger) + { + this.bufferPool = bufferPool; + this.bufferSize = bufferSize; + this.device = device; + this.logger = logger; + + buffers = new DiskWriteBuffer[numBuffers]; + currentIndex = 0; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal DiskWriteBuffer GetCurrentBuffer() => buffers[currentIndex]; + + internal DiskWriteBuffer MoveToAndInitializeNextBuffer() + { + currentIndex = (currentIndex + 1) & (buffers.Length - 1); + return GetAndInitializeCurrentBuffer(); + } + + internal DiskWriteBuffer GetAndInitializeCurrentBuffer() + { + var buffer = GetCurrentBuffer(); + if (buffer is null) + { + buffer = new DiskWriteBuffer(bufferPool.Get(bufferSize), device, logger); + buffers[currentIndex] = buffer; + } + + // By this time the next device file write position has been updated, even if some of the preceding writes are still in-flight. + var endPosition = filePosition.SegmentSize - filePosition.Offset; + if (endPosition > (uint)bufferSize) + endPosition = (uint)bufferSize; + buffer.WaitUntilFreeAndInitialize((int)endPosition); + return buffer; + } + + internal ObjectLogFilePositionInfo GetNextRecordStartPosition() + { + var startFilePos = filePosition; + var buffer = GetCurrentBuffer(); + if (buffer is not null) + startFilePos.Offset += (uint)(buffer.currentPosition - buffer.flushedUntilPosition); + return startFilePos; + } + + /// Resets start positions for the next partial flush. + internal DiskWriteBuffer OnBeginPartialFlush(ObjectLogFilePositionInfo filePos) + { + // We start every partial flush with the first buffer, starting at position 0. + filePosition = filePos; + currentIndex = 0; + countdownCallbackAndContext = new(); + return GetAndInitializeCurrentBuffer(); + } + + /// Called when a Write is completed. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal unsafe void OnRecordComplete() + { + // Currently nothing to do. We do not do end-of-record alignment in the ObjectLog file. + } + + /// + /// Finish all the current partial flush, including flushing any as-yet-unflushed data in the current buffer then calling the caller's callbacks + /// so flushedUntilAddresses can be updated. When this function exits, there will be IOs in flight. + /// + /// This write to the device is sector-aligned, which means the next fragment will probably rewrite the sector, since the currentPosition is probably + /// somewhere in the middle of the sector. + /// Starting pointer of the main log page span to write + /// Length of the main log page span to write + /// The main log device to write to + /// The offset in the main log to write at + /// Callback sent to the initial Flush() command. Called when we are done with this partial flush operation. + /// It usually signals the event so the caller knows the flush is complete and it can continue. + /// Context sent to . + /// The ending file position after the partial flush is complete + internal unsafe void OnPartialFlushComplete(byte* mainLogPageSpanPtr, int mainLogPageSpanLength, IDevice mainLogDevice, ulong alignedMainLogFlushAddress, + DeviceIOCompletionCallback externalCallback, object externalContext, out ObjectLogFilePositionInfo endObjectLogFilePosition) + { + // Lock this with a reference until we have set the callback and issue the write. This callback is for the main log page write, and + // when the countdownCallbackAndContext.Decrement hits 0 again, we're done with this partial flush range and will call the external callback. + countdownCallbackAndContext.Increment(); + countdownCallbackAndContext.Set(externalCallback, externalContext, (uint)mainLogPageSpanLength); + + // Issue the last ObjectLog write for this partial flush. + var buffer = GetCurrentBuffer(); + Debug.Assert(IsAligned(alignedMainLogFlushAddress, (int)device.SectorSize), "mainLogAlignedDeviceOffset is not aligned to sector size"); + Debug.Assert(IsAligned(buffer.flushedUntilPosition, (int)device.SectorSize), $"flushedUntilOffset {buffer.flushedUntilPosition} is not sector-aligned"); + Debug.Assert(buffer.currentPosition >= buffer.flushedUntilPosition, $"buffer.currentPosition {buffer.currentPosition} must be >= buffer.flushedUntilPosition {buffer.flushedUntilPosition}"); + + if (buffer.currentPosition > buffer.flushedUntilPosition) + { + // We have something to flush. First ensure sector-alignment of the flush; we'll "waste" some space to do so. This is necessary to avoid rewriting sectors, + // which can be a problem for some devices due to inefficiencies in rewriting or inability to back up (or both). + var sectorEnd = RoundUp(buffer.currentPosition, (int)device.SectorSize); + if (sectorEnd > buffer.currentPosition) + { + // Prepare to flush the final piece to disk by zero-initializing the sector-alignment padding. + new Span(buffer.memory.GetValidPointer() + buffer.currentPosition, sectorEnd - buffer.currentPosition).Clear(); + buffer.currentPosition = sectorEnd; + } + + // Now write the buffer to the device. + _ = Interlocked.Increment(ref numInFlightWrites); + buffer.FlushToDevice(ref filePosition, FlushToDeviceCallback, CreateDiskWriteCallbackContext()); + } + + // Update the object log file position for the caller. + endObjectLogFilePosition = filePosition; + + // Write the main log page to the mainLogDevice. + FlushToMainLogDevice(mainLogPageSpanPtr, mainLogPageSpanLength, mainLogDevice, alignedMainLogFlushAddress, CreateDiskWriteCallbackContext()); + + // We added a count to countdownCallbackAndContext at the start, and the callback state creation also added a count. Remove the one we added at the start. + countdownCallbackAndContext.Decrement(); + } + + internal DiskWriteCallbackContext CreateDiskWriteCallbackContext() => new(countdownCallbackAndContext); + internal DiskWriteCallbackContext CreateDiskWriteCallbackContext(RefCountedPinnedGCHandle refGcHandle) => new(countdownCallbackAndContext, refGcHandle); + internal DiskWriteCallbackContext CreateDiskWriteCallbackContext(GCHandle gcHandle) => new(countdownCallbackAndContext, gcHandle); + + /// Flush the current buffer. If we are in an operation that filled previous buffers, those will have been flushed already by earlier calls. + internal void FlushCurrentBuffer() + { + var buffer = GetCurrentBuffer(); + var writeCallbackContext = CreateDiskWriteCallbackContext(); + _ = Interlocked.Increment(ref numInFlightWrites); + buffer.FlushToDevice(ref filePosition, FlushToDeviceCallback, writeCallbackContext); + } + + /// Flush to disk for a span that is not associated with a particular buffer, such as fully-interior spans of a large overflow key or value. + internal unsafe void FlushToDevice(byte* spanPtr, int spanLength, DiskWriteCallbackContext writeCallbackContext) + { + Debug.Assert(IsAligned(spanLength, (int)device.SectorSize), "Span is not aligned to sector size"); + + _ = Interlocked.Increment(ref numInFlightWrites); + device.WriteAsync((IntPtr)spanPtr, filePosition.SegmentId, filePosition.Offset, (uint)spanLength, FlushToDeviceCallback, writeCallbackContext); + filePosition.Offset += (uint)spanLength; + } + + /// Flush a main-log page span to the main log device. This lets us coordinate the callbacks to be called on the last write, regardless of whether + /// that write is to main or object log. + internal unsafe void FlushToMainLogDevice(byte* spanPtr, int spanLength, IDevice mainLogDevice, ulong alignedMainLogFlushAddress, DiskWriteCallbackContext writeCallbackContext) + { + Debug.Assert(IsAligned(spanLength, (int)device.SectorSize), "Span is not aligned to sector size"); + Debug.Assert(IsAligned(alignedMainLogFlushAddress, (int)device.SectorSize), "mainLogAlignedDeviceOffset is not aligned to sector size"); + + _ = Interlocked.Increment(ref numInFlightWrites); + mainLogDevice.WriteAsync((IntPtr)spanPtr, alignedMainLogFlushAddress, (uint)spanLength, FlushToDeviceCallback, writeCallbackContext); + } + + private void FlushToDeviceCallback(uint errorCode, uint numBytes, object context) + { + if (errorCode != 0) + logger?.LogError($"{nameof(FlushToDeviceCallback)} error: {{errorCode}}", errorCode); + + // Try to signal the event; if we have finished the last write for this buffer, the count will hit zero and Set the event so any Waits we do on it will succeed. + // We don't wait on the result of individual device writes; we may wait due to a call (e.g. FlushAndEvict()) with a "wait" parameter set to true. + var writeCallbackContext = (DiskWriteCallbackContext)context; + + // If this returns 0 we have finished all in-flight writes for the writeCallbackContext.countdownCallbackAndContext instance, but there may be more instances + // active even if we have been disposed, so adjust and check the global count, and if *that* is zero, check the disposed state (being disposed ensures that no + // further partial flush ranges will be sent). + _ = Interlocked.Decrement(ref numInFlightWrites); + if (writeCallbackContext.Release() == 0 && numInFlightWrites == 0 && disposed) + ClearBuffers(); + } + + /// + public void Dispose() + { + // If we are here, then we have returned from the partial-flush loop and will not be incrementing numInFlightRangeBatches again, so if it is 0 + // we are done and can free the buffers. + disposed = true; + if (numInFlightWrites == 0) + ClearBuffers(); + } + + private void ClearBuffers() + { + // We should have no data to flush--the last partial flush should have ended with PartialFlushComplete which flushes the last of the data for that flush fragment, + // and we wait for that to finish before calling the caller's callback. However, we may have to wait for flushed data to complete; this may be from either the + // just-completed partial-flush range, or even from the range before that if the most recent range did not use all buffers; at the time this is called there may + // be one or more in-flight countdownCallbackAndContexts. So we just wait. + + // Atomic swap to avoid clearing twice, because the 'disposed' testing isn't atomic. + var localBuffers = Interlocked.Exchange(ref buffers, null); + if (localBuffers == null) + return; + + for (var ii = 0; ii < localBuffers.Length; ii++) + { + ref var buffer = ref localBuffers[ii]; + if (buffer is not null) + { + buffer.Wait(); + buffer.Dispose(); + buffer = null; + } + } + buffers = localBuffers; + } + + /// + public override string ToString() + { + var result = $"currIdx {currentIndex}; bufSize {bufferSize}; filePos {filePosition}, SecSize {(int)device.SectorSize}"; + var buffer = GetCurrentBuffer(); + if (buffer is not null) + result += $"; currBuf: [{buffer}]"; + return result; + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskReadBuffer.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskReadBuffer.cs new file mode 100644 index 00000000000..d7d711748ce --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskReadBuffer.cs @@ -0,0 +1,117 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Threading; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ + internal sealed unsafe class DiskReadBuffer : IDisposable + { + internal readonly IDevice device; + internal readonly ILogger logger; + + /// Signals when reads are complete. Allows multiple reads of buffer subsections to be enqueed, and uses a ManualResetEvent + /// so it remains signaled until Reset() is called (currently only one read for a single span of the buffer is done). + internal CountdownEvent countdownEvent; + + /// The buffer to read (part of) the page image into. + internal SectorAlignedMemory memory; + + /// + /// This is the initialization value for ; it means there is no data available for this buffer and no + /// in-flight read (we issue reads ahead of the buffer-array traversal, so this means that by the time we got to this buffer all + /// the data had already been read. + /// + const int NoPosition = -1; + + /// Current read position (we do not support write in this buffer). This class only supports Read and no Seek, + /// so currentPosition is always where will read from next. + /// This will be either 0 or greater than or equal to . + internal int currentPosition; + + /// Non-inclusive last position in this buffer; the number of byte read. If equals this, then we are out of space and + /// must move to the next buffer. + internal int endPosition; + + /// + /// The starting position in the file that we read this buffer from. + /// + internal ObjectLogFilePositionInfo startFilePosition; + + internal int AvailableLength => endPosition - currentPosition; + + internal ReadOnlySpan AvailableSpan => new(memory.GetValidPointer() + currentPosition, endPosition - currentPosition); + + internal DiskReadBuffer(SectorAlignedMemory memory, IDevice device, ILogger logger) + { + this.memory = memory; + countdownEvent = new CountdownEvent(0); // Start with 0; we'll increment at the time of read + this.device = device; + this.logger = logger; + Initialize(); + } + + internal void Initialize() + { + currentPosition = endPosition = NoPosition; + } + + internal ReadOnlySpan GetTailSpan(int start) => new(memory.GetValidPointer() + start, currentPosition - start); + + /// + /// Read the first chunk of an Object deserialization from the device. + /// + /// Sector-aligned position in the device + /// Start position in the buffer (relative to start of buffer) + /// Number of bytes to read + /// The callback. + internal void ReadFromDevice(ObjectLogFilePositionInfo filePosition, int startPosition, uint alignedReadLength, DeviceIOCompletionCallback callback) + { + IncrementOrResetCountdown(ref countdownEvent); + startFilePosition = filePosition; + + currentPosition = startPosition; + endPosition = 0; + device.ReadAsync(filePosition.SegmentId, filePosition.Offset, (IntPtr)memory.aligned_pointer, (uint)alignedReadLength, callback, context: this); + } + + internal static void IncrementOrResetCountdown(ref CountdownEvent countdownEvent) => DiskWriteBuffer.IncrementOrResetCountdown(ref countdownEvent); + + internal bool HasData => endPosition > 0; + + internal bool WaitForDataAvailable() + { + // Because we have issued reads ahead of the buffer wrap, if the currentPosition is NoPosition, we're done. + if (currentPosition == NoPosition) + return false; + if (!HasData) + countdownEvent.Wait(); + return true; + } + + internal ObjectLogFilePositionInfo GetCurrentFilePosition() + { + var bufferFilePos = startFilePosition; + bufferFilePos.Offset += (uint)currentPosition; + + // We only read from one segment into one buffer, so we should never exceed the segment size with this increment. + Debug.Assert(bufferFilePos.Offset < bufferFilePos.SegmentSize, $"Incremented bufferFilePos.Offset {bufferFilePos.Offset} should be < bufferFilePos.SegmentSize {bufferFilePos.SegmentSize}"); + return bufferFilePos; + } + + public void Dispose() + { + memory?.Return(); + memory = null; + countdownEvent.Dispose(); + countdownEvent = null; + } + + /// + public override string ToString() + => $"currPos {currentPosition}; endPos {endPosition}; avLen {AvailableLength}; countDown {countdownEvent?.CurrentCount}; buf: {memory}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskWriteBuffer.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskWriteBuffer.cs new file mode 100644 index 00000000000..7f4086313e3 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/DiskWriteBuffer.cs @@ -0,0 +1,126 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Threading; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + + internal sealed unsafe class DiskWriteBuffer : IDisposable + { + /// Signals when writes are complete. Allows multiple writes of buffer subsections to be enqueed, and uses a ManualResetEvent + /// so it remains signaled until Reset() is called. + CountdownEvent countdownEvent; + + /// The buffer to build the page image for writing. + internal SectorAlignedMemory memory; + + /// Current write position (we do not support read in this buffer). This class only supports Write and no Seek, + /// so currentPosition equals the current length. Relevant for object serialization only; reset to 0 at start of DoSerialize(). + /// is the current length, since it is *past* the last byte copied to the buffer. + internal int currentPosition; + + /// Last position flushed to in this buffer (i.e. 0->flushedUntilPosition have been flushed). This allows using the buffer for + /// multiple small writes sandwiched around large internal direct writes from overflow. + /// If is less than then there is unflushed data in the buffer. + internal int flushedUntilPosition; + + /// The end position of the buffer. Usually the size of the buffer, but may be less if we're at the end of a segment. + /// This should always be sector-aligned, because we start the partial flush sector-aligned and this is either a bufferSize past + /// the previous buffer or a sector-aligned distance from the end of the segment; the latter may be less than the end of the buffer + /// due to directly writing internal spans for Keys and Values that are less than a full buffer size. + internal int endPosition; + + internal int RemainingCapacity => endPosition - currentPosition; // DiskPageHeader.Size is included in currentPosition + + /// The remaining space in the buffer, from to . + internal Span RemainingSpan => new(memory.GetValidPointer(), RemainingCapacity); + + internal readonly IDevice device; + internal readonly ILogger logger; + + internal DiskWriteBuffer(SectorAlignedMemory memory, IDevice device, ILogger logger) + { + this.memory = memory; + this.device = device; + this.logger = logger; + } + + internal void WaitUntilFreeAndInitialize(int endPosition) + { + Debug.Assert(IsAligned(endPosition, (int)device.SectorSize), $"endPosition {endPosition} is not sector-aligned"); + + // First wait for any pending write in this buffer to complete. If this is our first time in this buffer there won't be a CountdownEvent yet; + // we defer that because we may not need all the buffers in the circular buffer. + countdownEvent?.Wait(); + + // Initialize fields. + this.endPosition = endPosition; + memory.valid_offset = 0; + currentPosition = 0; + flushedUntilPosition = 0; + } + + internal static CountdownEvent IncrementOrResetCountdown(ref CountdownEvent countdownEvent) + { + if (countdownEvent is null) + countdownEvent = new(1); + else if (!countdownEvent.TryAddCount(1)) + { + // This means we've enqueued one or more earlier writes which have completed. + // First wait to be sure the callback has signaled the contained event, then Reset the event with a new count. + countdownEvent.Wait(); // This should usually be immediate + countdownEvent.Reset(1); + } + return countdownEvent; + } + + internal void FlushToDevice(ref ObjectLogFilePositionInfo filePosition, DeviceIOCompletionCallback callback, DiskWriteCallbackContext pageWriteCallbackContext) + { + Debug.Assert(currentPosition <= endPosition, $"currentPosition ({currentPosition}) cannot exceed endPosition ({endPosition})"); + + // We are flushing the buffer. currentPosition must already be sector-aligned; either it is at endPosition (which is always sector-aligned), + // which is the normal "buffer is full so flush it" handling, or it is less than endPosition which means it is called from one of: + // a. OnPartialFlushComplete, in which case the caller has sector-aligned it before calling this + // b. OverflowByteArray sector-aligning writes at the beginning or end, which means we copied a sector-aligned number of bytes to the buffer. + Debug.Assert(IsAligned(currentPosition, (int)device.SectorSize), $"currentPosition ({currentPosition}) is not sector-aligned"); + Debug.Assert(IsAligned(filePosition.Offset, (int)device.SectorSize), $"Starting file flush position ({filePosition}) is not sector-aligned"); + pageWriteCallbackContext.SetBufferCountdownEvent(IncrementOrResetCountdown(ref countdownEvent)); + + var flushLength = (uint)(currentPosition - flushedUntilPosition); + Debug.Assert(IsAligned(flushLength, (int)device.SectorSize), $"flushLength {flushLength} is not sector-aligned"); + Debug.Assert(flushLength <= filePosition.RemainingSize, $"flushLength ({flushLength}) cannot be greater than filePosition.RemainingSize ({filePosition.RemainingSize})"); + + var spanPtr = memory.GetValidPointer() + flushedUntilPosition; + device.WriteAsync((IntPtr)spanPtr, filePosition.SegmentId, filePosition.Offset, flushLength, callback, pageWriteCallbackContext); + flushedUntilPosition = currentPosition; + + // This does not use .Advance() because we are already checking boundary conditions and calling .AdvanceToNextSegment() in ObjectLogWriter. + filePosition.Offset += flushLength; + } + + internal void Wait() => countdownEvent?.Wait(); + + public void Dispose() + { + memory?.Return(); + memory = null; + + Debug.Assert(countdownEvent is null || countdownEvent.CurrentCount == 0, $"Unexpected count ({countdownEvent.CurrentCount}) remains"); + countdownEvent?.Dispose(); + countdownEvent = null; + } + + /// + public override string ToString() + { + var countdownString = countdownEvent?.CurrentCount.ToString() ?? "null"; + return $"currPos {currentPosition}; endPos {endPosition}; remCap {RemainingCapacity}; flushedUntilPos {flushedUntilPosition}; countDown {countdownString}; buf: {memory}"; + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/IStreamBuffer.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/IStreamBuffer.cs new file mode 100644 index 00000000000..50166b82f30 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/IStreamBuffer.cs @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; + +namespace Tsavorite.core +{ + /// + /// This interface abstracts the process of writing a full buffer to storage or network, or reading (up to) a certain number of bytes. + /// + public interface IStreamBuffer : IDisposable + { + /// Indicates that the value is continued in the next chunk, after the current length. + internal const int ValueChunkContinuationBit = 1 << 31; + + /// Indicates that the value is completed in the current chunk (there is no next chunk). + internal const int NoValueChunkContinuationBit = 0; + + /// + /// The size after which a key (should be rare) or value is handled as an out-of-line allocation when reading from disk. + /// Must be less than DiskReadBufferSize / 2 and a sector multiple, so it is separate from (and less than) . + /// + internal const int DiskReadForceOverflowSize = 1 * 1024 * 1024; + + /// The size of the buffer used for writing data to and reading it from the disk. Must be a sector multiple. + internal const int BufferSize = 1 << LogSettings.kMinObjectLogSegmentSizeBits; + + /// Initial IO size to read. + internal static int InitialIOSize => Environment.SystemPageSize; + + /// + /// We use these buffers for only read or only write operations, never both at the same time. + /// + bool IsForWrite { get; } + + /// + /// Write a full buffer to storage or network and reset the buffer to the starting position. + /// + /// Optional cancellation token + void FlushAndReset(CancellationToken cancellationToken = default); + + /// + /// Write span of bytes to the storage or network buffer. Actual flushing (e.g. to disk) is done as needed.. + /// + /// The data span to write to the device. + /// Optional cancellation token + /// This implements the standard Stream functionality, called from the Value Serializer + void Write(ReadOnlySpan data, CancellationToken cancellationToken = default); + + /// + /// Read more bytes from the disk or network, up to , and store in the buffer. It may not read all bytes + /// depending on the internal buffer management. + /// + /// The span to receive data from the device + /// Optional cancellation token + /// This implements the standard Stream functionality, called from the Value Serializer + /// The number of bytes read into , which may be less than . + int Read(Span destinationSpan, CancellationToken cancellationToken = default); + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogFilePositionInfo.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogFilePositionInfo.cs new file mode 100644 index 00000000000..342bf43ca93 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogFilePositionInfo.cs @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; +using System.Runtime.CompilerServices; + +namespace Tsavorite.core +{ + /// + /// Represents the information about the segment and offset of a location within the object log file. + /// + internal struct ObjectLogFilePositionInfo + { + /// Object log segment size bits + internal int SegmentSizeBits; + + /// The word containing the data. + internal ulong word; + + internal readonly bool IsSet => SegmentSizeBits != 0; + + internal ObjectLogFilePositionInfo(int segSizeBits) + { + SegmentSizeBits = segSizeBits; + } + + internal ObjectLogFilePositionInfo(ulong word, int segSizeBits) + { + SegmentSizeBits = segSizeBits; + this.word = word; + } + + /// The offset within the current . + public ulong Offset + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + readonly get + { + var mask = (ulong)(1L << SegmentSizeBits) - 1L; + return word & mask; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + set + { + var mask = (ulong)(1L << SegmentSizeBits) - 1L; + Debug.Assert((value & ~mask) <= SegmentSize, $"New Offset ({(value & ~mask)}) exceeds max segment size"); + word = (word & ~mask) | (value & mask); + } + } + + /// The current segment in the file. + public int SegmentId + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + readonly get + { + var mask = (ulong)((1L << ((sizeof(long) * 8) - SegmentSizeBits)) - 1L); + return (int)((word >> SegmentSizeBits) & mask); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + set + { + var mask = (ulong)((1L << ((sizeof(long) * 8) - SegmentSizeBits)) - 1L); + word = (word & (mask << SegmentSizeBits)) | (((ulong)value & mask) << SegmentSizeBits); + } + } + + public void Advance(ulong size) + { + var remaining = SegmentSize - Offset; + if (size < remaining) + { + Offset += size; + return; + } + + // Note: If size == remaining, we advance to the start of the next segment. + size -= remaining; + SegmentId += (int)(size / SegmentSize) + 1; + Offset += size & (SegmentSize - 1); + } + + public void AdvanceToNextSegment() + { + SegmentId++; + Offset = 0; + } + + public static ulong operator-(ObjectLogFilePositionInfo left, ObjectLogFilePositionInfo right) + { + Debug.Assert(left.SegmentSizeBits == right.SegmentSizeBits, "Segment size bits must match to compute distance"); + Debug.Assert(left.word >= right.word, "comparison position must be greater"); + var segmentDiff = (ulong)(left.SegmentId - right.SegmentId); + if (segmentDiff == 0) + return left.Offset - right.Offset; + return (segmentDiff - 1) * left.SegmentSize + (left.SegmentSize - right.Offset) + left.Offset; + } + + public readonly ulong SegmentSize => 1UL << SegmentSizeBits; + + public readonly ulong RemainingSize => SegmentSize - Offset; + + /// + public override readonly string ToString() => $"Segment {SegmentId}, Offset {Offset:N0}, Bits {SegmentSizeBits}, SegSize {SegmentSize:N0}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogReader.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogReader.cs new file mode 100644 index 00000000000..be50e138c02 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogReader.cs @@ -0,0 +1,230 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Threading; + +namespace Tsavorite.core +{ + /// + /// The class that manages IO read of ObjectAllocator records. It manages the read buffer at two levels: + /// + /// At the higher level, called by IO routines, it manages the overall record reading, including issuing additional reads as the buffer is drained. + /// At the lower level, it provides the stream for the valueObjectSerializer, which is called via Deserialize() by the higher level. + /// + /// + internal unsafe partial class ObjectLogReader : IStreamBuffer + where TStoreFunctions : IStoreFunctions + { + IObjectSerializer valueObjectSerializer; + PinnedMemoryStream> pinnedMemoryStream; + + /// The current record header; used for chunks to identify when they need to extract the optionals after the final chunk. + internal RecordInfo recordInfo; + + /// The circular buffer we cycle through for object-log deserialization. + readonly CircularDiskReadBuffer readBuffers; + + /// The implementation to use + internal readonly TStoreFunctions storeFunctions; + + /// If true, we are in the Deserialize call. If not we ignore things like etc. + bool inDeserialize; + + /// The cumulative length of object data read from the device during deserialization. + internal ulong deserializedLength; + + /// The total capacity of the buffer. + public bool IsForWrite => false; + +#pragma warning disable IDE0290 // Use primary constructor + public ObjectLogReader(CircularDiskReadBuffer readBuffers, TStoreFunctions storeFunctions) + { + this.readBuffers = readBuffers; + this.storeFunctions = storeFunctions ?? throw new ArgumentNullException(nameof(storeFunctions)); + } + + /// + /// Called when one or more records are to be read via ReadAsync. + /// + /// The initial file position to read + /// The cumulative length of all object-log entries for the span of records to be read. We read ahead for all record + /// in the ReadAsync call. + internal void OnBeginReadRecords(ObjectLogFilePositionInfo filePosition, ulong totalLength) + { + inDeserialize = false; + deserializedLength = 0UL; + readBuffers.OnBeginReadRecords(filePosition, totalLength); + } + + /// + public void FlushAndReset(CancellationToken cancellationToken = default) => throw new InvalidOperationException("FlushAndReset is not supported for DiskStreamReadBuffer"); + + /// + public void Write(ReadOnlySpan data, CancellationToken cancellationToken = default) => throw new InvalidOperationException("Write is not supported for DiskStreamReadBuffer"); + + /// + /// Get the object log entries for Overflow Keys and Values and Object Values for the record at . We create the log record here, + /// because we are calling this over a pages from iterator frames or Restore. + /// + /// Pointer to the initial record read from disk, either from iterator or Restore. + /// Number of bytes available at + /// The requested key, if not ReadAtAddress; we will compare to see if it matches the record. + /// The to place Overflow and Object Keys and Values in. + /// Number of bits in segment size + /// The output , which has its Key and Value ObjectIds filled in in the log record. + /// False if requestedKey is set and we read an Overflow key and it did not match; otherwise true + public bool ReadRecordObjects(long physicalAddress, int recordSize, ReadOnlySpan requestedKey, ObjectIdMap transientObjectIdMap, int segmentSizeBits, out LogRecord logRecord) + { + logRecord = new LogRecord(physicalAddress, transientObjectIdMap); + Debug.Assert(logRecord.GetInlineRecordSizes().actualSize <= recordSize, $"RecordSize ({recordSize}) is less than required LogRecord size ({logRecord.GetInlineRecordSizes().actualSize})"); + return logRecord.Info.RecordIsInline || ReadRecordObjects(ref logRecord, requestedKey, segmentSizeBits); + } + + /// + /// Get the object log entries for Overflow Keys and Values and Object Values for the input . We do not create the log record here; + /// that was already done by the caller (probably from a single-record disk IO). + /// + /// The input , which already has its inline components set and will have its Key and Value ObjectIds filled in by this call. + /// The requested key, if not ReadAtAddress; we will compare to see if it matches the record. + /// The to place Overflow and Object Keys and Values in. + /// Number of bits in segment size + /// False if requestedKey is set and we read an Overflow key and it did not match; otherwise true + public bool ReadRecordObjects(ref LogRecord logRecord, ReadOnlySpan requestedKey, ObjectIdMap transientObjectIdMap, int segmentSizeBits) + => logRecord.Info.RecordIsInline || ReadRecordObjects(ref logRecord, requestedKey, segmentSizeBits); + + /// + /// Get the object log entries for Overflow Keys and Values and Object Values for the record in , which came + /// from the initial IO operation. + /// + /// The initial record read from disk from Pending IO, so it is of size or less. + /// The requested key, if not ReadAtAddress; we will compare to see if it matches the record. + /// Number of bits in segment size + /// False if requestedKey is set and we read an Overflow key and it did not match; otherwise true + public bool ReadRecordObjects(ref DiskLogRecord diskLogRecord, ReadOnlySpan requestedKey, int segmentSizeBits) + => diskLogRecord.logRecord.Info.RecordIsInline || ReadRecordObjects(ref diskLogRecord.logRecord, requestedKey, segmentSizeBits); + + /// + /// Get the object log entries for Overflow Keys and Values and Object Values for the : + /// + /// If there is an Overflow key, read it and if we have a compare it and return false if it does not match. + /// Otherwise, store the Key Overflow in the transient in . + /// If we don't have , this is either ReadAtAddress (which is an implicit match) or Scan or Restore. + /// If we have an Overflow or Object value, read and store it in the transient in . + /// + /// + /// The initial record read from disk from Pending IO, so it is of size or less. + /// The requested key, if not ReadAtAddress; we will compare to see if it matches the record. + /// Number of bits in segment size + /// False if requestedKey is set and we read an Overflow key and it did not match; otherwise true + public bool ReadRecordObjects(ref LogRecord logRecord, ReadOnlySpan requestedKey, int segmentSizeBits) + { + Debug.Assert(logRecord.Info.RecordHasObjects, $"Inline records should have been checked by the caller"); + + var positionWord = logRecord.GetObjectLogRecordStartPositionAndLengths(out var keyLength, out var valueLength); + readBuffers.OnBeginRecord(new ObjectLogFilePositionInfo(positionWord, segmentSizeBits)); + + // TODO: Optimize the reading of large internal sector-aligned parts of Overflow Keys and Values to read directly into the overflow, similar to how ObjectLogWriter writes + // directly from overflow. This requires changing the read-ahead in CircularDiskReadBuffer.OnBeginReadRecords and the "backfill" in CircularDiskReadBuffer.MoveToNextBuffer. + + // Note: Similar logic to this is in DiskLogRecord.Deserialize. + if (logRecord.Info.KeyIsOverflow) + { + // This assignment also allocates the slot in ObjectIdMap. The varbyte length info should be unchanged from ObjectIdSize. + logRecord.KeyOverflow = new OverflowByteArray(keyLength, startOffset:0, endOffset:0, zeroInit:false); + _ = Read(logRecord.KeyOverflow.Span); + if (!requestedKey.IsEmpty && !storeFunctions.KeysEqual(requestedKey, logRecord.KeyOverflow.Span)) + return false; + } + + if (logRecord.Info.ValueIsOverflow) + { + // This assignment also allocates the slot in ObjectIdMap. The varbyte length info should be unchanged from ObjectIdSize. + logRecord.ValueOverflow = new OverflowByteArray((int)valueLength, startOffset: 0, endOffset: 0, zeroInit: false); + _ = Read(logRecord.ValueOverflow.Span); + } + else if (logRecord.Info.ValueIsObject) + { + // Info.ValueIsObject is true. This assignment also allocates the slot in ObjectIdMap and updates the varbyte length to be ObjectIdSize. + logRecord.ValueObject = DoDeserialize(); + } + return true; + } + + /// + public int Read(Span destinationSpan, CancellationToken cancellationToken = default) + { + // This is called by valueObjectSerializer.Deserialize() to read up to destinationSpan.Length bytes. + // It is also currently called internally for Overflow. + var prevCopyLength = 0; + var destinationSpanAppend = destinationSpan.Slice(prevCopyLength); + + // Read from the circular buffer. + var buffer = readBuffers.GetCurrentBuffer(); + while (true) + { + cancellationToken.ThrowIfCancellationRequested(); // IDevice does not support cancellation, so just check this here + + var copyLength = buffer.AvailableLength; + if (copyLength > destinationSpanAppend.Length) + copyLength = destinationSpanAppend.Length; + + if (copyLength > 0) + { + buffer.AvailableSpan.Slice(0, copyLength).CopyTo(destinationSpanAppend); + buffer.currentPosition += copyLength; + if (inDeserialize) + deserializedLength += (uint)copyLength; + if (copyLength == destinationSpanAppend.Length) + return destinationSpan.Length; + } + + prevCopyLength += copyLength; + if (buffer.AvailableLength == 0) + { + if (!readBuffers.MoveToNextBuffer(out buffer)) + return prevCopyLength; + } + destinationSpanAppend = destinationSpan.Slice(prevCopyLength); + } + } + + IHeapObject DoDeserialize() + { + deserializedLength = 0; + inDeserialize = true; + + // If we haven't yet instantiated the serializer do so now. + if (valueObjectSerializer is null) + { + pinnedMemoryStream = new(this); + valueObjectSerializer = storeFunctions.CreateValueObjectSerializer(); + valueObjectSerializer.BeginDeserialize(pinnedMemoryStream); + } + + valueObjectSerializer.Deserialize(out var valueObject); + OnDeserializeComplete(valueObject); + return valueObject; + } + + void OnDeserializeComplete(IHeapObject valueObject) + { + if (valueObject.SerializedSizeIsExact) + Debug.Assert(valueObject.SerializedSize == (long)deserializedLength, $"valueObject.SerializedSize(Exact) {valueObject.SerializedSize} != deserializedLength {deserializedLength}"); + else + valueObject.SerializedSize = (long)deserializedLength; + + // TODO add size tracking; do not track deserialization size changes if we are deserializing to a frame + + inDeserialize = false; + } + + /// + public void Dispose() + { + pinnedMemoryStream?.Dispose(); + valueObjectSerializer?.EndDeserialize(); + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogWriter.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogWriter.cs new file mode 100644 index 00000000000..37e2f5bfd51 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/ObjectLogWriter.cs @@ -0,0 +1,297 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.InteropServices; +using System.Threading; +using static Tsavorite.core.Utility; + +namespace Tsavorite.core +{ + /// + /// The class that manages IO writing of Overflow and Object Keys and Values for records. It manages the write buffer at two levels: + /// + /// At the higher level, called by routines, it manages the overall Key and Value writing, including flushing to disk as the buffer is filled. + /// At the lower level, it provides the stream for the valueObjectSerializer, which is called via Serialize() by the higher level. + /// + /// + /// This handles only Overflow Keys and Values, and Object Values; inline Keys and Values (of any length) are written to the main log device as part of the main log record. + internal unsafe partial class ObjectLogWriter : IStreamBuffer + where TStoreFunctions : IStoreFunctions + { + readonly IDevice device; + IObjectSerializer valueObjectSerializer; + PinnedMemoryStream> pinnedMemoryStream; + + /// The circular buffer we cycle through for parallelization of writes. + internal CircularDiskWriteBuffer flushBuffers; + + /// The implementation to use + internal readonly TStoreFunctions storeFunctions; + + /// The current buffer being written to in the circular buffer list. + internal DiskWriteBuffer writeBuffer; + + /// In the most common case, SerializedSizeIsExact is true and this is the expected length of the serialized value object + /// (used to verify the serialized size after serialization completes). + ulong expectedSerializedLength; + + /// For object serialization, the cumulative length of the value bytes. + ulong valueObjectBytesWritten; + + /// The maximum number of key or value bytes to copy into the buffer rather than enqueue a DirectWrite. + internal const int MaxCopySpanLen = 128 * 1024; + + /// If true, we are in the Serialize call. If not we ignore things like etc. + bool inSerialize; + + /// The total capacity of the buffer. + public bool IsForWrite => true; + + /// Constructor. Creates the circular buffer pool. +#pragma warning disable IDE0290 // Use primary constructor + public ObjectLogWriter(IDevice device, CircularDiskWriteBuffer flushBuffers, TStoreFunctions storeFunctions) + { + this.device = device ?? throw new ArgumentNullException(nameof(device)); + this.flushBuffers = flushBuffers ?? throw new ArgumentNullException(nameof(flushBuffers)); + this.storeFunctions = storeFunctions; + } + + /// + /// This is a no-op because we have already flushed under control of the Write() and OnPartialFlushComplete() methods. + public void FlushAndReset(CancellationToken cancellationToken = default) { } + + internal ObjectLogFilePositionInfo GetNextRecordStartPosition() => flushBuffers.GetNextRecordStartPosition(); + + /// Resets start positions for the next partial flush. + internal DiskWriteBuffer OnBeginPartialFlush(ObjectLogFilePositionInfo filePosition) + { + expectedSerializedLength = 0; + valueObjectBytesWritten = 0; + inSerialize = false; + writeBuffer = flushBuffers.OnBeginPartialFlush(filePosition); + return writeBuffer; + } + + /// + /// Finish all the current partial flushes, then write the main log page (or page fragment). + /// + /// Starting pointer of the main log page span to write + /// Length of the main log page span to write + /// The main log device to write to + /// The offset in the main log to write at; aligned to sector + /// Callback sent to the initial Flush() command. Called when we are done with this partial flush operation. + /// Context sent to . + /// The ending file position after the partial flush is complete + internal unsafe void OnPartialFlushComplete(byte* mainLogPageSpanPtr, int mainLogPageSpanLength, IDevice mainLogDevice, ulong alignedMainLogFlushAddress, + DeviceIOCompletionCallback externalCallback, object externalContext, out ObjectLogFilePositionInfo endFilePosition) + => flushBuffers.OnPartialFlushComplete(mainLogPageSpanPtr, mainLogPageSpanLength, mainLogDevice, alignedMainLogFlushAddress, + externalCallback, externalContext, out endFilePosition); + + /// + /// Write Overflow and Object Keys and values in a to the device. + /// + /// The whose Keys and Values are to be written to the device. + /// This only writes Overflow and Object Keys and Values; inline portions of the record are written separately. + /// The number of bytes written for the value object, if any (Overflow lengths and the length of objects that support + /// are already known, but an object that does not support that will not know its + /// ) until we've serialized it). + public ulong WriteRecordObjects(in LogRecord logRecord) + { + Debug.Assert(logRecord.Info.RecordHasObjects, "Cannot call ObjectLogWriter with an inline record"); + + // If the key is overflow, start with that. (Inline keys are written as part of the main-log record.) + if (logRecord.Info.KeyIsOverflow) + WriteDirect(logRecord.KeyOverflow); + + if (logRecord.Info.ValueIsOverflow) + WriteDirect(logRecord.ValueOverflow); + else if (logRecord.Info.ValueIsObject) + { + var obj = logRecord.ValueObject; + if (obj.SerializedSizeIsExact) + { + if (obj.SerializedSize >= IHeapObject.MaxSerializedObjectSize) + throw new TsavoriteException($"Object size exceeds max serialization limit of {IHeapObject.MaxSerializedObjectSize}"); + expectedSerializedLength = (ulong)obj.SerializedSize; + } + DoSerialize(obj); + if (!obj.SerializedSizeIsExact && obj.SerializedSize >= IHeapObject.MaxSerializedObjectSize) + throw new TsavoriteException($"Object size exceeds max serialization limit of {IHeapObject.MaxSerializedObjectSize}"); + } + flushBuffers.OnRecordComplete(); + return valueObjectBytesWritten; + } + + /// Start off the write using the full span of the . + /// The to write. + void WriteDirect(OverflowByteArray overflow) => WriteDirect(overflow, overflow.ReadOnlySpan, refCountedGCHandle: default); + + /// Write the of the . + /// The to write. + /// The span of to write. Initially it is the full ; if the write + /// spans segments, then it is a recursive call for the last segment's fraction. + /// The refcounted GC handle if this is a recursive call + void WriteDirect(OverflowByteArray overflow, ReadOnlySpan fullDataSpan, RefCountedPinnedGCHandle refCountedGCHandle) + { + if (overflow.Length <= MaxCopySpanLen) + Write(fullDataSpan); + else + { + // 1. Write the sector-aligning start fragment into the buffers and flush the current buffer (if we cross a buffer boundary, + // previous buffers will already have been flushed). + var dataStart = 0; + var copyLength = RoundUp(writeBuffer.currentPosition, (int)device.SectorSize) - writeBuffer.currentPosition; + if (copyLength != 0) + { + Debug.Assert(refCountedGCHandle is null, $"If refCountedGCHandle is not null then buffer.currentPosition ({writeBuffer.currentPosition}) should already be sector-aligned"); + Write(fullDataSpan.Slice(dataStart, copyLength)); + dataStart += copyLength; + flushBuffers.FlushCurrentBuffer(); + } + + // 2. Flush the sector-aligned span interior. We are writing direct to the device from a byte[], so we have to pin the array. + // We may have to split across multiple segments. + var interiorLen = RoundDown(overflow.Array.Length - dataStart, (int)device.SectorSize); + var segmentRemainingLen = flushBuffers.filePosition.RemainingSize; + var gcHandle = (refCountedGCHandle is null) ? GCHandle.Alloc(overflow.Array, GCHandleType.Pinned) : default; + var localGcHandle = refCountedGCHandle?.gcHandle ?? gcHandle; + var overflowStartPtr = (byte*)localGcHandle.AddrOfPinnedObject() + overflow.StartOffset; + if ((uint)interiorLen <= segmentRemainingLen) + { + // We have enough room in the segment to write the full interior span in one chunk. + var writeCallback = refCountedGCHandle is null + ? flushBuffers.CreateDiskWriteCallbackContext(gcHandle) + : flushBuffers.CreateDiskWriteCallbackContext(refCountedGCHandle); + flushBuffers.FlushToDevice(overflowStartPtr + dataStart, interiorLen, writeCallback); + dataStart += interiorLen; + } + else + { + // Multi-segment write so we will need to refcount the GCHandle. SegmentRemainingLength is <= int.MaxValue so we can cast it to int. + // TODO: This and other segment-limiting logic could be pushed down into StorageDeviceBase, which could iterate on the segments. + // However this could have complications with e.g. callback and countdown counts (there would be more than one callback invocation + // on that; this could be handled by defining some way for the StorageDeviceBase to know the calback uses a CountdownEvent and + // incrementing that count, or by having a local callback, similarly to how CircularDiskWriteBuffer handles multiple possibly-concurrent + // writes before calling the main callback, that handles doing the "final" callback). In this case we could defer the "segment id" logic + // to StorageDeviceBase, and just have a ulong position, from which we could compute the segment id (e.g. for truncation), and + // ObjectLogFilePositionInfo would be simplified. + Debug.Assert(segmentRemainingLen <= int.MaxValue, $"segmentRemainingLen ({segmentRemainingLen}) should be <= int.MaxValue"); + + // Create the refcounted pinned GCHandle with a refcount of 1, so that if a read completes while we're still setting up, we won't get an early unpin. + refCountedGCHandle ??= new RefCountedPinnedGCHandle(gcHandle, initialCount: 1); + + // Copy chunks to segments and advance the segment. + while (interiorLen > (int)segmentRemainingLen) + { + var writeCallback = flushBuffers.CreateDiskWriteCallbackContext(refCountedGCHandle); + flushBuffers.FlushToDevice(overflowStartPtr + dataStart, (int)segmentRemainingLen, writeCallback); + dataStart += (int)segmentRemainingLen; + + Debug.Assert(flushBuffers.filePosition.RemainingSize == 0, $"Expected to be at end of segment but there were {flushBuffers.filePosition.RemainingSize} bytes remaining"); + flushBuffers.filePosition.AdvanceToNextSegment(); + segmentRemainingLen = flushBuffers.filePosition.RemainingSize; + } + + // Now we know we will fit in the last segment, so call recursively to optimize the "copy vs. direct" final fragment. + // First adjust the endPosition in case we don't have a full buffer of space remaining in the segment. + if ((ulong)writeBuffer.RemainingCapacity > flushBuffers.filePosition.RemainingSize) + writeBuffer.endPosition = (int)flushBuffers.filePosition.RemainingSize - writeBuffer.currentPosition; + WriteDirect(overflow, fullDataSpan.Slice(dataStart), refCountedGCHandle); + } + + // 3. Copy the end sector-aligning fragment to the buffers. + if (dataStart < overflow.Length) + Write(fullDataSpan.Slice(dataStart)); + } + + // Release the initial refcount on this, if we created it. This will let it final-release when all writes are complete. + refCountedGCHandle?.Release(); + } + + /// + public void Write(ReadOnlySpan data, CancellationToken cancellationToken = default) + { + // This is called by valueObjectSerializer.Serialize() as well as internally. No other calls should write data to flushBuffer.memory in a way + // that increments flushBuffer.currentPosition, since we manage chained-chunk continuation and DiskPageHeader offsetting here. + + // Copy to the buffer. If it does not fit in the remaining capacity, we will write as much as does, flush the buffer, and move to next buffer. + var dataStart = 0; + while (data.Length - dataStart > 0) + { + Debug.Assert(writeBuffer.RemainingCapacity > 0, + $"RemainingCapacity {writeBuffer.RemainingCapacity} should not be 0 (data.Length {data.Length}, dataStart {dataStart}); this should have already triggered an OnChunkComplete call, which would have reset the buffer"); + cancellationToken.ThrowIfCancellationRequested(); // IDevice does not support cancellation, so just check this here + + // If it won't all fit in the remaining buffer, write as much as will. + var requestLength = data.Length - dataStart; + if (requestLength > writeBuffer.RemainingCapacity) + requestLength = writeBuffer.RemainingCapacity; + + data.Slice(dataStart, requestLength).CopyTo(writeBuffer.memory.TotalValidSpan.Slice(writeBuffer.currentPosition)); + dataStart += requestLength; + writeBuffer.currentPosition += requestLength; + if (inSerialize) + valueObjectBytesWritten += (uint)requestLength; + + // See if we're at the end of the buffer. + if (writeBuffer.RemainingCapacity == 0) + OnBufferComplete(); + } + } + + /// At the end of a buffer, do any processing, flush the current buffer, and move to the next buffer. + /// Called during Serialize(). + void OnBufferComplete() + { + // This should only be called when the object serialization hits the end of the buffer; for partial buffers we will call + // OnSerializeComplete() after the Serialize() call has returned. "End of buffer" ends before lengthSpaceReserve if any. + Debug.Assert(writeBuffer.currentPosition == writeBuffer.endPosition, $"CurrentPosition {writeBuffer.currentPosition} must be at writeBuffer.endPosition {writeBuffer.endPosition})."); + + flushBuffers.FlushCurrentBuffer(); + writeBuffer = flushBuffers.MoveToAndInitializeNextBuffer(); + } + + void DoSerialize(IHeapObject valueObject) + { + // valueCumulativeLength is only relevant for object serialization; we increment it on all device writes to avoid "if", so here we reset it to the appropriate + // "start at 0" by making it the negative of currentPosition. Subsequently if we write e.g. an int, we'll have Length and Position = (-currentPosition + currentPosition + 4). + inSerialize = true; + valueObjectBytesWritten = 0; + + // If we haven't yet instantiated the serializer do so now. + if (valueObjectSerializer is null) + { + pinnedMemoryStream = new(this); + valueObjectSerializer = storeFunctions.CreateValueObjectSerializer(); + valueObjectSerializer.BeginSerialize(pinnedMemoryStream); + } + + valueObjectSerializer.Serialize(valueObject); + OnSerializeComplete(valueObject); + } + + void OnSerializeComplete(IHeapObject valueObject) + { + if (valueObject.SerializedSizeIsExact) + { + if (valueObjectBytesWritten != expectedSerializedLength) + throw new TsavoriteException($"Expected value length {expectedSerializedLength} does not match actual value length {valueObjectBytesWritten}."); + } + else + valueObject.SerializedSize = (long)valueObjectBytesWritten; + inSerialize = false; + } + + /// + public int Read(Span destinationSpan, CancellationToken cancellationToken = default) => throw new InvalidOperationException("Read is not supported for DiskStreamWriteBuffer"); + + /// + public void Dispose() + { + pinnedMemoryStream?.Dispose(); + valueObjectSerializer?.EndSerialize(); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/PinnedMemoryStream.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/PinnedMemoryStream.cs new file mode 100644 index 00000000000..042b2db7774 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/PinnedMemoryStream.cs @@ -0,0 +1,176 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.IO; +using System.Threading; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + /// + /// This is a simple stream over a pinned memory buffer, such as a SectorAlignedMemory or network buffer. + /// + internal class PinnedMemoryStream : Stream + where TStreamBuffer : IStreamBuffer + { + TStreamBuffer streamBuffer; + + public PinnedMemoryStream(TStreamBuffer streamBuffer) + { + this.streamBuffer = streamBuffer; + } + + /// Whether the stream is opened for Read + public override bool CanRead => !streamBuffer.IsForWrite; + + /// This stream implementation cannot Seek + public override bool CanSeek => false; + + /// Whether the stream is opened for Write + public override bool CanWrite => streamBuffer.IsForWrite; + + /// + protected override void Dispose(bool disposing) + { + streamBuffer.Dispose(); + base.Dispose(disposing); + } + + /// Flush the internal buffer + public override void Flush() => streamBuffer.FlushAndReset(); + + /// Flush the internal buffer asynchronously + public override Task FlushAsync(CancellationToken cancellationToken) + { + if (cancellationToken.IsCancellationRequested) + return Task.FromCanceled(cancellationToken); + + try + { + streamBuffer.FlushAndReset(cancellationToken); + return Task.CompletedTask; + } + catch (Exception ex) + { + return Task.FromException(ex); + } + } + + /// The amount of data in the internal streamBuffer. Not supported because we chunk and thus may not have all data. + public override long Length + { + get => throw new InvalidOperationException("Stream does not support get_Length."); + } + + /// The current position of the stream seeking; not supported + public override long Position + { + get => throw new InvalidOperationException("Stream does not support get_Position."); + set => throw new InvalidOperationException("Stream does not support set_Position."); + } + + /// Copy data from the internal streamBuffer into the buffer; the streamBuffer handles Flush, Reset, and Read more + /// (e.g. from disk or network) as needed. + /// Buffer to copy the bytes into. + /// Index in the buffer to start copying to. + /// Desired number of bytes to copy to the buffer. + /// Number of bytes actually read. + public override int Read(byte[] buffer, int offset, int count) + { + ValidateBufferArguments(buffer, offset, count); + return streamBuffer.Read(new Span(buffer, offset, count)); + } + + /// Copy data from the internal streamBuffer into the destination span; the streamBuffer handles Flush, Reset, and Read more + /// (e.g. from disk or network) as needed. + public override int Read(Span destinationSpan) => streamBuffer.Read(destinationSpan); + + /// Asynchronously copy data from the internal streamBuffer into the memory buffer; the streamBuffer handles Flush, Reset, and Read more + /// (e.g. from disk or network) as needed. + /// Buffer to read the bytes to. + /// Token that can be used to cancel this operation. + public override ValueTask ReadAsync(Memory buffer, CancellationToken cancellationToken = default) + { + if (cancellationToken.IsCancellationRequested) + return ValueTask.FromCanceled(cancellationToken); + + try + { + return new ValueTask(Read(buffer.Span)); + } + catch (Exception ex) + { + return ValueTask.FromException(ex); + } + } + + /// Returns the byte at the current streamBuffer position and advances the position + /// The byte read (as an int) + public override unsafe int ReadByte() + { + Span span = stackalloc byte[1]; + return streamBuffer.Read(span) > 0 ? span[0] : -1; + } + + /// Seeking is not supported in this stream. + public override long Seek(long offset, SeekOrigin loc) => throw new InvalidOperationException("Stream does not support Seek."); + + /// Seeking is not supported in this stream. + public override void SetLength(long value) => throw new InvalidOperationException("Stream does not support SetLength."); + + /// Write the buffer to the stream; the streamBuffer handles Flush, Reset, and Writing iteratively + /// (e.g. to disk or network) as needed. + /// Buffer to write the bytes from. + /// Index in the buffer to start writing from. + /// Desired number of bytes to write from the buffer. + public override void Write(byte[] buffer, int offset, int count) + { + ValidateBufferArguments(buffer, offset, count); + streamBuffer.Write(new ReadOnlySpan(buffer, offset, count)); + } + + /// Write the buffer to the stream; the streamBuffer handles Flush, Reset, and Writing iteratively + /// (e.g. to disk or network) as needed. + public override void Write(ReadOnlySpan destinationSpan) => streamBuffer.Write(destinationSpan); + + /// Asynchronously write the buffer to the stream; the streamBuffer handles Flush, Reset, and Writing iteratively + /// (e.g. to disk or network) as needed. + /// Buffer to write the bytes from. + /// Index in the buffer to start writing from. + /// Desired number of bytes to write from the buffer. + /// Task that can be awaited + public override Task WriteAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken = default) + { + ValidateBufferArguments(buffer, offset, count); + return WriteAsync(new ReadOnlySpan(buffer, offset, count), cancellationToken).AsTask(); + } + + /// Asynchronously write the buffer to the stream; the streamBuffer handles Flush, Reset, and Writing iteratively + /// (e.g. to disk or network) as needed. + /// Buffer to write the bytes from. + /// Token that can be used to cancel the operation. + public override ValueTask WriteAsync(ReadOnlyMemory memoryBuffer, CancellationToken cancellationToken = default) + => WriteAsync(memoryBuffer.Span, cancellationToken); + + private ValueTask WriteAsync(ReadOnlySpan destinationSpan, CancellationToken cancellationToken) + { + if (cancellationToken.IsCancellationRequested) + return ValueTask.FromCanceled(cancellationToken); + + try + { + streamBuffer.Write(destinationSpan, cancellationToken); + return ValueTask.CompletedTask; + } + catch (Exception ex) + { + return ValueTask.FromException(ex); + } + } + + /// Writes a byte at the next streamBuffer position and advances the position + public override unsafe void WriteByte(byte value) + => streamBuffer.Write(new ReadOnlySpan(&value, 1)); + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/VarbyteLengthUtility.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/VarbyteLengthUtility.cs new file mode 100644 index 00000000000..17f2726c1a3 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectSerialization/VarbyteLengthUtility.cs @@ -0,0 +1,361 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; +using System.Numerics; +using System.Runtime.CompilerServices; + +namespace Tsavorite.core +{ + /// + /// Utilities for varlen bytes: one indicator byte identifying the number of key and value bytes. The layout of this indicator byte is: + /// + /// Indicators: flags, such as filler and ignore optionals + /// Number of bytes in key length; may be inline length or if Overflow. Max is + /// Number of bytes in value length; may be inline length or if Overflow or Object + /// + /// This is followed by the actual key length and value length, which may be inline length or if Overflow + /// or object. For in=memory objects, the max inline key size is 16MB to remain in 3 bytes, and the max inline value size is int.MaxValue + /// which is 4 bytes, so the total is 8 bytes which can be atomically updated. + /// + public static unsafe class VarbyteLengthUtility + { +#pragma warning disable IDE1006 // Naming Styles: Must begin with uppercase letter + const long kReservedBitMask = 0 << 7; // Reserved bit + + /// + /// When we are updating a record and changing length components (value length, or the presence or absence or length of optional fields), + /// we must not let Scan see a record with inconsistent lengths. Thus, we set this bit and adjust value length to the entire record while + /// updating the record, and ignores the optional fields and uses only the key and value lengths + /// (along with RecordInfo and the varbyte length word) to calculate record length. + /// + internal const long kIgnoreOptionalsBitMask = 1 << 6; + + /// If this is set, then we have extra length in the record after any optional fields. We may have some extra length that is + /// less than the size of an int even if this bit is not set, due to record-alignment padding. + internal const long kHasFillerBitMask = 1 << 5; + + // The bottom 5 bits are actual length bytecounts + /// + /// 2 bits for the number of bytes for the key length: + /// + /// In-memory: this is limited to 16MB inline, so 3 bytes; allows Overflow, + /// which takes only (4) bytes + /// On-disk: this is limited to max Overflow length, so 4 bytes + /// + /// + const long kKeyLengthBitMask = 3 << 3; + /// + /// 3 bits for the number of bytes for the value length: + /// + /// In-memory: this is limited to 16MB inline, so 3 bytes; allows Overflow and Object, + /// which take only (4) bytes + /// On-disk: this is limited to either max Object length, but since we have an effective limit of bits + /// , this will not be greater than 6 bytes. + /// + /// + const long kValueLengthBitMask = 7; +#pragma warning restore IDE1006 // Naming Styles + + /// The minimum number of length metadata bytes--NumIndicatorBytes, 1 byte key length, 1 byte value length + public const int MinLengthMetadataBytes = 3; + /// The maximum number of length metadata bytes--NumIndicatorBytes, 4 bytes key length, 7 bytes value length + internal const int MaxLengthMetadataBytes = 12; + /// The number of indicator bytes; currently 1 for the length indicator. + internal const int NumIndicatorBytes = 3; + + /// The maximum number of key length bytes in the in-memory single-long word representation. We use zero-based sizes and add 1, so + /// 1 bit allows us to specify 1 or 2 bytes; we max at 2, or . Anything over this becomes overflow. + internal const int MaxKeyLengthBytesInWord = 1; + /// The maximum number of value length bytes in the in-memory single-long word representation. We use zero-based sizes and add 1, so + /// 2 bits allows us to specify 1 to 4 bytes; we max at 3, or . Anything over this becomes overflow. + internal const int MaxValueLengthBytesInWord = 2; + + /// Read var-length bytes at the given location. + /// This is compatible with little-endian 'long'; thus, the indicator byte is the low byte of the word, then keyLengthBytes, valueLengthBytes, keyLength, valueLength in ascending address order + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static long ReadVarbyteLength(int numBytes, byte* ptrToFirstByte) + { + long value = 0; + for (var ii = 0; ii < numBytes; ii++) + value |= (long)*(ptrToFirstByte + ii) << (ii * 8); + return value; + } + + /// Read var-length bytes in the given word. Used for in-memory and limited to 3-byte keys and 4-byte values + /// which combine with the indicator byte to fit into a 'long'. The shift operations are faster than the pointer-based alternative implementation + /// used for disk-image generation, which has the data expanded inline so may have 4-byte keys and 8-byte values. + /// The word being queried + /// If we are querying for value, this is the number of bytes in the key; otherwise it is 0 + /// The number of bytes in the target (key or value) + /// This assumes little-endian; thus, the indicator byte is the low byte of the word, then keyLengthBytes, valueLengthBytes, keyLength, valueLength in ascending address order + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int ReadVarbyteLengthInWord(long word, int precedingNumBytes, int targetNumBytes) + => (int)((word >> ((NumIndicatorBytes + precedingNumBytes) * 8)) & ((1L << (targetNumBytes * 8)) - 1)); + + /// Write var-length bytes at the given location. + /// This is compatible with little-endian 'long'; thus, the indicator byte is the low byte of the word, then keyLengthBytes, valueLengthBytes, keyLength, valueLength in ascending address order + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static void WriteVarbyteLength(long value, int numBytes, byte* ptrToFirstByte) + { + for (var ii = 0; ii < numBytes; ii++) + { + *(ptrToFirstByte + ii) = (byte)(value & 0xFF); + value >>= 8; + } + Debug.Assert(value == 0, "len too short"); + } + + /// Write var-length bytes into the given word. Used for in-memory and limited to 3-byte keys and 4-byte values + /// which combine with the indicator byte to fit into a 'long'. The shift operations are faster than the pointer-based alternative implementation + /// used for disk-image generation, which has the data expanded inline so may have 4-byte keys and 8-byte values. + /// The word being updated + /// The target value being set into the word (key or value length) + /// If we are setting the value, this is the number of bytes in the key; otherwise it is 0 + /// The number of bytes in the target (key or value) + /// This assumes little-endian; thus, the indicator byte (containing flags, keyLengthBytes, valueLengthBytes) is the low byte of the word, + /// then keyLength, then valueLength, in ascending address order + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static void WriteVarbyteLengthInWord(ref long word, int targetValue, int precedingNumBytes, int targetNumBytes) + { + // This is ascending order, so we will shift over the lower-order bytes. + var shift = (NumIndicatorBytes + precedingNumBytes) * 8; + var targetMask = (1L << (targetNumBytes * 8)) - 1; + + // Mask off the target area of the word (i.e. keep everything except where we will shift-OR the target into. + word &= ~(targetMask << shift); + + // Now mask the target value to include only what we are going to keep, then shift that into the target area of the word. + word |= (targetValue & targetMask) << shift; + } + + internal static int GetKeyLength(int numBytes, byte* ptrToFirstByte) => (int)ReadVarbyteLength(numBytes, ptrToFirstByte); + + internal static long GetValueLength(int numBytes, byte* ptrToFirstByte) => ReadVarbyteLength(numBytes, ptrToFirstByte); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static byte ConstructIndicatorByte(int keyLength, long valueLength, out int keyByteCount, out int valueByteCount) + { + keyByteCount = GetByteCount(keyLength); + valueByteCount = GetByteCount(valueLength); + return ConstructIndicatorByte(keyByteCount, valueByteCount); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static byte ConstructIndicatorByte(int keyByteCount, int valueByteCount) + { + return (byte)( + ((long)(keyByteCount - 1) << 3) // Shift key into position; subtract 1 for 0-based + | (long)(valueByteCount - 1)); // Value does not need to be shifted; subtract 1 for 0-based + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static (int keyLengthBytes, int valueLengthBytes, bool hasFiller) DeconstructIndicatorByte(byte indicatorByte) + { + var keyLengthBytes = (int)((indicatorByte & kKeyLengthBitMask) >> 3) + 1; // add 1 due to 0-based + var valueLengthBytes = (int)(indicatorByte & kValueLengthBitMask) + 1; // add 1 due to 0-based + var hasFiller = (indicatorByte & kHasFillerBitMask) != 0; + return (keyLengthBytes, valueLengthBytes, hasFiller); + } + + /// + /// Create a copy of the word with the value length taking up all the record after the key and with the IgnoreOptionals flag set. + /// This is used to ensure consistent Scan while a record is undergoing modification. + /// + /// The varbyte length word we're "updating". + /// Number of bytes in the key length + /// Number of bytes in the value length + /// Length of the value + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static long CreateIgnoreOptionalsVarbyteWord(long originalWord, int keyLengthBytes, int valueLengthBytes, int valueLength) + { + // The key remains the same so we don't need to rewrite its length. And the byte counts do not change. + Debug.Assert(valueLength < (1 << (valueLengthBytes * 8)), $"ValueLength {valueLength} overflows valueLengthBytes {valueLengthBytes}"); + var word = originalWord; + var ptr = (byte*)&word; + *ptr = (byte)((*ptr & ~kHasFillerBitMask) | kIgnoreOptionalsBitMask); + WriteVarbyteLengthInWord(ref word, valueLength, precedingNumBytes: keyLengthBytes, valueLengthBytes); + return word; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static bool HasIgnoreOptionals(byte indicatorByte) => (indicatorByte & kIgnoreOptionalsBitMask) != 0; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static bool HasFiller(byte indicatorByte) => (indicatorByte & kHasFillerBitMask) != 0; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static void SetHasFiller(long indicatorAddress) => *(byte*)indicatorAddress = (byte)(*(byte*)indicatorAddress | kHasFillerBitMask); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static void ClearHasFiller(long indicatorAddress) => *(byte*)indicatorAddress = (byte)(*(byte*)indicatorAddress & ~kHasFillerBitMask); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int GetByteCount(long value) => ((sizeof(long) * 8) - BitOperations.LeadingZeroCount((ulong)(value | 1)) + 7) / 8; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static (int length, long dataAddress) GetKeyFieldInfo(long indicatorAddress) + { + var (keyLengthBytes, valueLengthBytes, _ /*hasFiller*/) = DeconstructIndicatorByte(*(byte*)indicatorAddress); + + // Move past the indicator byte; the next bytes are key length + var keyLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: 0, keyLengthBytes); + + // Move past the key and value length bytes to the start of the key data + return (keyLength, indicatorAddress + NumIndicatorBytes + keyLengthBytes + valueLengthBytes); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static (long length, long dataAddress) GetValueFieldInfo(long indicatorAddress) + { + var (keyLengthBytes, valueLengthBytes, _ /*hasFiller*/) = DeconstructIndicatorByte(*(byte*)indicatorAddress); + + // Move past the indicator byte; the next bytes are key length + var keyLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: 0, keyLengthBytes); + + // Move past the key length bytes; the next bytes are valueLength + var valueLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: keyLengthBytes, valueLengthBytes); + + // Move past the key and value length bytes and the key data to the start of the value data + return (valueLength, indicatorAddress + NumIndicatorBytes + keyLengthBytes + valueLengthBytes + keyLength); + } + + /// + /// Get the value data pointer, as well as the pointer to length, length, and number of length bytes. This is to support in-place updating. + /// + /// The value data pointer + internal static byte* GetFieldPtr(long indicatorAddress, bool isKey, out byte* lengthPtr, out int lengthBytes, out long length) + { + var ptr = (byte*)indicatorAddress; + var (keyLengthBytes, valueLengthBytes, _ /*hasFiller*/) = DeconstructIndicatorByte(*ptr); + ptr++; + + // Move past the indicator byte; the next bytes are key length + var keyLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: 0, keyLengthBytes); + if (isKey) + { + lengthPtr = ptr; + lengthBytes = keyLengthBytes; + length = keyLength; + return ptr + keyLengthBytes + valueLengthBytes; + } + + // Move past the key length bytes; the next bytes are valueLength. Read those, then skip over the key bytes to get the value data pointer. + lengthPtr = ptr + keyLengthBytes; + lengthBytes = valueLengthBytes; + length = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: keyLengthBytes, lengthBytes); + return lengthPtr + lengthBytes + keyLength; + } + + /// + /// Get the value data pointer, as well as the pointer to length, length, and number of length bytes. This is to support in-place updating. + /// + /// The value data pointer + internal static (int keyLength, int valueLength, int offsetToKeyStart) GetInlineKeyAndValueSizes(long indicatorAddress) + { + var ptr = (byte*)indicatorAddress; + var (keyLengthBytes, valueLengthBytes, _ /*hasFiller*/) = DeconstructIndicatorByte(*ptr); + + // Move past the indicator byte; the next bytes are key length + var keyLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: 0, keyLengthBytes); + + // Move past the key bytes; the next bytes are valueLength + var valueLength = ReadVarbyteLengthInWord(*(long*)indicatorAddress, precedingNumBytes: keyLengthBytes, valueLengthBytes); + return (keyLength, valueLength, RecordInfo.Size + NumIndicatorBytes + keyLengthBytes + valueLengthBytes); + } + + /// + /// Construct the in-memory inline varbyte indicator word consisting of keyLengthBytes, valueLengthBytes, and a "has filler" indicator. + /// This is used to atomically update the varbyte length information so scanning will be consistent. + /// + /// The inline length of the key + /// The inline length of the value + /// Either kHasFillerBitMask if we have set a filler length into the in-memory record, or 0 + /// Receives the number of bytes in the key length + /// Receives the number of bytes in the value length + /// + internal static unsafe long ConstructInlineVarbyteLengthWord(int keyLength, int valueLength, long flagBits, out int keyLengthBytes, out int valueLengthBytes) + { + keyLengthBytes = GetByteCount(keyLength); + valueLengthBytes = GetByteCount(valueLength); + return ConstructInlineVarbyteLengthWord(keyLengthBytes, keyLength, valueLengthBytes, valueLength, flagBits); + } + + /// + /// Construct the in-memory inline varbyte indicator word consisting of keyLengthBytes, valueLengthBytes, and a "has filler" indicator. + /// This is used to atomically update the varbyte length information so scanning will be consistent. + /// + /// Number of bytes in the key length + /// The inline length of the key + /// Number of bytes in the value length + /// The inline length of the value + /// , , or 0 + /// + internal static unsafe long ConstructInlineVarbyteLengthWord(int keyLengthBytes, int keyLength, int valueLengthBytes, int valueLength, long flagBits) + { + var word = (long)0; + var ptr = (byte*)&word; + *ptr++ = (byte)(ConstructIndicatorByte(keyLengthBytes, valueLengthBytes) | flagBits); + ptr += 2; // Space for RecordType and Namespace + + WriteVarbyteLengthInWord(ref word, keyLength, precedingNumBytes: 0, keyLengthBytes); + WriteVarbyteLengthInWord(ref word, valueLength, precedingNumBytes: keyLengthBytes, valueLengthBytes); + return word; + } + + /// + /// Deconstruct the in-memory inline varbyte indicator word to return keyLengthBytes, valueLengthBytes, and the "has filler" indicator. + /// This is used to atomically update the varbyte length information so scanning will be consistent. + /// + /// keyLengthBytes, valueLengthBytes, and the "has filler" indicator + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe (int keyLength, int valueLength, bool hasFiller) DeconstructInlineVarbyteLengthWord(long word) + { + var ptr = (byte*)&word; + (var keyLengthBytes, var valueLengthBytes, var hasFiller) = DeconstructIndicatorByte(*ptr++); + Debug.Assert(keyLengthBytes <= MaxKeyLengthBytesInWord, "Inline keyLengthBytes limit exceeded"); + Debug.Assert(valueLengthBytes <= MaxValueLengthBytesInWord, "Inline valueLengthBytes limit exceeded"); + + var keyLength = ReadVarbyteLengthInWord(*(long*)ptr, precedingNumBytes: 0, keyLengthBytes); + var valueLength = ReadVarbyteLengthInWord(*(long*)ptr, precedingNumBytes: keyLengthBytes, valueLengthBytes); + return (keyLength, valueLength, hasFiller); + } + + /// + /// Update the key and value lengths in the in-memory inline varbyte indicator word + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe long CreateUpdatedInlineVarbyteLengthWord(long originalWord, int keyLengthBytes, int valueLengthBytes, int valueLength, long hasFillerBit) + { + // Mask off the filler bit; we'll reset it on return. + var word = originalWord & ~(kHasFillerBitMask | kIgnoreOptionalsBitMask); + WriteVarbyteLengthInWord(ref word, valueLength, precedingNumBytes: keyLengthBytes, valueLengthBytes); + return word | hasFillerBit; + } + + /// + /// Update the Key length in the in-memory inline varbyte indicator word. Used when mapping from on-disk objectlog pointers to in-memory ObjectIds. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe void UpdateVarbyteKeyLengthByteInWord(long indicatorAddress, byte valueLength) + { + var word = *(long*)indicatorAddress; + (var keyLengthBytes, var valueLengthBytes, var _ /*hasFiller*/) = DeconstructIndicatorByte(*(byte*)&word); + WriteVarbyteLengthInWord(ref word, valueLength, precedingNumBytes: 0, keyLengthBytes); + *(long*)indicatorAddress = word; + } + + /// + /// Update the Value length in the in-memory inline varbyte indicator word. Used when mapping from on-disk objectlog pointers to in-memory ObjectIds. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe void UpdateVarbyteValueLengthByteInWord(long indicatorAddress, byte valueLength) + { + var word = *(long*)indicatorAddress; + (var keyLengthBytes, var valueLengthBytes, var _ /*hasFiller*/) = DeconstructIndicatorByte(*(byte*)&word); + WriteVarbyteLengthInWord(ref word, valueLength, precedingNumBytes: keyLengthBytes, valueLengthBytes); + *(long*)indicatorAddress = word; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs index f60e5051727..57e9758d17a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs @@ -6,64 +6,46 @@ namespace Tsavorite.core { /// - /// Holds optional fields (ETag and Expiration, as well as managing Filler) during a record-resizing operation. - /// Ensures proper zeroinit handling and restoring these fields to their correct location (and updating FillerLength). + /// Saves optional fields ETag and Expiration during a record-resizing operation and restores them when done. /// + /// + /// We don't save ObjectLogPosition; that's only used during Serialization. The caller (TrySetValueLength) adjusts filler + /// address and length by the growth (positive or negative) of the object value, so no address adjustment or zeroing of + /// space is needed. + /// internal unsafe struct OptionalFieldsShift { - internal long ETag; - internal long Expiration; - - internal static OptionalFieldsShift SaveAndClear(long address, ref RecordInfo recordInfo) => new(address, ref recordInfo); - - private OptionalFieldsShift(long address, ref RecordInfo recordInfo) => GetAndZero(address, ref recordInfo); + long eTag; + long expiration; [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void GetAndZero(long address, ref RecordInfo recordInfo) + internal void Save(long address, RecordInfo recordInfo) { - // We are calling this when we are shifting, so zero out the old value. We do not need to spend cycles clearing the Has* bits in - // RecordInfo because we will not be doing data operations during this shift, and we have already verified they will be within - // range of the record's allocated size, and here we are zeroing them so zero-init is maintained. if (recordInfo.HasETag) { - ETag = *(long*)address; - *(long*)address = 0; + eTag = *(long*)address; address += LogRecord.ETagSize; } if (recordInfo.HasExpiration) { - Expiration = *(long*)address; - *(long*)address = 0; + expiration = *(long*)address; address += LogRecord.ExpirationSize; } - - // For Filler we do need to clear the bit, as we may end up with no filler. We don't preserve the existing value; it will be calculated in Restore(). - if (recordInfo.HasFiller) - { - *(int*)address = 0; - recordInfo.ClearHasFiller(); - } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Restore(long address, ref RecordInfo recordInfo, int fillerLen) + internal void Restore(long address, RecordInfo recordInfo) { - // Restore after shift. See comments in GetAndZero for more details. if (recordInfo.HasETag) { - *(long*)address = ETag; + *(long*)address = eTag; address += LogRecord.ETagSize; } if (recordInfo.HasExpiration) { - *(long*)address = Expiration; + *(long*)address = expiration; address += LogRecord.ExpirationSize; } - if (fillerLen >= LogRecord.FillerLengthSize) - { - *(long*)address = fillerLen; - recordInfo.SetHasFiller(); - } } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/OverflowByteArray.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/OverflowByteArray.cs new file mode 100644 index 00000000000..0ff0e9ff473 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/OverflowByteArray.cs @@ -0,0 +1,79 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; + +namespace Tsavorite.core +{ + /// A byte[] wrapper that encodes start and end offsets of the actual data in the first sizeof() bytes in the array. + /// Used primarily for sector-aligned reads directly into the overflow byte[]. + public struct OverflowByteArray + { + /// Define the header of an overflow allocation. Overflow allocations are typically large so use a full int to allow greater than 64k offsets, + /// which makes it possible to read more information with a single IO and then copy it out to other destinations. Sector sizes may be up to 64k on NTFS systems, + /// which is sizeof(ushort) bytes, so the use of full ints removes boundary concerns (e.g. reading a value followed by optional bytes may cross a sector boundary, + /// in which case we need an end offset greater than a single sector). + struct OverflowHeader + { + internal const int Size = 2 * sizeof(int); + internal int startOffset, endOffset; + } + + internal readonly byte[] Array { get; init; } + + internal readonly bool IsEmpty => Array is null; + + internal readonly int StartOffset => Unsafe.As(ref Array[0]).startOffset + OverflowHeader.Size; + + readonly int EndOffset => Unsafe.As(ref Array[0]).endOffset; + + internal readonly int Length => Array.Length - StartOffset - EndOffset; + + /// Span of data between offsets + internal readonly ReadOnlySpan ReadOnlySpan => Array.AsSpan().Slice(StartOffset, Length); + /// Span of data between offsets + internal readonly Span Span => Array.AsSpan().Slice(StartOffset, Length); + + /// Span of all data, including before and after offsets; this is for aligned Read from the device. + internal readonly Span AlignedReadSpan => Array.AsSpan().Slice(OverflowHeader.Size); + + /// Construct an from a byte[] allocated by . + internal OverflowByteArray(byte[] data) => Array = data; + + internal OverflowByteArray(int length, int startOffset, int endOffset, bool zeroInit) + { + // Allocate with enough extra space for the metadata (offset from start and end) + Array = !zeroInit + ? GC.AllocateUninitializedArray(length + OverflowHeader.Size) + : (new byte[length + OverflowHeader.Size]); + ref var header = ref Unsafe.As(ref Array[0]); + header.startOffset = startOffset; + header.endOffset = endOffset; + } + + /// Increase the offset from the start, e.g. after having extracted the key that was read in the same IO operation as the value. + /// This is 'readonly' because it does not alter the array field, only its contents. + internal readonly void AdjustOffsetFromStart(int increment) => Unsafe.As(ref Array[0]).startOffset += increment; + /// Increase the offset from the end, e.g. after having extracted the optionals that were read in the same IO operation as the value. + /// This is 'readonly' because it does not alter the > array field, only its contents. + internal readonly void AdjustOffsetFromEnd(int increment) => Unsafe.As(ref Array[0]).endOffset += increment; + + internal readonly void SetOffsets(int offsetFromStart, int offsetFromEnd) + { + Debug.Assert(offsetFromStart > 0 && offsetFromStart < Array.Length - 1, "offsetFromStart is out of range"); + Debug.Assert(offsetFromEnd > 0 && offsetFromEnd < Array.Length - 1, "offsetFromEnd is out of range"); + Debug.Assert(offsetFromStart < offsetFromEnd, "offsetFromStart must be less than offsetFromEnd"); + ref var header = ref Unsafe.As(ref Array[0]); + header.startOffset = offsetFromStart; + header.endOffset = offsetFromEnd; + } + + /// Get the of a byte[] allocated by constructor. + internal static ReadOnlySpan AsReadOnlySpan(object value) => new OverflowByteArray(Unsafe.As(value)).ReadOnlySpan; + + /// Get the of a byte[] allocated by constructor. + internal static Span AsSpan(object value) => new OverflowByteArray(Unsafe.As(value)).Span; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/PageHeader.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/PageHeader.cs new file mode 100644 index 00000000000..2f82258f881 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/PageHeader.cs @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Runtime.InteropServices; + +namespace Tsavorite.core +{ + [StructLayout(LayoutKind.Explicit, Size = Size)] + internal struct PageHeader + { + const ushort CurrentVersion = 1; + + /// The number of bits in the size of the struct. Currently set to make the size that the 0'th page offset was in earlier versions; 64 bytes + internal const int SizeBits = 6; + + /// The size of the struct. Must be a power of 2. Currently set to the size that the 0'th page offset was; 64 bytes + internal const int Size = 1 << SizeBits; + + /// Version of this page header. + [FieldOffset(0)] + internal ushort version; + + [FieldOffset(sizeof(ushort))] + internal ushort unusedUshort1; + + [FieldOffset(sizeof(int))] + internal int unusedInt1; + + /// The lowest object-log position on this main-log page, if ObjectAllocator. Contains both segmentId and offset on segment + [FieldOffset(sizeof(long))] + internal ulong objectLogLowestPosition; + + // Unused; as they become used, start with higher # + [FieldOffset(sizeof(long) * 2)] + internal long unusedLong6; + [FieldOffset(sizeof(long) * 3)] + internal long unusedLong5; + [FieldOffset(sizeof(long) * 4)] + internal long unusedLong4; + [FieldOffset(sizeof(long) * 5)] + internal long unusedLong3; + [FieldOffset(sizeof(long) * 6)] + internal long unusedLong2; + [FieldOffset(sizeof(long) * 7)] + internal long unusedLong1; + + /// + /// Initializes the struct. + /// + /// + internal void Initialize() + { + this = default; + version = CurrentVersion; + objectLogLowestPosition = 0; + } + + internal static unsafe void Initialize(long physicalAddressOfStartOfPage) => (*(PageHeader*)physicalAddressOfStartOfPage).Initialize(); + + /// + /// Set the lowest object-log position on this main-log page, if ObjectAllocator. + /// + /// The position in the object log. + internal void SetLowestObjectLogPosition(in ObjectLogFilePositionInfo position) + { + if (objectLogLowestPosition == 0) + objectLogLowestPosition = position.word; + } + + public override readonly string ToString() + => $"ver {version}, lowObjLogPos {objectLogLowestPosition}, us1 {unusedUshort1}, ui1 {unusedInt1}, ul1 {unusedLong1}, ul2 {unusedLong2}, ul3 {unusedLong3}, ul4 {unusedLong4}, ul5 {unusedLong5}, ul6 {unusedLong6}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs index 6c08fe99efa..6d8c847c192 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs @@ -14,38 +14,48 @@ namespace Tsavorite.core /// public abstract class ScanIteratorBase { - /// - /// Frame size - /// + /// Frame size (1 or 2) protected readonly int frameSize; - /// - /// Begin address. Cannot be readonly due to SnapCursorToLogicalAddress - /// + /// Begin address of the scan. Cannot be readonly due to SnapCursorToLogicalAddress protected long beginAddress; - /// - /// End address - /// + /// End address of the scan protected readonly long endAddress; - /// - /// Epoch - /// + /// Epoch from the store protected readonly LightEpoch epoch; - /// - /// Current and next address for iteration - /// - protected long currentAddress, nextAddress; + /// Current address for iteration + protected long currentAddress; + /// Next address for iteration + protected long nextAddress; + + /// vector for waiting for frame-load completion. + /// This array is in parallel with , , and . + private CountdownEvent[] loadCompletionEvents; + + /// vector for canceling the wait for frame-load completion. + /// This array is in parallel with , , and . + private CancellationTokenSource[] loadCTSs; + + /// Vector of endAddresses for the currently loaded pages of the frames. + /// This array is in parallel with , , and . + private long[] loadedPages; + + /// Vector of endAddresses for the currently in-flight, and possibly completed, loading of pages of the frames. + /// This is updated atomically when we start the of a page. + /// This array is in parallel with , , and . + private long[] nextLoadedPages; - private CountdownEvent[] loaded; - private CancellationTokenSource[] loadedCancel; - private long[] loadedPage; - private long[] nextLoadedPage; + /// Number of bits in the size of the log page private readonly int logPageSizeBits; + + /// Whether to include closed records in the scan protected readonly bool includeClosedRecords; - protected readonly bool returnTombstoned; + + /// The circular buffer we cycle through for object-log deserialization. + readonly CircularDiskReadBuffer readBuffers; /// /// Current address @@ -72,22 +82,18 @@ public abstract class ScanIteratorBase /// protected ILogger logger; + /// + /// Buffering for holding copies of in-memory records + /// protected InMemoryScanBufferingMode memScanBufferingMode; /// /// Constructor /// - /// - /// - /// - /// - /// - /// - /// - /// - public unsafe ScanIteratorBase(long beginAddress, long endAddress, DiskScanBufferingMode scanBufferingMode, InMemoryScanBufferingMode memScanBufferingMode, + public unsafe ScanIteratorBase(CircularDiskReadBuffer readBuffers, long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, InMemoryScanBufferingMode memScanBufferingMode, bool includeClosedRecords, LightEpoch epoch, int logPageSizeBits, bool initForReads = true, ILogger logger = null) { + this.readBuffers = readBuffers; this.logger = logger; this.memScanBufferingMode = memScanBufferingMode; @@ -103,16 +109,17 @@ public unsafe ScanIteratorBase(long beginAddress, long endAddress, DiskScanBuffe currentAddress = -1; nextAddress = beginAddress; - if (scanBufferingMode == DiskScanBufferingMode.SinglePageBuffering) + if (diskScanBufferingMode == DiskScanBufferingMode.SinglePageBuffering) frameSize = 1; - else if (scanBufferingMode == DiskScanBufferingMode.DoublePageBuffering) + else if (diskScanBufferingMode == DiskScanBufferingMode.DoublePageBuffering) frameSize = 2; - else if (scanBufferingMode == DiskScanBufferingMode.NoBuffering) + else if (diskScanBufferingMode == DiskScanBufferingMode.NoBuffering) { frameSize = 0; return; } - if (initForReads) InitializeForReads(); + if (initForReads) + InitializeForReads(); } /// @@ -120,15 +127,15 @@ public unsafe ScanIteratorBase(long beginAddress, long endAddress, DiskScanBuffe /// public virtual void InitializeForReads() { - loaded = new CountdownEvent[frameSize]; - loadedCancel = new CancellationTokenSource[frameSize]; - loadedPage = new long[frameSize]; - nextLoadedPage = new long[frameSize]; + loadCompletionEvents = new CountdownEvent[frameSize]; + loadCTSs = new CancellationTokenSource[frameSize]; + loadedPages = new long[frameSize]; + nextLoadedPages = new long[frameSize]; for (int i = 0; i < frameSize; i++) { - loadedPage[i] = -1; - nextLoadedPage[i] = -1; - loadedCancel[i] = new CancellationTokenSource(); + loadedPages[i] = -1; + nextLoadedPages[i] = -1; + loadCTSs[i] = new CancellationTokenSource(); } currentAddress = -1; nextAddress = beginAddress; @@ -137,11 +144,12 @@ public virtual void InitializeForReads() /// /// Buffer and load /// - /// - /// - /// - /// - /// + /// The current logical address + /// The page containing the current logical address + /// The frame index of the current page (the page modulo the number of frames) + /// Head address of the log + /// Address to stop the scan at + /// True we had to await the event here; /// protected unsafe bool BufferAndLoad(long currentAddress, long currentPage, long currentFrame, long headAddress, long endAddress) { @@ -149,17 +157,12 @@ protected unsafe bool BufferAndLoad(long currentAddress, long currentPage, long { var nextPage = currentPage + i; - // Convert to absolute addresses as we are going to disk. The LogAddress methods (GetPage, etc.) work with both absolute and AddressType-prefixed addresses. - currentAddress = AbsoluteAddress(currentAddress); - headAddress = AbsoluteAddress(headAddress); - endAddress = AbsoluteAddress(endAddress); - - var pageStartAddress = GetStartAbsoluteLogicalAddressOfPage(nextPage, logPageSizeBits); // Cannot load page if it is entirely in memory or beyond the end address + var pageStartAddress = GetLogicalAddressOfStartOfPage(nextPage, logPageSizeBits); if (pageStartAddress >= headAddress || pageStartAddress >= endAddress) continue; - var pageEndAddress = GetStartAbsoluteLogicalAddressOfPage(nextPage + 1, logPageSizeBits); + var pageEndAddress = GetLogicalAddressOfStartOfPage(nextPage + 1, logPageSizeBits); if (endAddress < pageEndAddress) pageEndAddress = endAddress; if (headAddress < pageEndAddress) @@ -167,28 +170,39 @@ protected unsafe bool BufferAndLoad(long currentAddress, long currentPage, long var nextFrame = (currentFrame + i) % frameSize; - long val; - while ((val = nextLoadedPage[nextFrame]) < pageEndAddress || loadedPage[nextFrame] < pageEndAddress) + while (true) { - if (val < pageEndAddress && Interlocked.CompareExchange(ref nextLoadedPage[nextFrame], pageEndAddress, val) == val) + // Get the endAddress of the next page being loaded for this frame. If it is already loaded, as indicated by being >= the required endAddress, we're done. + var val = nextLoadedPages[nextFrame]; + if (val >= pageEndAddress && loadedPages[nextFrame] >= pageEndAddress) + break; + + // If the endAddress of the next page being loaded is less than the endAddress we need for the next page for this frame, + // try to atomically exchange it with the endAddress we need. If successful, issue the load. + if (val < pageEndAddress && Interlocked.CompareExchange(ref nextLoadedPages[nextFrame], pageEndAddress, val) == val) { var tmp_page = i; if (epoch != null) { epoch.BumpCurrentEpoch(() => { - AsyncReadPagesFromDeviceToFrame(tmp_page + GetPage(currentAddress, logPageSizeBits), 1, endAddress, Empty.Default, out loaded[nextFrame], 0, null, null, loadedCancel[nextFrame]); - loadedPage[nextFrame] = pageEndAddress; + AsyncReadPagesFromDeviceToFrame(readBuffers, tmp_page + GetPageOfAddress(currentAddress, logPageSizeBits), 1, endAddress, + Empty.Default, out loadCompletionEvents[nextFrame], 0, null, null, loadCTSs[nextFrame]); + loadedPages[nextFrame] = pageEndAddress; }); } else { - AsyncReadPagesFromDeviceToFrame(tmp_page + GetPage(currentAddress, logPageSizeBits), 1, endAddress, Empty.Default, out loaded[nextFrame], 0, null, null, loadedCancel[nextFrame]); - loadedPage[nextFrame] = pageEndAddress; + AsyncReadPagesFromDeviceToFrame(readBuffers, tmp_page + GetPageOfAddress(currentAddress, logPageSizeBits), 1, endAddress, + Empty.Default, out loadCompletionEvents[nextFrame], 0, null, null, loadCTSs[nextFrame]); + loadedPages[nextFrame] = pageEndAddress; } } else + { + // Someone else already incremented nextLoadedPage[nextFrame], so give them a chance to work, then try again. epoch?.ProtectAndDrain(); + } } } return WaitForFrameLoad(currentAddress, currentFrame); @@ -203,13 +217,13 @@ protected unsafe bool NeedBufferAndLoad(long currentAddress, long currentPage, l { var nextPage = currentPage + i; - var pageStartAddress = GetStartAbsoluteLogicalAddressOfPage(nextPage, logPageSizeBits); + var pageStartAddress = GetLogicalAddressOfStartOfPage(nextPage, logPageSizeBits); // Cannot load page if it is entirely in memory or beyond the end address if (pageStartAddress >= headAddress || pageStartAddress >= endAddress) continue; - var pageEndAddress = GetStartAbsoluteLogicalAddressOfPage(nextPage + 1, logPageSizeBits); + var pageEndAddress = GetLogicalAddressOfStartOfPage(nextPage + 1, logPageSizeBits); if (endAddress < pageEndAddress) pageEndAddress = endAddress; if (headAddress < pageEndAddress) @@ -217,30 +231,39 @@ protected unsafe bool NeedBufferAndLoad(long currentAddress, long currentPage, l var nextFrame = (currentFrame + i) % frameSize; - if (nextLoadedPage[nextFrame] < pageEndAddress || loadedPage[nextFrame] < pageEndAddress) + if (nextLoadedPages[nextFrame] < pageEndAddress || loadedPages[nextFrame] < pageEndAddress) return true; } return false; } - internal abstract void AsyncReadPagesFromDeviceToFrame(long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, + internal abstract void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null); + /// + /// Wait for the current frame to complete loading + /// + /// + /// + /// True if we had to wait for the current frame load to complete; else false + /// private bool WaitForFrameLoad(long currentAddress, long currentFrame) { - if (loaded[currentFrame].IsSet) + if (loadCompletionEvents[currentFrame].IsSet) return false; try { epoch?.Suspend(); - loaded[currentFrame].Wait(loadedCancel[currentFrame].Token); // Ensure we have completed ongoing load + loadCompletionEvents[currentFrame].Wait(loadCTSs[currentFrame].Token); // Ensure we have completed ongoing load } catch (Exception e) { - loadedPage[currentFrame] = -1; - loadedCancel[currentFrame] = new CancellationTokenSource(); - Utility.MonotonicUpdate(ref nextAddress, GetStartAbsoluteLogicalAddressOfPage(1 + GetPage(currentAddress, logPageSizeBits), logPageSizeBits), out _); + // Exception occurred so skip the page containing the currentAddress, and reinitialize the loaded page and cancellation token for the current frame. + // The exception may have been an OperationCanceledException. + loadedPages[currentFrame] = -1; + loadCTSs[currentFrame] = new CancellationTokenSource(); + Utility.MonotonicUpdate(ref nextAddress, GetLogicalAddressOfStartOfPage(1 + GetPageOfAddress(currentAddress, logPageSizeBits), logPageSizeBits), out _); throw new TsavoriteException("Page read from storage failed, skipping page. Inner exception: " + e.ToString()); } finally @@ -255,16 +278,16 @@ private bool WaitForFrameLoad(long currentAddress, long currentFrame) /// public virtual void Dispose() { - if (loaded != null) + if (loadCompletionEvents != null) { // Wait for ongoing reads to complete/fail for (int i = 0; i < frameSize; i++) { - if (loadedPage[i] != -1) + if (loadedPages[i] != -1) { try { - loaded[i].Wait(loadedCancel[i].Token); + loadCompletionEvents[i].Wait(loadCTSs[i].Token); } catch { } } @@ -277,15 +300,15 @@ public virtual void Dispose() ///
public void Reset() { - loaded = new CountdownEvent[frameSize]; - loadedCancel = new CancellationTokenSource[frameSize]; - loadedPage = new long[frameSize]; - nextLoadedPage = new long[frameSize]; + loadCompletionEvents = new CountdownEvent[frameSize]; + loadCTSs = new CancellationTokenSource[frameSize]; + loadedPages = new long[frameSize]; + nextLoadedPages = new long[frameSize]; for (int i = 0; i < frameSize; i++) { - loadedPage[i] = -1; - nextLoadedPage[i] = -1; - loadedCancel[i] = new CancellationTokenSource(); + loadedPages[i] = -1; + nextLoadedPages[i] = -1; + loadCTSs[i] = new CancellationTokenSource(); } currentAddress = -1; nextAddress = beginAddress; diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SerializationPhase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SerializationPhase.cs new file mode 100644 index 00000000000..afd48a8f3f4 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SerializationPhase.cs @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + public enum SerializationPhase : int + { + /// + /// Serialization has not been started. + /// + REST, + + /// + /// Serialization is in progress. + /// + SERIALIZING, + + /// + /// Serialization has been completed. + /// + SERIALIZED + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs index 0f22e7c3ad0..2f42ee5ec21 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs @@ -6,7 +6,9 @@ namespace Tsavorite.core { - // Allocator for ReadOnlySpan Keys and Span Values. + /// + /// Allocator for ReadOnlySpan{byte} Keys and Span{byte} Values. + /// public struct SpanByteAllocator : IAllocator where TStoreFunctions : IStoreFunctions { @@ -30,27 +32,13 @@ public readonly AllocatorBase GetBase() where TAllocator : IAllocator => (AllocatorBase)(object)_this; - /// - public readonly bool IsFixedLength => false; - /// public readonly bool HasObjectLog => false; /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetStartLogicalAddress(long page) => _this.GetStartLogicalAddressOfPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddressOnPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void InitializeValue(long physicalAddress, in RecordSizeInfo sizeInfo) => _this.InitializeValue(physicalAddress, in sizeInfo); + public readonly void InitializeRecord(ReadOnlySpan key, long logicalAddress, in RecordSizeInfo sizeInfo, ref LogRecord newLogRecord) + => newLogRecord.InitializeRecord(key, in sizeInfo); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -94,14 +82,6 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.AllocatePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly bool IsAllocated(int pageIndex) => _this.IsAllocated(pageIndex); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe void PopulatePage(byte* src, int required_bytes, long destinationPageIndex) => _this.PopulatePage(src, required_bytes, destinationPageIndex); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void MarkPage(long logicalAddress, long version) => _this.MarkPage(logicalAddress, version); @@ -118,18 +98,9 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.FreePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() - => SpanByteAllocatorImpl.GetSegmentOffsets(); - /// public readonly int OverflowPageCount => _this.OverflowPageCount; - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => _this.SerializeKey(key, logicalAddress, ref logRecord); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly LogRecord CreateLogRecord(long logicalAddress) => _this.CreateLogRecord(logicalAddress); @@ -138,6 +109,12 @@ public readonly long[] GetSegmentOffsets() [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => _this.CreateLogRecord(logicalAddress, physicalAddress); + /// + public readonly LogRecord CreateRemappedLogRecordOverTransientMemory(long logicalAddress, long physicalAddress) => _this.CreateLogRecord(logicalAddress, physicalAddress); + + /// + public readonly ObjectIdMap TranssientObjectIdMap => default; + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs index 005c7543206..e26098208e3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs @@ -9,27 +9,16 @@ namespace Tsavorite.core { - using static LogAddress; - using static Utility; - // Allocator for ReadOnlySpan Key and Span Value. internal sealed unsafe class SpanByteAllocatorImpl : AllocatorBase> where TStoreFunctions : IStoreFunctions { - /// Circular buffer definition - /// The long is actually a byte*, but storing as 'long' makes going through logicalAddress/physicalAddress translation more easily - long* pagePointers; - private OverflowPool> freePagePool; public SpanByteAllocatorImpl(AllocatorSettings settings, TStoreFunctions storeFunctions, Func> wrapperCreator) : base(settings.LogSettings, storeFunctions, wrapperCreator, settings.evictCallback, settings.epoch, settings.flushCallback, settings.logger) { freePagePool = new OverflowPool>(4, p => { }); - - var bufferSizeInBytes = (nuint)RoundUp(sizeof(long*) * BufferSize, Constants.kCacheLineBytes); - pagePointers = (long*)NativeMemory.AlignedAlloc(bufferSizeInBytes, Constants.kCacheLineBytes); - NativeMemory.Clear(pagePointers, bufferSizeInBytes); } internal int OverflowPageCount => freePagePool.Count; @@ -51,15 +40,14 @@ internal void AllocatePage(int index) IncrementAllocatedPageCount(); if (freePagePool.TryGet(out var item)) - { pagePointers[index] = item.pointer; - // TODO resize the values[index] arrays smaller if they are above a certain point - return; + else + { + // No free pages are available so allocate new + pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); + NativeMemory.Clear((void*)pagePointers[index], (nuint)PageSize); } - - // No free pages are available so allocate new - pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); - NativeMemory.Clear((void*)pagePointers[index], (nuint)PageSize); + PageHeader.Initialize(pagePointers[index]); } void ReturnPage(int index) @@ -81,22 +69,7 @@ void ReturnPage(int index) internal LogRecord CreateLogRecord(long logicalAddress) => CreateLogRecord(logicalAddress, GetPhysicalAddress(logicalAddress)); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new LogRecord(physicalAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => SerializeKey(key, logicalAddress, ref logRecord, maxInlineKeySize: int.MaxValue, objectIdMap: null); - - public override void Initialize() => Initialize(FirstValidAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeValue(long physicalAddress, in RecordSizeInfo sizeInfo) - { - // Value is always inline in the SpanByteAllocator - var valueAddress = LogRecord.GetValueAddress(physicalAddress); - - LogRecord.GetInfoRef(physicalAddress).SetValueIsInline(); - _ = LogField.SetInlineDataLength(valueAddress, sizeInfo.FieldInfo.ValueDataSize); - } + internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new (physicalAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] public RecordSizeInfo GetRMWCopyRecordSize(in TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) @@ -158,8 +131,8 @@ public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) { FieldInfo = new() { - KeyDataSize = key.Length, - ValueDataSize = 0, // No payload for the default value + KeySize = key.Length, + ValueSize = 0, // No payload for the default value HasETag = false, HasExpiration = false } @@ -170,25 +143,27 @@ public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) public void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) { - // For SpanByteAllocator, we are always inline. + // For SpanByteAllocator, we are always inline. Keys are limited to 3 bytes though, to make the Varbyte indicator word assignment atomic. // Key sizeInfo.KeyIsInline = true; - var keySize = sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize; + var keySize = sizeInfo.FieldInfo.KeySize; + if (keySize > LogSettings.kMaxInlineKeySize) + throw new TsavoriteException($"Max inline key size is {LogSettings.kMaxInlineKeySize}"); // Value - sizeInfo.MaxInlineValueSpanSize = int.MaxValue; // Not currently doing out-of-line for SpanByteAllocator + sizeInfo.MaxInlineValueSize = int.MaxValue; // Not currently doing out-of-line for SpanByteAllocator sizeInfo.ValueIsInline = true; - var valueSize = sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize; + var valueSize = sizeInfo.FieldInfo.ValueSize; // Record - sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; - sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); + sizeInfo.CalculateSizes(keySize, valueSize); } [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) { - logRecord.ClearOptionals(); + if (logRecord.IsSet) + logRecord.ClearOptionals(); // Key and Value are always inline in the SpanByteAllocator so this is a no-op } @@ -202,48 +177,23 @@ public override void Dispose() var localFreePagePool = Interlocked.Exchange(ref freePagePool, null); if (localFreePagePool != null) { - base.Dispose(); localFreePagePool.Dispose(); - - if (pagePointers is not null) - { - for (var ii = 0; ii < BufferSize; ii++) - { - if (pagePointers[ii] != 0) - NativeMemory.AlignedFree((void*)pagePointers[ii]); - } - NativeMemory.AlignedFree((void*)pagePointers); - pagePointers = null; - } + base.Dispose(); } } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetPhysicalAddress(long logicalAddress) - { - // Index of page within the circular buffer, and offset on the page. TODO move this (and pagePointers) to AllocatorBase) - var pageIndex = GetPageIndexForAddress(logicalAddress); - var offset = GetOffsetOnPage(logicalAddress); - return *(pagePointers + pageIndex) + offset; - } - - internal bool IsAllocated(int pageIndex) => pagePointers[pageIndex] != 0; - - protected override void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) - => WriteAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * flushPage), + protected override void WriteAsync(CircularDiskWriteBuffer flushBuffers, long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) + => WriteInlinePageAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * flushPage), (uint)AlignedPageSizeBytes, callback, asyncResult, device); - protected override void WriteAsyncToDevice(long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, - PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long[] localSegmentOffsets, long fuzzyStartLogicalAddress) + protected override void WriteAsyncToDevice(CircularDiskWriteBuffer flushBuffers, long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, + PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long fuzzyStartLogicalAddress) { VerifyCompatibleSectorSize(device); - WriteAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), + WriteInlinePageAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), (uint)AlignedPageSizeBytes, callback, asyncResult, device); } - internal void ClearPage(long page, int offset) - => NativeMemory.Clear((byte*)pagePointers[page % BufferSize] + offset, (nuint)(PageSize - offset)); - internal void FreePage(long page) { ClearPage(page, 0); @@ -251,36 +201,16 @@ internal void FreePage(long page) ReturnPage((int)(page % BufferSize)); } - protected override void ReadAsync(ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, - DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device, IDevice objlogDevice) - => device.ReadAsync(alignedSourceAddress, (IntPtr)pagePointers[destinationPageIndex], aligned_read_length, callback, asyncResult); - - /// - /// Invoked by users to obtain a record from disk. It uses sector aligned memory to read - /// the record efficiently into memory. - /// - /// - /// - /// - /// - /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) - => throw new InvalidOperationException("AsyncReadRecordObjectsToMemory invalid for SpanByteAllocator"); - - internal static long[] GetSegmentOffsets() => null; - - internal void PopulatePage(byte* src, int required_bytes, long destinationPage) - { - throw new TsavoriteException("SpanByteAllocator memory pages are sector aligned - use direct copy"); - // Buffer.MemoryCopy(src, (void*)pointers[destinationPage % BufferSize], required_bytes, required_bytes); - } + protected override void ReadAsync(CircularDiskReadBuffer readBuffers, ulong alignedSourceAddress, IntPtr destinationPtr, uint aligned_read_length, + DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device) + => device.ReadAsync(alignedSourceAddress, destinationPtr, aligned_read_length, callback, asyncResult); /// /// Iterator interface for pull-scanning Tsavorite log /// public override ITsavoriteScanIterator Scan(TsavoriteKV> store, long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, bool includeClosedRecords) - => new RecordScanIterator>(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeClosedRecords: includeClosedRecords, logger: logger); + => new SpanByteScanIterator>(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeClosedRecords: includeClosedRecords, logger: logger); /// /// Implementation for push-scanning Tsavorite log, called from LogAccessor @@ -288,7 +218,7 @@ public override ITsavoriteScanIterator Scan(TsavoriteKV(TsavoriteKV> store, long beginAddress, long endAddress, ref TScanFunctions scanFunctions, DiskScanBufferingMode diskScanBufferingMode) { - using RecordScanIterator> iter = new(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, logger: logger); + using SpanByteScanIterator> iter = new(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, logger: logger); return PushScanImpl(beginAddress, endAddress, ref scanFunctions, iter); } @@ -298,9 +228,9 @@ internal override bool Scan(TsavoriteKV(TsavoriteKV> store, ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress, bool resetCursor = true, bool includeTombstones = false) { - using RecordScanIterator> iter = new(store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, + using SpanByteScanIterator> iter = new(store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: maxAddress < long.MaxValue, logger: logger); - return ScanLookup>>(store, scanCursorState, + return ScanLookup>>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress, resetCursor: resetCursor, includeTombstones: includeTombstones); } @@ -310,14 +240,14 @@ internal override bool ScanCursor(TsavoriteKV(TsavoriteKV> store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) { - using RecordScanIterator> iter = new(store, this, beginAddress, epoch, logger: logger); + using SpanByteScanIterator> iter = new(store, this, beginAddress, epoch, logger: logger); return IterateHashChain(store, key, beginAddress, ref scanFunctions, iter); } /// internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver observer) { - using var iter = new RecordScanIterator>(store: null, this, beginAddress, endAddress, epoch, DiskScanBufferingMode.NoBuffering, InMemoryScanBufferingMode.NoBuffering, + using var iter = new SpanByteScanIterator>(store: null, this, beginAddress, endAddress, epoch, DiskScanBufferingMode.NoBuffering, InMemoryScanBufferingMode.NoBuffering, includeClosedRecords: false, assumeInMemory: true, logger: logger); observer?.OnNext(iter); } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs similarity index 67% rename from libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs rename to libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs index bf567c9588f..ce4c2cd139e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs @@ -12,13 +12,13 @@ namespace Tsavorite.core /// /// Scan iterator for hybrid log /// - public sealed unsafe class RecordScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator + public sealed unsafe class SpanByteScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { private readonly TsavoriteKV store; private readonly AllocatorBase hlogBase; - private readonly BlittableFrame frame; // TODO remove GenericFrame + private readonly BlittableFrame frame; private SectorAlignedMemory recordBuffer; private readonly bool assumeInMemory; @@ -38,11 +38,12 @@ public sealed unsafe class RecordScanIterator : Sca /// Epoch to use for protection; may be null if is true. /// Provided address range is known by caller to be in memory, even if less than HeadAddress /// - internal RecordScanIterator(TsavoriteKV store, AllocatorBase hlogBase, + internal SpanByteScanIterator(TsavoriteKV store, AllocatorBase hlogBase, long beginAddress, long endAddress, LightEpoch epoch, DiskScanBufferingMode diskScanBufferingMode, InMemoryScanBufferingMode memScanBufferingMode = InMemoryScanBufferingMode.NoBuffering, bool includeClosedRecords = false, bool assumeInMemory = false, ILogger logger = null) - : base(beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, endAddress, diskScanBufferingMode, memScanBufferingMode, includeClosedRecords, epoch, hlogBase.LogPageSizeBits, logger: logger) + : base(readBuffers: default, beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, endAddress, + diskScanBufferingMode, memScanBufferingMode, includeClosedRecords, epoch, hlogBase.LogPageSizeBits, logger: logger) { this.store = store; this.hlogBase = hlogBase; @@ -54,9 +55,10 @@ internal RecordScanIterator(TsavoriteKV store, Allo /// /// Constructor for use with tail-to-head push iteration of the passed key's record versions /// - internal RecordScanIterator(TsavoriteKV store, AllocatorBase hlogBase, + internal SpanByteScanIterator(TsavoriteKV store, AllocatorBase hlogBase, long beginAddress, LightEpoch epoch, ILogger logger = null) - : base(beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, hlogBase.GetTailAddress(), DiskScanBufferingMode.SinglePageBuffering, InMemoryScanBufferingMode.NoBuffering, false, epoch, hlogBase.LogPageSizeBits, logger: logger) + : base(readBuffers: default, beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddressOnPage(0) : beginAddress, hlogBase.GetTailAddress(), + DiskScanBufferingMode.SinglePageBuffering, InMemoryScanBufferingMode.NoBuffering, false, epoch, hlogBase.LogPageSizeBits, logger: logger) { this.store = store; this.hlogBase = hlogBase; @@ -65,6 +67,7 @@ internal RecordScanIterator(TsavoriteKV store, Allo frame = new BlittableFrame(frameSize, hlogBase.PageSize, hlogBase.GetDeviceSectorSize()); } + #region TODO Unify with ObjectScanIterator /// public bool SnapCursorToLogicalAddress(ref long cursor) { @@ -93,6 +96,7 @@ private bool InitializeGetNextAndAcquireEpoch(out long stopAddress) if (diskLogRecord.IsSet) { hlogBase._wrapper.DisposeRecord(ref diskLogRecord, DisposeReason.DeserializedFromDisk); + diskLogRecord.Dispose(); diskLogRecord = default; } diskLogRecord = default; @@ -146,33 +150,36 @@ internal long SnapToLogicalAddressBoundary(ref long logicalAddress, long headAdd totalSizes = (int)hlogBase.BeginAddress; } - // We don't have to worry about going past the end of the page because we're using offset to bound the scan. - if (logicalAddress >= headAddress) + while (totalSizes <= offset) { - while (totalSizes <= offset) - { - var allocatedSize = new LogRecord(physicalAddress).GetInlineRecordSizes().allocatedSize; - if (totalSizes + allocatedSize > offset) - break; - totalSizes += allocatedSize; - physicalAddress += allocatedSize; - } - } - else - { - while (totalSizes <= offset) - { - var allocatedSize = new DiskLogRecord(physicalAddress).GetSerializedLength(); - if (totalSizes + allocatedSize > offset) - break; - totalSizes += allocatedSize; - physicalAddress += allocatedSize; - } + var allocatedSize = new LogRecord(physicalAddress).GetInlineRecordSizes().allocatedSize; + if (totalSizes + allocatedSize > offset) + break; + totalSizes += allocatedSize; + physicalAddress += allocatedSize; } return logicalAddress += totalSizes - offset; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + long GetPhysicalAddress(long currentAddress, long headAddress, long currentPage, long offset) + => currentAddress >= headAddress || assumeInMemory + ? hlogBase.GetPhysicalAddress(currentAddress) + : frame.GetPhysicalAddress(currentPage, offset); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, long currentPage, long offset, out long allocatedSize) + { + var physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); + + // We are just getting sizes so no need for ObjectIdMap + var logRecord = new LogRecord(physicalAddress); + (var _, allocatedSize) = logRecord.GetInlineRecordSizes(); + return logRecord.physicalAddress; + } + #endregion TODO Unify with ObjectScanIterator + /// /// Get next record in iterator /// @@ -190,47 +197,52 @@ public unsafe bool GetNext() return false; var offset = hlogBase.GetOffsetOnPage(currentAddress); - - // TODO: This can process past end of page if it is a too-small record at end of page and its recordInfo is not null - var physicalAddress = GetPhysicalAddressAndAllocatedSize(currentAddress, headAddress, currentPage, offset, out long allocatedSize); - - // It's safe to use LogRecord here even for on-disk because both start with the RecordInfo. + var physicalAddress = GetPhysicalAddressAndAllocatedSize(currentAddress, headAddress, currentPage, offset, out var allocatedSize); var recordInfo = LogRecord.GetInfo(physicalAddress); - if (recordInfo.IsNull) // We are probably past end of allocated records on page. - { - nextAddress = currentAddress + RecordInfo.GetLength(); - continue; - } - // If record does not fit on page, skip to the next page. - if (offset + allocatedSize > hlogBase.PageSize) + // If record does not fit on page, skip to the next page. Offset should always be at least PageHeader.Size; if it's zero, it means + // our record size aligned perfectly with end of page, so we must move to the next page (skipping its PageHeader). + if (offset == 0 || offset + allocatedSize > hlogBase.PageSize) { - nextAddress = hlogBase.GetStartLogicalAddressOfPage(1 + hlogBase.GetPage(currentAddress)); + var nextPage = hlogBase.GetPage(currentAddress); + nextAddress = hlogBase.GetFirstValidLogicalAddressOnPage(offset == 0 ? nextPage : nextPage + 1); continue; } nextAddress = currentAddress + allocatedSize; - var skipOnScan = includeClosedRecords ? false : recordInfo.SkipOnScan; + var skipOnScan = !includeClosedRecords && recordInfo.SkipOnScan; if (skipOnScan || recordInfo.IsNull) continue; if (currentAddress >= headAddress || assumeInMemory) { // TODO: for this PR we always buffer the in-memory records; pull iterators require it, and currently push iterators are implemented on top of pull. - // So create a disk log record from the in-memory record. - var logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress); - nextAddress = currentAddress + logRecord.GetInlineRecordSizes().allocatedSize; - - // We will return control to the caller, which means releasing epoch protection, and we don't want the caller to lock. - // Copy the entire record into bufferPool memory, so we do not have a ref to log data outside epoch protection. - // Lock to ensure no value tearing while copying to temp storage. + // Copy the entire record into bufferPool memory so we don't have a ref to log data outside epoch protection. OperationStackContext stackCtx = default; try { + // Lock to ensure no value tearing while copying to temp storage. if (currentAddress >= headAddress && store is not null) + { + var logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress, physicalAddress); store.LockForScan(ref stackCtx, logRecord.Key); - diskLogRecord.Serialize(in logRecord, hlogBase.bufferPool, valueSerializer: default, ref recordBuffer); + } + + if (recordBuffer == null) + recordBuffer = hlogBase.bufferPool.Get((int)allocatedSize); + else if (recordBuffer.AlignedTotalCapacity < (int)allocatedSize) + { + recordBuffer.Return(); + recordBuffer = hlogBase.bufferPool.Get((int)allocatedSize); + } + + // These objects are still alive in the log, so do not dispose the value object if any (SpanByteAllocator has none). + // Don't pass the recordBuffer to diskLogRecord; we reuse that here. + var remapPtr = recordBuffer.GetValidPointer(); + Buffer.MemoryCopy((byte*)physicalAddress, remapPtr, allocatedSize, allocatedSize); + var memoryLogRecord = hlogBase._wrapper.CreateRemappedLogRecordOverTransientMemory(currentAddress, (long)remapPtr); + diskLogRecord = new DiskLogRecord(in memoryLogRecord, obj => { }); } finally { @@ -241,10 +253,9 @@ public unsafe bool GetNext() else { // We advance a record at a time in the IO frame so set the diskLogRecord to the current frame offset and advance nextAddress. - diskLogRecord = new(physicalAddress); - if (diskLogRecord.Info.ValueIsObject) - _ = diskLogRecord.DeserializeValueObject(hlogBase.storeFunctions.CreateValueObjectSerializer()); - nextAddress = currentAddress + diskLogRecord.GetSerializedLength(); + // We dispose the object here because it is read from the disk, unless we transfer it such as by CopyToTail (SpanByteAllocator has no objects). + diskLogRecord = new(new LogRecord(physicalAddress, hlogBase._wrapper.TranssientObjectIdMap), + obj => store.storeFunctions.DisposeValueObject(obj, DisposeReason.DeserializedFromDisk)); } } finally @@ -279,7 +290,7 @@ bool IPushScanIterator.BeginGetPrevInMemory(ReadOnlySpan key, out LogRecor logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress); nextAddress = logRecord.Info.PreviousAddress; - bool skipOnScan = includeClosedRecords ? false : logRecord.Info.SkipOnScan; + var skipOnScan = !includeClosedRecords && logRecord.Info.SkipOnScan; if (skipOnScan || logRecord.Info.IsNull || !hlogBase.storeFunctions.KeysEqual(logRecord.Key, key)) { epoch?.Suspend(); @@ -294,35 +305,21 @@ bool IPushScanIterator.BeginGetPrevInMemory(ReadOnlySpan key, out LogRecor void IPushScanIterator.EndGetPrevInMemory() => epoch?.Suspend(); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - long GetPhysicalAddress(long currentAddress, long headAddress, long currentPage, long offset) - { - if (currentAddress >= headAddress || assumeInMemory) - return hlogBase._wrapper.CreateLogRecord(currentAddress).physicalAddress; - return frame.GetPhysicalAddress(currentPage, offset); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, long currentPage, long offset, out long allocatedSize) - { - if (currentAddress >= headAddress || assumeInMemory) - { - var logRecord = hlogBase._wrapper.CreateLogRecord(currentAddress); - (var _, allocatedSize) = logRecord.GetInlineRecordSizes(); - return logRecord.physicalAddress; - } - - long physicalAddress = frame.GetPhysicalAddress(currentPage, offset); - allocatedSize = new DiskLogRecord(physicalAddress).GetSerializedLength(); - return physicalAddress; - } - #region ISourceLogRecord /// public ref RecordInfo InfoRef => ref diskLogRecord.InfoRef; /// public RecordInfo Info => diskLogRecord.Info; + /// + public byte RecordType => diskLogRecord.RecordType; + + /// + public byte Namespace => diskLogRecord.Namespace; + + /// + public ObjectIdMap ObjectIdMap => diskLogRecord.ObjectIdMap; + /// public bool IsSet => diskLogRecord.IsSet; @@ -336,13 +333,17 @@ long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, l public byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; /// - public Span ValueSpan => diskLogRecord.ValueSpan; + public OverflowByteArray KeyOverflow + { + get => diskLogRecord.KeyOverflow; + set => diskLogRecord.KeyOverflow = value; + } /// - public IHeapObject ValueObject => diskLogRecord.ValueObject; + public Span ValueSpan => diskLogRecord.ValueSpan; /// - public ReadOnlySpan RecordSpan => diskLogRecord.RecordSpan; + public IHeapObject ValueObject => diskLogRecord.ValueObject; /// public bool IsPinnedValue => diskLogRecord.IsPinnedValue; @@ -350,6 +351,13 @@ long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, l /// public byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; + /// + public OverflowByteArray ValueOverflow + { + get => diskLogRecord.ValueOverflow; + set => diskLogRecord.ValueOverflow = value; + } + /// public long ETag => diskLogRecord.ETag; @@ -357,32 +365,23 @@ long GetPhysicalAddressAndAllocatedSize(long currentAddress, long headAddress, l public long Expiration => diskLogRecord.Expiration; /// - public void ClearValueObject(Action disposer) { } // Not relevant for iterators + public void ClearValueIfHeap(Action disposer) { } // Not relevant for "iterator as logrecord" /// - public bool AsLogRecord(out LogRecord logRecord) - { - logRecord = default; - return false; - } + public bool IsMemoryLogRecord => false; /// - public bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) - { - diskLogRecord = this.diskLogRecord; - return true; - } + public unsafe ref LogRecord AsMemoryLogRecordRef() => throw new InvalidOperationException("Cannot cast a DiskLogRecord to a memory LogRecord."); + + /// + public bool IsDiskLogRecord => true; + + /// + public unsafe ref DiskLogRecord AsDiskLogRecordRef() => ref Unsafe.AsRef(in diskLogRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public RecordFieldInfo GetRecordFieldInfo() => new() - { - KeyDataSize = Key.Length, - ValueDataSize = Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : ValueSpan.Length, - ValueIsObject = Info.ValueIsObject, - HasETag = Info.HasETag, - HasExpiration = Info.HasExpiration - }; + public RecordFieldInfo GetRecordFieldInfo() => diskLogRecord.GetRecordFieldInfo(); #endregion // ISourceLogRecord /// @@ -398,28 +397,21 @@ public override void Dispose() frame?.Dispose(); } - internal override void AsyncReadPagesFromDeviceToFrame(long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, + internal override void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) - => hlogBase.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice); + => hlogBase.AsyncReadPagesFromDeviceToFrame(readBuffers, readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice, cts); private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) { var result = (PageAsyncReadResult)context; - if (errorCode != 0) + if (errorCode == 0) + _ = result.handle?.Signal(); + else { logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); result.cts?.Cancel(); } - - // Deserialize valueObject in frame (if present) - var diskLogRecord = new DiskLogRecord(frame.GetPhysicalAddress(result.page, offset: 0)); - if (diskLogRecord.Info.ValueIsObject) - _ = diskLogRecord.DeserializeValueObject(hlogBase.storeFunctions.CreateValueObjectSerializer()); - - if (errorCode == 0) - _ = result.handle?.Signal(); - Interlocked.MemoryBarrier(); } } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs index 80c2528092c..f677ef85307 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs @@ -7,6 +7,7 @@ namespace Tsavorite.core { // This is unused; just allows things to build. TsavoriteLog does not do key comparisons or value operations; it is just a memory allocator +#pragma warning disable IDE0065 // Misplaced using directive using TsavoriteLogStoreFunctions = StoreFunctions; /// @@ -28,27 +29,13 @@ public readonly AllocatorBase GetBase => (AllocatorBase)(object)_this; - /// - public readonly bool IsFixedLength => true; - /// public readonly bool HasObjectLog => false; /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetStartLogicalAddress(long page) => _this.GetStartLogicalAddressOfPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddressOnPage(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void InitializeValue(long physicalAddress, in RecordSizeInfo _) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + public readonly void InitializeRecord(ReadOnlySpan key, long logicalAddress, in RecordSizeInfo _, ref LogRecord newLogRecord) + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -94,15 +81,6 @@ public readonly RecordSizeInfo GetUpsertRecordSize _this.AllocatePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly bool IsAllocated(int pageIndex) => _this.IsAllocated(pageIndex); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe void PopulatePage(byte* src, int required_bytes, long destinationPageIndex) - => TsavoriteLogAllocatorImpl.PopulatePage(src, required_bytes, destinationPageIndex); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void MarkPage(long logicalAddress, long version) => _this.MarkPage(logicalAddress, version); @@ -111,25 +89,13 @@ public readonly unsafe void PopulatePage(byte* src, int required_bytes, long des [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void MarkPageAtomic(long logicalAddress, long version) => _this.MarkPageAtomic(logicalAddress, version); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void ClearPage(long page, int offset = 0) => _this.ClearPage(page, offset); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly void FreePage(long pageIndex) => _this.FreePage(pageIndex); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); // TODO remove all the SegmentOffset stuff - /// public readonly int OverflowPageCount => _this.OverflowPageCount; - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ReadOnlySpan key, long physicalAddress, ref LogRecord logRecord) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly LogRecord CreateLogRecord(long logicalAddress) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); @@ -138,6 +104,12 @@ public readonly unsafe void PopulatePage(byte* src, int required_bytes, long des [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + /// + public readonly LogRecord CreateRemappedLogRecordOverTransientMemory(long logicalAddress, long physicalAddress) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + public readonly ObjectIdMap TranssientObjectIdMap => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs index 803b8651044..16167f0b1d8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs @@ -3,43 +3,32 @@ using System; using System.Diagnostics; -using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; using System.Threading; namespace Tsavorite.core { - using static LogAddress; - // This is unused; just allows things to build. TsavoriteLog does not do key comparisons or value operations; it is just a memory allocator. using TsavoriteLogStoreFunctions = StoreFunctions; /// Simple log allocator used by TsavoriteLog public sealed unsafe class TsavoriteLogAllocatorImpl : AllocatorBase { - // Circular buffer definition - private readonly byte[][] values; - private readonly long[] pointers; - private readonly long* nativePointers; - - private readonly OverflowPool> overflowPagePool; + private readonly OverflowPool> freePagePool; + /// Constructor +#pragma warning disable IDE0290 // Use primary constructor public TsavoriteLogAllocatorImpl(AllocatorSettings settings) : base(settings.LogSettings, new TsavoriteLogStoreFunctions(), @this => new TsavoriteLogAllocator(@this), evictCallback: null, settings.epoch, settings.flushCallback, settings.logger) { - overflowPagePool = new OverflowPool>(4, p => { }); - - if (BufferSize > 0) - { - values = new byte[BufferSize][]; - pointers = GC.AllocateArray(BufferSize, true); - nativePointers = (long*)Unsafe.AsPointer(ref pointers[0]); - } + freePagePool = new OverflowPool>(4, p => { }); } + /// public override void Reset() { base.Reset(); - for (int index = 0; index < BufferSize; index++) + for (var index = 0; index < BufferSize; index++) { if (IsAllocated(index)) FreePage(index); @@ -50,100 +39,71 @@ public override void Reset() void ReturnPage(int index) { Debug.Assert(index < BufferSize); - if (values[index] != null) + if (pagePointers[index] != default) { - _ = overflowPagePool.TryAdd(new PageUnit + _ = freePagePool.TryAdd(new() { - pointer = pointers[index], - value = values[index] + pointer = pagePointers[index], + value = Empty.Default }); - values[index] = null; - pointers[index] = 0; + pagePointers[index] = default; _ = Interlocked.Decrement(ref AllocatedPageCount); } } - public override void Initialize() => Initialize(FirstValidAddress); - /// /// Dispose memory allocator /// public override void Dispose() { base.Dispose(); - overflowPagePool.Dispose(); + freePagePool.Dispose(); } /// - /// Allocate memory page, pinned in memory, and in sector aligned form, if possible + /// Allocate memory page, pinned in memory /// /// internal void AllocatePage(int index) { IncrementAllocatedPageCount(); - if (overflowPagePool.TryGet(out var item)) + if (freePagePool.TryGet(out var item)) + pagePointers[index] = item.pointer; + else { - pointers[index] = item.pointer; - values[index] = item.value; - return; + // No free pages are available so allocate new + pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); + NativeMemory.Clear((void*)pagePointers[index], (nuint)PageSize); } - - var adjustedSize = PageSize + 2 * sectorSize; - - byte[] tmp = GC.AllocateArray(adjustedSize, true); - long p = (long)Unsafe.AsPointer(ref tmp[0]); - pointers[index] = (p + (sectorSize - 1)) & ~((long)sectorSize - 1); - values[index] = tmp; - } - - internal int OverflowPageCount => overflowPagePool.Count; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetPhysicalAddress(long logicalAddress) - { - // Index of page within the circular buffer, and offset on the page. TODO move this (and pagePointers) to AllocatorBase) - var pageIndex = GetPageIndexForAddress(logicalAddress); - var offset = GetOffsetOnPage(logicalAddress); - return *(nativePointers + pageIndex) + offset; + PageHeader.Initialize(pagePointers[index]); } - internal bool IsAllocated(int pageIndex) => values[pageIndex] != null; + internal int OverflowPageCount => freePagePool.Count; - protected override void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) + /// + protected override void WriteAsync(CircularDiskWriteBuffer flushBuffers, long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) { - WriteAsync((IntPtr)pointers[flushPage % BufferSize], + WriteInlinePageAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * flushPage), (uint)AlignedPageSizeBytes, callback, asyncResult, device); } - protected override void WriteAsyncToDevice - (long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, - PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long[] localSegmentOffsets, long fuzzyStartLogicalAddress) + /// + protected override void WriteAsyncToDevice(CircularDiskWriteBuffer flushBuffers, long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, + PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long fuzzyStartLogicalAddress) { VerifyCompatibleSectorSize(device); var alignedPageSize = (pageSize + (sectorSize - 1)) & ~(sectorSize - 1); - WriteAsync((IntPtr)pointers[flushPage % BufferSize], + WriteInlinePageAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), (uint)alignedPageSize, callback, asyncResult, device); } - internal void ClearPage(long page, int offset) - { - if (offset == 0) - Array.Clear(values[page % BufferSize], offset, values[page % BufferSize].Length - offset); - else - { - // Adjust array offset for cache alignment - offset += (int)(pointers[page % BufferSize] - (long)Unsafe.AsPointer(ref values[page % BufferSize][0])); - Array.Clear(values[page % BufferSize], offset, values[page % BufferSize].Length - offset); - } - } - internal void FreePage(long page) { ClearPage(page, 0); @@ -151,24 +111,12 @@ internal void FreePage(long page) ReturnPage((int)(page % BufferSize)); } - protected override void ReadAsync(ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, - DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device, IDevice objlogDevice) - => device.ReadAsync(alignedSourceAddress, (IntPtr)pointers[destinationPageIndex], aligned_read_length, callback, asyncResult); - - /// - /// Invoked by users to obtain a record from disk. It uses sector aligned memory to read - /// the record efficiently into memory. - /// - /// - /// - /// - /// - /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) - => throw new InvalidOperationException("AsyncReadRecordObjectsToMemory invalid for TsavoriteLogAllocator"); + protected override void ReadAsync(CircularDiskReadBuffer readBuffers, ulong alignedSourceAddress, IntPtr destinationPtr, uint aligned_read_length, + DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device) + => device.ReadAsync(alignedSourceAddress, destinationPtr, aligned_read_length, callback, asyncResult); - internal static void PopulatePage(byte* src, int required_bytes, long destinationPage) - => throw new TsavoriteException("TsavoriteLogAllocator memory pages are sector aligned - use direct copy"); + private protected override bool VerifyRecordFromDiskCallback(ref AsyncIOContext ctx, out long prevAddressToRead, out int prevLengthToRead) + => throw new TsavoriteException("TsavoriteLogAllocator does not support VerifyRecordFromDiskCallback"); /// /// Iterator interface for pull-scanning Tsavorite log @@ -188,13 +136,15 @@ internal override bool Scan(TsavoriteKV internal override bool ScanCursor(TsavoriteKV store, - ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress, bool resetCursor = true, bool includeTombstones = false) + ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress, + bool resetCursor = true, bool includeTombstones = false) => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// /// Implementation for push-iterating key versions, called from LogAccessor /// - internal override bool IterateKeyVersions(TsavoriteKV store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) + internal override bool IterateKeyVersions(TsavoriteKV store, ReadOnlySpan key, + long beginAddress, ref TScanFunctions scanFunctions) => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// @@ -233,7 +183,6 @@ internal void AsyncReadPagesFromDeviceToFrame( page = readPage, context = context, handle = completed, - frame = frame, cts = cts }; diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 2002dcf73a4..341f27eb588 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -433,7 +433,8 @@ public bool IterateLookup(ref TScanFunctions scanFunctions, ref { if (untilAddress == -1) untilAddress = store.Log.TailAddress; - return ScanCursor(ref cursor, count: long.MaxValue, scanFunctions, endAddress: untilAddress, validateCursor: validateCursor, maxAddress: maxAddress, resetCursor: resetCursor, includeTombstones: includeTombstones); + return ScanCursor(ref cursor, count: long.MaxValue, scanFunctions, endAddress: untilAddress, validateCursor: validateCursor, maxAddress: maxAddress, + resetCursor: resetCursor, includeTombstones: includeTombstones); } /// @@ -459,7 +460,8 @@ public bool IterateLookup(ref TScanFunctions scanFunctions, ref public bool ScanCursor(ref long cursor, long count, TScanFunctions scanFunctions, long endAddress = long.MaxValue, bool validateCursor = false, long maxAddress = long.MaxValue, bool resetCursor = true, bool includeTombstones = false) where TScanFunctions : IScanIteratorFunctions - => store.hlogBase.ScanCursor(store, scanCursorState ??= new(), ref cursor, count, scanFunctions, endAddress, validateCursor, maxAddress, resetCursor: resetCursor, includeTombstones: includeTombstones); + => store.hlogBase.ScanCursor(store, scanCursorState ??= new(), ref cursor, count, scanFunctions, endAddress, validateCursor, maxAddress, + resetCursor: resetCursor, includeTombstones: includeTombstones); /// /// Resume session on current thread. IMPORTANT: Call SuspendThread before any async op. diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs index 96199e9fd1d..a607257ed15 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs @@ -105,6 +105,7 @@ bool TryLock(ReadOnlySpan keys, TimeSpan t /// /// /// key whose lock is to be promoted. + /// On success, the caller must update the ILockableKey.LockType so the unlock has the right type bool TryPromoteLock(TTransactionalKey key) where TTransactionalKey : ITransactionalKey; diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs index 36b5dcd8906..42e7d92fe2d 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs @@ -74,8 +74,8 @@ public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadO public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input) => throw new NotImplementedException("IHeapObject value) is not supported in this ISessionFunctions implementation"); public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref TInput input) - where TSourceLogRecord : ISourceLogRecord - => new() { KeyDataSize = key.Length, ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject }; + where TSourceLogRecord : ISourceLogRecord + => new() { KeySize = key.Length, ValueSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject }; /// /// No reads during compaction diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs index 1219b8fcce2..43b152c97af 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs @@ -170,6 +170,11 @@ public bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, logRecord.InfoRef.SetTombstone(); status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord | StatusCode.Expired); } + else if (rmwInfo.Action == RMWAction.WrongType) + { + logRecord.InfoRef.SetTombstone(); + status = OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, StatusCode.WrongType); + } else status = OperationStatus.SUCCESS; return false; diff --git a/libs/storage/Tsavorite/cs/src/core/Device/IDevice.cs b/libs/storage/Tsavorite/cs/src/core/Device/IDevice.cs index 901438e3827..3e5d044f99f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Device/IDevice.cs +++ b/libs/storage/Tsavorite/cs/src/core/Device/IDevice.cs @@ -86,7 +86,7 @@ public interface IDevice : IDisposable /* Segmented addressing API */ /// - /// Write + /// Write to the file. The alignedSourceAddress must be pinned. /// /// /// @@ -94,10 +94,13 @@ public interface IDevice : IDisposable /// /// /// + /// While this supports concurrent writes, the caller should try as much as possible to sequentialize the writes, as the IDevice implementation + /// may require append-only behavior and thus will have to buffer. For similar reasons, do not back up and re-write; depending on the IDevice implementation, + /// this may fail or be inefficient. void WriteAsync(IntPtr sourceAddress, int segmentId, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context); /// - /// Read + /// Read from the file. The alignedSourceAddress must be pinned. /// /// /// @@ -110,17 +113,22 @@ public interface IDevice : IDisposable /* Direct addressing API */ /// - /// Write + /// Write to the file. The alignedSourceAddress must be pinned. If inheriting from , that provides an implementation of this that calculates the segmentId + /// and then invokes the overload with that segmentId. /// /// /// /// /// /// + /// While this supports concurrent writes, the caller should try as much as possible to sequentialize the writes, as the IDevice implementation + /// may require append-only behavior and thus will have to buffer. For similar reasons, do not back up and re-write; depending on the IDevice implementation, + /// this may fail or be inefficient. void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDestinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context); /// - /// Read + /// Read from the file. The alignedDestinationAddress must be pinned. If inheriting from , that provides an implementation of this that calculates the segmentId + /// and then invokes the overload with that segmentId. /// /// /// diff --git a/libs/storage/Tsavorite/cs/src/core/Device/LocalStorageDevice.cs b/libs/storage/Tsavorite/cs/src/core/Device/LocalStorageDevice.cs index a18b3222f97..0e6762d7ad6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Device/LocalStorageDevice.cs +++ b/libs/storage/Tsavorite/cs/src/core/Device/LocalStorageDevice.cs @@ -48,6 +48,13 @@ public unsafe class LocalStorageDevice : StorageDeviceBase private IntPtr ioCompletionPort; + /// + public override string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + return $"secSize {sectorSize}, numPend {numPending}, RO {bstr(readOnly)}, preAll {bstr(preallocateFile)}, delClose {bstr(deleteOnClose)}, noFiBuf {bstr(disableFileBuffering)}"; + } + /// /// Constructor /// diff --git a/libs/storage/Tsavorite/cs/src/core/Device/StorageDeviceBase.cs b/libs/storage/Tsavorite/cs/src/core/Device/StorageDeviceBase.cs index 4903dcf103f..56f620147ba 100644 --- a/libs/storage/Tsavorite/cs/src/core/Device/StorageDeviceBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Device/StorageDeviceBase.cs @@ -146,7 +146,7 @@ protected internal static string GetSegmentFilename(string filename, int segment public virtual bool Throttle() => false; /// - /// Write operation + /// Write operation; compute the segment id and call the IDevice implementation. /// /// /// @@ -163,7 +163,7 @@ public void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDestinationAddr } /// - /// Read operation + /// Read operation; compute the segment id and call the IDevice implementation. /// /// /// diff --git a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs index 3184eaae17b..4c164c30d1e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs +++ b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs @@ -183,7 +183,8 @@ public void ProtectAndDrain() public void Suspend() { Release(); - if (drainCount > 0) SuspendDrain(); + if (drainCount > 0) + SuspendDrain(); } /// @@ -196,6 +197,30 @@ public void Resume() ProtectAndDrain(); } + /// + /// Thread resumes its epoch entry if it has not already been acquired + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool ResumeIfNotProtected() + { + if (ThisInstanceProtected()) + return false; + Resume(); + return true; + } + + /// + /// Thread resumes its epoch entry if it has not already been acquired + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool SuspendIfProtected() + { + if (!ThisInstanceProtected()) + return false; + Suspend(); + return true; + } + /// /// Increment global current epoch /// @@ -408,9 +433,7 @@ static int ReserveEntry() // Try to acquire entry if (0 == (threadIndexAligned + Metadata.startOffset1)->threadId) { - if (0 == Interlocked.CompareExchange( - ref (threadIndexAligned + Metadata.startOffset1)->threadId, - Metadata.threadId, 0)) + if (0 == Interlocked.CompareExchange(ref (threadIndexAligned + Metadata.startOffset1)->threadId, Metadata.threadId, 0)) return Metadata.startOffset1; } @@ -424,7 +447,7 @@ static int ReserveEntry() if (Metadata.startOffset1 > kTableSize) { Metadata.startOffset1 -= kTableSize; - Thread.Yield(); + _ = Thread.Yield(); } } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 0ccd9848c24..57f2d59c492 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -75,14 +75,15 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv protected static void CollectMetadata(SystemState next, TsavoriteKV store) { - // Collect object log offsets only after flushes - // are completed +#if READ_WRITE + // Collect object log offsets only after flushes are completed var seg = store.hlog.GetSegmentOffsets(); if (seg != null) { store._hybridLogCheckpoint.info.objectLogSegmentOffsets = new long[seg.Length]; Array.Copy(seg, store._hybridLogCheckpoint.info.objectLogSegmentOffsets, seg.Length); } +#endif // READ_WRITE } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs index 743471b56bf..5d5369d1d4d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -49,7 +49,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri long startPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.snapshotStartFlushedLogicalAddress); long endPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.finalLogicalAddress); if (store._hybridLogCheckpoint.info.finalLogicalAddress > - store.hlog.GetStartLogicalAddress(endPage)) + store.hlogBase.GetLogicalAddressOfStartOfPage(endPage)) { endPage++; } @@ -58,7 +58,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri // handle corrupted or unexpected concurrent page changes during the flush, e.g., by // resuming epoch protection if necessary. Correctness is not affected as we will // only read safe pages during recovery. - store.hlogBase.AsyncFlushPagesToDevice( + store.hlogBase.AsyncFlushPagesForSnapshot( startPage, endPage, store._hybridLogCheckpoint.info.finalLogicalAddress, diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs index b2aee6d87dd..54f1397c0ad 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs @@ -111,7 +111,8 @@ internal void TransferFrom(ref TsavoriteKV { // Transfers the containers from the pendingContext, then null them; this is called before pendingContext.Dispose(). - keyContainer = new SpanByteHeapContainer(pendingContext.Key, bufferPool); + keyContainer = pendingContext.request_key; + pendingContext.request_key = null; inputContainer = pendingContext.input; pendingContext.input = default; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogAddress.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogAddress.cs index 6e5501963bf..6f41cb4af70 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogAddress.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogAddress.cs @@ -10,71 +10,39 @@ namespace Tsavorite.core /// Static utility class for manipulating logical addresses. public static class LogAddress { - // Address is 50 bits, with the top 2 being the address type. - public const int kAddressBits = 50; + /// Address is 48 bits, with the top bit being the readcache indicator. + public const int kAddressBits = 48; + /// Mask off the address from a long; e.g. is 48 bits, with the top bit being the readcache indicator. public const long kAddressBitMask = (1L << kAddressBits) - 1; - internal const int kAddressTypeBits = 2; // Address type bits (2 bits for 4 types, 1 currently reserved) - internal const int kAddressTypeBitOffset = kAddressBits - kAddressTypeBits; - internal const long kAddressTypeBitMask = ((1L << kAddressTypeBits) - 1) << kAddressTypeBitOffset; - // Get the absolute address by masking out the address type bits. - internal const long kAbsoluteAddressBitMask = (1L << (kAddressBits - kAddressTypeBits)) - 1; - - // AddressType is ordered in descending order with memory locations first, to match the sequence of - // comparisons in Internal(RUMD) and ensure OnDisk < InMemory: - // if (is in readcache) - // else (is in memory) - // else (is on disk) - // 00 is the lowest and is currently reserved - internal const long kReservedAddressTypeBitMask = 0L; - internal const long kIsReadCacheBitMask = 0b11L << kAddressTypeBitOffset; // 3 - internal const long kIsInLogMemoryBitMask = 0b10L << kAddressTypeBitOffset; // 2 - internal const long kIsOnDiskBitMask = 0b01L << kAddressTypeBitOffset; // 1 + internal const long kAbsoluteAddressBitMask = ((1L << kAddressBits) - 1) & ~RecordInfo.kIsReadCacheBitMask; + /// Invalid record logical address; used for initialization. Zero means an IsNull RecordInfo is Invalid. public const long kInvalidAddress = 0L; + /// Invalid record logical address used for some specific initializations. public const long kTempInvalidAddress = 1L; - /// FirstValidAddress is set for kIsInLogMemoryBitMask due to TsavoriteLog usage. ReadCache does not work with - /// TsavoriteLog, so we will override that in readCacheBase initialization. - public const long FirstValidAddress = 64L | kIsInLogMemoryBitMask; + /// First valid address in the log; ensures space for page header and that 0 and 1 are never valid addresses. + public const long FirstValidAddress = PageHeader.Size; /// The max valid address is the in-memory mask (which is greater than the on-disk mask) and the full absolute address range. - public const long MaxValidAddress = kAbsoluteAddressBitMask | kIsInLogMemoryBitMask; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetLogAddressType(bool isReadCache) => isReadCache ? kIsReadCacheBitMask : kIsInLogMemoryBitMask; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool IsReadCache(long address) => (address & kIsReadCacheBitMask) == kIsReadCacheBitMask; - // Unused in favor of allocatorBase.SetAddressType - //[MethodImpl(MethodImplOptions.AggressiveInlining)] - //internal static long SetIsReadCache(long address) => (address & kAbsoluteAddressBitMask) | kIsReadCacheBitMask; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool IsInLogMemory(long address) => (address & kIsInLogMemoryBitMask) == kIsInLogMemoryBitMask; - // Unused in favor of allocatorBase.SetAddressType - //[MethodImpl(MethodImplOptions.AggressiveInlining)] - //internal static long SetIsInLogMemory(long address) => (address & kAbsoluteAddressBitMask) | kIsInLogMemoryBitMask; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool IsOnDisk(long address) => (address & kIsOnDiskBitMask) == kIsOnDiskBitMask; - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long SetIsOnDisk(long address) => (address & kAbsoluteAddressBitMask) | kIsOnDiskBitMask; + public const long MaxValidAddress = kAbsoluteAddressBitMask; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool IsInMemory(long address) => (address & kAddressTypeBitMask) >= kIsInLogMemoryBitMask; + internal static bool IsReadCache(long address) => (address & RecordInfo.kIsReadCacheBitMask) == RecordInfo.kIsReadCacheBitMask; + /// Get the absolute address (no readcache bit) [MethodImpl(MethodImplOptions.AggressiveInlining)] public static long AbsoluteAddress(long address) => address & kAbsoluteAddressBitMask; /// Utility shared between AllocatorBase and ScanIteratorBase [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetPage(long logicalAddress, int logPageSizeBits) => AbsoluteAddress(logicalAddress) >> logPageSizeBits; + internal static long GetPageOfAddress(long logicalAddress, int logPageSizeBits) => AbsoluteAddress(logicalAddress) >> logPageSizeBits; /// Utility shared between AllocatorBase and ScanIteratorBase [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long GetStartAbsoluteLogicalAddressOfPage(long page, int logPageSizeBits) => page << logPageSizeBits; + internal static long GetLogicalAddressOfStartOfPage(long page, int logPageSizeBits) => page << logPageSizeBits; /// Pretty-print the address public static string AddressString(long address) @@ -82,15 +50,11 @@ public static string AddressString(long address) var absoluteAddress = AbsoluteAddress(address); if (IsReadCache(address)) return $"rc:{absoluteAddress}"; - if (IsInLogMemory(address)) - return $"log:{absoluteAddress}"; - if (IsOnDisk(address)) - return $"disk:{absoluteAddress}"; if (address == kInvalidAddress) return "kInvalid"; if (address == kTempInvalidAddress) return "kTempInvalid"; - return $"unk:{address}"; + return $"log:{address}"; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs index 8b2447f0e9b..064a16cd103 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs @@ -15,24 +15,45 @@ internal class LogSettings /// Maximum number of bits for a page size public const int kMaxPageSizeBits = 30; - /// Minimum number of bits for a segment (segments consist of one or more pages) - public const int kMinSegmentSizeBits = kMinPageSizeBits; - /// Maximum number of bits for a page size (segments consist of one or more pages) + /// Minimum number of bits for a main-log segment (segments consist of one or more pages) + public const int kMinMainLogSegmentSizeBits = kMinPageSizeBits; + /// Minimum number of bits for a segment (segments consist of one or more pages). This minimum size is also the size of the buffer, + /// so the segment must be a multiple of this (which is guaranteed as both are powers of 2, as long as this minimum is observed). + /// During flush we may create multiple buffers, depending on the degree of parallelism allowed by page concurrency and . + public const int kMinObjectLogSegmentSizeBits = 22; // 4MB + /// Maximum number of bits for a main-log or object-log segment (segments consist of one or more pages). This is also the size of the read/write buffers + /// for object serialization to the object log. public const int kMaxSegmentSizeBits = 62; /// Minimum number of bits for the size of the in-memory portion of the log - public const int kMinMemorySizeBits = kMinSegmentSizeBits; + public const int kMinMemorySizeBits = kMinPageSizeBits; /// Maximum number of bits for the size of the in-memory portion of the log public const int kMaxMemorySizeBits = kMaxSegmentSizeBits; + /// Minimum per flush operation. Must be a power of 2 + public const int kMinFlushBuffers = 2; + /// Maximum per flush operation. Must be a power of 2 + public const int kMaxFlushBuffers = 64; + + /// Minimum per flush operation. Must be a power of 2 + public const int kMinDeserializationBuffers = 2; + /// Maximum per flush operation. Must be a power of 2 + public const int kMaxDeserializationBuffers = 64; + /// Default number of bits for the size of an inline (not overflow) key public const int kDefaultMaxInlineKeySizeBits = kLowestMaxInlineSizeBits + 1; + /// Max inline key size is 1 byte for the length (0 or 1, with 1 added to make a range of 1-2), so that the in-memory varbyte indicator word is <= sizeof(long) for atomic assignment. + public const int kMaxInlineKeySize = 1 << 16; // 64KB + /// Default number of bits for the size of an inline (not overflow) value, for - public const int kDefaultMaxInlineValueSizeBits = 10; + public const int kDefaultMaxInlineValueSizeBits = 12; // 4KB + + /// Max inline value size is 2 bytes for the length (0 to 3, with 1 added to make a range of 1-4, and we max at 3), so that the in-memory varbyte indicator word is <= sizeof(long) for atomic assignment. + public const int kMaxInlineValueSize = 1 << 24; // 16MB /// Minimum number of bits for the size of an overflow (int inline) key or value - public const int kLowestMaxInlineSizeBits = kMinPageSizeBits - 1; + public const int kLowestMaxInlineSizeBits = kMinPageSizeBits; /// Maximum size of a string is 512MB public const int kMaxStringSizeBits = 29; @@ -53,19 +74,19 @@ internal class LogSettings public int PageSizeBits = 25; /// - /// Size of a segment (group of pages), in bits + /// Size of a segment (group of pages) in the main log, in bits /// public int SegmentSizeBits = 30; /// - /// Total size of in-memory part of log, in bits + /// Size of a segment (group of pages) in the object log, in bits /// - public int MemorySizeBits = 34; + public int ObjectLogSegmentSizeBits = 30; /// - /// Size of pages for ObjectAllocator key and optional fields space + /// Total size of in-memory part of log, in bits /// - public int ObjectLogVariableSpacePageSizeBits = 25; + public int MemorySizeBits = 34; /// /// Controls how many pages should be empty to account for non-power-of-two-sized log @@ -83,8 +104,7 @@ internal class LogSettings public ReadCopyOptions ReadCopyOptions; /// - /// Settings for optional read cache - /// Overrides the "copy reads to tail" setting + /// Settings for optional read cache. Overrides the "copy reads to tail" setting. /// public ReadCacheSettings ReadCacheSettings = null; @@ -102,5 +122,19 @@ internal class LogSettings /// Maximum size of a value stored inline in the in-memory portion of the main log for . /// public int MaxInlineValueSizeBits = kDefaultMaxInlineValueSizeBits; + + /// + /// Number of page buffers during a Flush operation on a page or portion of a page. There may be multiple sets of buffers at any given time, + /// depending on page parallelism. Must be a power of 2. + /// + /// Validated for all allocators, but only used by . + public int NumberOfFlushBuffers = 4; + + /// + /// Number of page buffers during a Flush operation on a page or portion of a page. There may be multiple sets of buffers at any given time, + /// depending on page parallelism. Must be a power of 2. + /// + /// Validated for all allocators, but only used by . + public int NumberOfDeserializationBuffers = 4; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs index 04723dcc208..e761fe8fbfc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs @@ -7,6 +7,7 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; public partial class TsavoriteKV : TsavoriteBase @@ -19,7 +20,7 @@ internal unsafe struct PendingContext : ISourceLogRec internal OperationType type; /// - /// DiskLogRecord carries either the input to RUMD operations or a log record image. It is used for: + /// DiskLogRecord carries a log record image. It is used for: /// /// Pending RUMD operations; in this case it contains only the key for all operations, and values for Upsert. /// Optionals (ETag and Expiration) are presumed to be carried in @@ -33,21 +34,34 @@ internal unsafe struct PendingContext : ISourceLogRec /// internal DiskLogRecord diskLogRecord; + /// The Key that was sent to this operation if it was RUMD. + internal SpanByteHeapContainer request_key; + /// The hash of if it is present. + internal long keyHash; + + /// The Input that was sent to this operation if it was RUMD. internal IHeapContainer input; + /// The Output to be returned from this operation if it was RUM. internal TOutput output; + /// The user Context that was sent to this operation if it was RUMD. internal TContext userContext; - internal long keyHash; - // Some additional information about the previous attempt + /// The id of this operation in the queue. internal long id; + + /// The logical address of the found record, if any; used to create . internal long logicalAddress; + + /// The initial highest logical address of the search; used to limit search ranges when the pending operation completes (e.g. to see if a duplicate was inserted). internal long initialLatestLogicalAddress; // operationFlags values internal ushort operationFlags; +#pragma warning disable IDE1006 // Naming Styles internal const ushort kNoOpFlags = 0; internal const ushort kIsNoKey = 0x0001; internal const ushort kIsReadAtAddress = 0x0002; +#pragma warning restore IDE1006 // Naming Styles internal ReadCopyOptions readCopyOptions; // Two byte enums @@ -96,6 +110,8 @@ public void Dispose() { diskLogRecord.Dispose(); diskLogRecord = default; + request_key?.Dispose(); + request_key = default; input?.Dispose(); input = default; } @@ -117,67 +133,56 @@ internal void SerializeForReadOrRMW(ReadOnlySpan { if (diskLogRecord.IsSet) return; - diskLogRecord.SerializeForPendingReadOrRMW(key, bufferPool); - CopyIOC(ref input, output, userContext, sessionFunctions); - } + request_key?.Dispose(); + request_key = new(key, bufferPool); - /// - /// Serialize a or and Input, Output, and userContext into the local for Pending operations - /// - /// The log record. This may be either in-memory or from disk IO - /// Input to the operation - /// Output from the operation - /// User context for the operation - /// Session functions wrapper for the operation - /// Allocator for backing storage - /// Serializer for value object (if any); if null, the object is to be held as an object (e.g. for Pending IO operations) - /// rather than serialized to a byte stream (e.g. for out-of-process operations) - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Serialize(in TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, TContext userContext, TSessionFunctionsWrapper sessionFunctions, - SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - where TSourceLogRecord : ISourceLogRecord - { - Serialize(in srcLogRecord, bufferPool, valueSerializer); - CopyIOC(ref input, output, userContext, sessionFunctions); + if (this.input == default) + { + if (typeof(TInput) == typeof(PinnedSpanByte)) + this.input = new SpanByteHeapContainer(Unsafe.As(ref input), sessionFunctions.Store.hlogBase.bufferPool) as IHeapContainer; + else + this.input = new StandardHeapContainer(ref input); + } + this.output = output; + sessionFunctions.ConvertOutputToHeap(ref input, ref this.output); + this.userContext = userContext; } /// - /// Serialize a or into the local for Pending operations + /// Does an in-memory transfer of a record into the pending context. The transfer operates based on the implementation of the : + /// + /// If it is a , it comes from the log or possibly an iterator; we copy the inline record data into a + /// and then reassign its ObjectIds to the . + /// Otherwise it is a and we can transfer its into the local . + /// It will already have a in its contained . + /// /// /// The log record to be copied into the . This may be either in-memory or from disk IO + /// + /// + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Serialize(in TSourceLogRecord srcLogRecord, SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer) + internal void TransferFrom(in TSourceLogRecord srcLogRecord, SectorAlignedBufferPool bufferPool, ObjectIdMap transientObjectIdMap, Action objectDisposer) where TSourceLogRecord : ISourceLogRecord { Debug.Assert(!diskLogRecord.IsSet, "Should not try to reset PendingContext.diskLogRecord"); - if (srcLogRecord.AsLogRecord(out var logRecord)) + if (srcLogRecord.IsMemoryLogRecord) { - diskLogRecord.Serialize(in logRecord, bufferPool, valueSerializer); + ref var memoryLogRecord = ref srcLogRecord.AsMemoryLogRecordRef(); + diskLogRecord = DiskLogRecord.CopyFrom(in memoryLogRecord, bufferPool, transientObjectIdMap, objectDisposer); return; } // If the inputDiskLogRecord owns its memory, transfer it to the local diskLogRecord; otherwise we need to deep copy. - _ = srcLogRecord.AsDiskLogRecord(out var inputDiskLogRecord); - if (inputDiskLogRecord.OwnsMemory) - diskLogRecord.TransferFrom(ref inputDiskLogRecord); - else - diskLogRecord.CloneFrom(ref inputDiskLogRecord, bufferPool, preferDeserializedObject: true); + Debug.Assert(srcLogRecord.IsDiskLogRecord, $"Unknown SrcLogRecord implementation: {srcLogRecord}"); + ref var inputDiskLogRecord = ref srcLogRecord.AsDiskLogRecordRef(); + diskLogRecord = DiskLogRecord.TransferFrom(ref inputDiskLogRecord); } - private void CopyIOC(ref TInput input, TOutput output, TContext userContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal void TransferFrom(ref DiskLogRecord inputDiskLogRecord) { - if (this.input == default) - { - if (typeof(TInput) == typeof(PinnedSpanByte)) - this.input = new SpanByteHeapContainer(Unsafe.As(ref input), sessionFunctions.Store.hlogBase.bufferPool) as IHeapContainer; - else - this.input = new StandardHeapContainer(ref input); - } - this.output = output; - sessionFunctions.ConvertOutputToHeap(ref input, ref this.output); - this.userContext = userContext; + Debug.Assert(!diskLogRecord.IsSet, "Should not try to reset PendingContext.diskLogRecord"); + diskLogRecord = DiskLogRecord.TransferFrom(ref inputDiskLogRecord); } #endregion // Serialized Record Creation @@ -188,6 +193,15 @@ private void CopyIOC(ref TInput input, TOutput output, /// public readonly RecordInfo Info => diskLogRecord.Info; + /// + public byte RecordType => diskLogRecord.RecordType; + + /// + public byte Namespace => diskLogRecord.Namespace; + + /// + public readonly ObjectIdMap ObjectIdMap => diskLogRecord.ObjectIdMap; + /// public readonly bool IsSet => diskLogRecord.IsSet; @@ -198,7 +212,14 @@ private void CopyIOC(ref TInput input, TOutput output, public readonly bool IsPinnedKey => diskLogRecord.IsPinnedKey; /// - public byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; + public readonly byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; + + /// + public OverflowByteArray KeyOverflow + { + readonly get => diskLogRecord.KeyOverflow; + set => diskLogRecord.KeyOverflow = value; + } /// public readonly unsafe Span ValueSpan => diskLogRecord.ValueSpan; @@ -207,13 +228,17 @@ private void CopyIOC(ref TInput input, TOutput output, public readonly IHeapObject ValueObject => diskLogRecord.ValueObject; /// - public ReadOnlySpan RecordSpan => diskLogRecord.RecordSpan; + public readonly bool IsPinnedValue => diskLogRecord.IsPinnedValue; /// - public bool IsPinnedValue => diskLogRecord.IsPinnedValue; + public readonly byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; /// - public byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; + public OverflowByteArray ValueOverflow + { + readonly get => diskLogRecord.ValueOverflow; + set => diskLogRecord.ValueOverflow = value; + } /// public readonly long ETag => diskLogRecord.ETag; @@ -222,28 +247,26 @@ private void CopyIOC(ref TInput input, TOutput output, public readonly long Expiration => diskLogRecord.Expiration; /// - public readonly void ClearValueObject(Action disposer) { } // Not relevant for PendingContext + public readonly void ClearValueIfHeap(Action disposer) { } // Not relevant for PendingContext /// - public readonly bool AsLogRecord(out LogRecord logRecord) - { - logRecord = default; - return false; - } + public readonly bool IsMemoryLogRecord => diskLogRecord.IsMemoryLogRecord; /// - public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) - { - diskLogRecord = this.diskLogRecord; - return true; - } + public readonly unsafe ref LogRecord AsMemoryLogRecordRef() => ref diskLogRecord.AsMemoryLogRecordRef(); + + /// + public readonly bool IsDiskLogRecord => true; + + /// + public readonly unsafe ref DiskLogRecord AsDiskLogRecordRef() => ref Unsafe.AsRef(in diskLogRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] public readonly RecordFieldInfo GetRecordFieldInfo() => new() { - KeyDataSize = Key.Length, - ValueDataSize = Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : ValueSpan.Length, + KeySize = Key.Length, + ValueSize = Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : ValueSpan.Length, ValueIsObject = Info.ValueIsObject, HasETag = Info.HasETag, HasExpiration = Info.HasExpiration diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs index 5e3f79add97..4e616dda8cc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs @@ -9,23 +9,28 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; // RecordInfo layout (64 bits total, high to low): // RecordInfo bits: - // [Unused1][Modified][InNewVersion][Filler][Dirty][Unused2][Sealed][Valid][Tombstone] - // [HasExpiration][HasETag][ValueIsObject][ValueIsInline][KeyIsInline] + // [Unused1][Modified][InNewVersion][Unused2][Dirty][Unused3][Sealed][Valid][Tombstone] + // [HasExpiration][HasETag][ValueIsObject][ValueIsInline][KeyIsInline][Unused4][Unused5] // LogAddress bits (where A = address): - // [AddressTypeHigh][AddressTypeLow] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] + // [R][AAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [StructLayout(LayoutKind.Explicit, Size = 8)] public struct RecordInfo { + public const int Size = sizeof(long); + #pragma warning disable IDE1006 // Naming Styles: Must begin with uppercase letter - const int kTotalSizeInBytes = 8; - const int kTotalBits = kTotalSizeInBytes * 8; + const int kTotalBits = Size * 8; - // Other marker bits. Unused* means bits not yet assigned; use the highest number when assigning - const int kKeyIsInlineBitOffset = kAddressBits; + // Other marker bits. Unused* means bits not yet assigned + const int kIsReadCacheBitOffset = kAddressBits - 1; + const int kUnused5BitOffset = kIsReadCacheBitOffset + 1; + const int kUnused4BitOffset = kUnused5BitOffset + 1; + const int kKeyIsInlineBitOffset = kUnused4BitOffset + 1; const int kValueIsInlineBitOffset = kKeyIsInlineBitOffset + 1; const int kValueIsObjectBitOffset = kValueIsInlineBitOffset + 1; const int kHasETagBitOffset = kValueIsObjectBitOffset + 1; @@ -33,13 +38,16 @@ public struct RecordInfo const int kTombstoneBitOffset = kHasExpirationBitOffset + 1; const int kValidBitOffset = kTombstoneBitOffset + 1; const int kSealedBitOffset = kValidBitOffset + 1; - const int kUnused2BitOffset = kSealedBitOffset + 1; - const int kDirtyBitOffset = kUnused2BitOffset + 1; - const int kFillerBitOffset = kDirtyBitOffset + 1; - const int kInNewVersionBitOffset = kFillerBitOffset + 1; + const int kUnused3BitOffset = kSealedBitOffset + 1; + const int kDirtyBitOffset = kUnused3BitOffset + 1; + const int kUnused2BitOffset = kDirtyBitOffset + 1; + const int kInNewVersionBitOffset = kUnused2BitOffset + 1; const int kModifiedBitOffset = kInNewVersionBitOffset + 1; const int kUnused1BitOffset = kModifiedBitOffset + 1; + internal const long kIsReadCacheBitMask = 1L << kIsReadCacheBitOffset; + const long kUnused5BitMask = 1L << kUnused5BitOffset; + const long kUnused4BitMask = 1L << kUnused4BitOffset; const long kKeyIsInlineBitMask = 1L << kKeyIsInlineBitOffset; const long kValueIsInlineBitMask = 1L << kValueIsInlineBitOffset; const long kValueIsObjectBitMask = 1L << kValueIsObjectBitOffset; @@ -48,9 +56,9 @@ public struct RecordInfo const long kTombstoneBitMask = 1L << kTombstoneBitOffset; const long kValidBitMask = 1L << kValidBitOffset; const long kSealedBitMask = 1L << kSealedBitOffset; - const long kUnused2BitMask = 1L << kUnused2BitOffset; + const long kUnused3BitMask = 1L << kUnused3BitOffset; const long kDirtyBitMask = 1L << kDirtyBitOffset; - const long kFillerBitMask = 1L << kFillerBitOffset; + const long kUnused2BitMask = 1L << kUnused2BitOffset; const long kInNewVersionBitMask = 1L << kInNewVersionBitOffset; const long kModifiedBitMask = 1L << kModifiedBitOffset; const long kUnused1BitMask = 1L << kUnused1BitOffset; @@ -214,18 +222,6 @@ public bool Modified } } - public readonly bool HasFiller - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get => (word & kFillerBitMask) > 0; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void SetHasFiller() => word |= kFillerBitMask; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ClearHasFiller() => word &= ~kFillerBitMask; - public readonly bool IsInNewVersion { [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -291,13 +287,16 @@ public long PreviousAddress public void SetHasExpiration() => word |= kHasExpirationBitMask; public void ClearHasExpiration() => word &= ~kHasExpirationBitMask; + public readonly bool HasOptionalFields => (word & (kHasETagBitMask | kHasExpirationBitMask)) != 0; + // Note: KeyIsOveflow bit is not needed as it is the negation of KeyIsInline public readonly bool KeyIsInline => (word & kKeyIsInlineBitMask) != 0; public void SetKeyIsInline() => word |= kKeyIsInlineBitMask; public void ClearKeyIsInline() => word &= ~kKeyIsInlineBitMask; public readonly bool KeyIsOverflow => !KeyIsInline; + public void SetKeyIsOverflow() => word &= ~kKeyIsInlineBitMask; - // Note: ValueIsOveflow bit is not needed as it is the negation of (ValueIsInline | ValueIsObject) + // Note: a ValueIsOverflow bit is not needed as it is the negation of (ValueIsInline | ValueIsObject) public readonly bool ValueIsInline => (word & kValueIsInlineBitMask) != 0; public void SetValueIsInline() => word = (word & ~kValueIsObjectBitMask) | kValueIsInlineBitMask; public void ClearValueIsInline() => word &= ~kValueIsInlineBitMask; @@ -305,14 +304,21 @@ public long PreviousAddress public readonly bool ValueIsObject => (word & kValueIsObjectBitMask) != 0; public void SetValueIsObject() => word = (word & ~kValueIsInlineBitMask) | kValueIsObjectBitMask; - // "Overflow" is determined by lack of Inline and lack of Object + // Value "Overflow" is determined by lack of Inline and lack of Object public readonly bool ValueIsOverflow => !ValueIsInline && !ValueIsObject; public void SetValueIsOverflow() => word &= ~(kValueIsInlineBitMask | kValueIsObjectBitMask); + public void SetKeyAndValueInline() => word = (word & ~kValueIsObjectBitMask) | kKeyIsInlineBitMask | kValueIsInlineBitMask; + public readonly bool RecordIsInline => (word & (kKeyIsInlineBitMask | kValueIsInlineBitMask)) == (kKeyIsInlineBitMask | kValueIsInlineBitMask); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int GetLength() => kTotalSizeInBytes; + public readonly bool RecordHasObjects => (word & (kKeyIsInlineBitMask | kValueIsInlineBitMask)) != (kKeyIsInlineBitMask | kValueIsInlineBitMask); + + internal bool IsReadCache + { + readonly get => (word & kIsReadCacheBitMask) != 0; + set => word = value ? word | kIsReadCacheBitMask : word & ~kIsReadCacheBitMask; + } internal bool Unused1 { @@ -326,13 +332,30 @@ internal bool Unused2 set => word = value ? word | kUnused2BitMask : word & ~kUnused2BitMask; } + internal bool Unused3 + { + readonly get => (word & kUnused3BitMask) != 0; + set => word = value ? word | kUnused3BitMask : word & ~kUnused3BitMask; + } + + internal bool Unused4 + { + readonly get => (word & kUnused4BitMask) != 0; + set => word = value ? word | kUnused4BitMask : word & ~kUnused4BitMask; + } + + internal bool Unused5 + { + readonly get => (word & kUnused5BitMask) != 0; + set => word = value ? word | kUnused5BitMask : word & ~kUnused5BitMask; + } + public override readonly string ToString() { - var paRC = IsReadCache(PreviousAddress) ? "(rc)" : string.Empty; static string bstr(bool value) => value ? "T" : "F"; - return $"prev {AddressString(PreviousAddress)}{paRC}, valid {bstr(Valid)}, tomb {bstr(Tombstone)}, seal {bstr(IsSealed)}," - + $" mod {bstr(Modified)}, dirty {bstr(Dirty)}, fill {bstr(HasFiller)}, KisInl {KeyIsInline}, VisInl {ValueIsInline}, VisObj {bstr(ValueIsObject)}," - + $" ETag {bstr(HasETag)}, Expir {bstr(HasExpiration)}, Un1 {bstr(Unused1)}, Un2 {bstr(Unused2)}"; + return $"prev {AddressString(PreviousAddress)}, valid {bstr(Valid)}, tomb {bstr(Tombstone)}, seal {bstr(IsSealed)}, isRC {bstr(IsReadCache)}," + + $" mod {bstr(Modified)}, dirty {bstr(Dirty)}, KisInl {KeyIsInline}, VisInl {ValueIsInline}, VisObj {bstr(ValueIsObject)}," + + $" ETag {bstr(HasETag)}, Expir {bstr(HasExpiration)}, Un1 {bstr(Unused1)}, Un2 {bstr(Unused2)}, Un3 {bstr(Unused3)}, Un4 {bstr(Unused4)}, Un5 {bstr(Unused5)}"; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs index ff9fe253cb7..74a02464dc4 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs @@ -82,6 +82,11 @@ public enum RMWAction /// ExpireAndStop, + /// + /// Stop the operation immediately with a "wrong type" error + /// + WrongType, + /// /// Stop the operation immediately and return. /// @@ -120,7 +125,7 @@ public struct RMWInfo public bool PreserveCopyUpdaterSourceRecord { get; set; } /// - /// If set true by CopyUpdater, the source ValueObject will be cleared immediately (to manage object size tracking most effectively). + /// If set true by RMW and there is a source ValueObject it will be cleared immediately (to manage object size tracking most effectively). /// public bool ClearSourceValueObject { get; set; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs index 76505e1f374..0c42088173a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs @@ -94,8 +94,7 @@ public virtual bool PostCopyUpdater(in TSourceLogRecord srcLog /// public virtual bool InitialDeleter(ref LogRecord dstLogRecord, ref DeleteInfo deleteInfo) { - if (dstLogRecord.Info.ValueIsObject) - dstLogRecord.ClearValueObject(_ => { }); + dstLogRecord.ClearValueIfHeap(_ => { }); return true; } public virtual void PostInitialDeleter(ref LogRecord dstLogRecord, ref DeleteInfo deleteInfo) { } @@ -115,14 +114,14 @@ public virtual RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, re => throw new NotImplementedException("GetRMWInitialFieldInfo requires knowledge of TInput"); /// public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref TInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length, ValueIsObject = false }; + => new() { KeySize = key.Length, ValueSize = value.Length, ValueIsObject = false }; /// public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; /// public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref TInput input) where TSourceLogRecord : ISourceLogRecord - => new() { KeyDataSize = key.Length, ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject }; + => new() { KeySize = key.Length, ValueSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject }; /// public virtual void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs index f884cd1660b..0eb567d83c3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System; -using System.Diagnostics; using System.Runtime.CompilerServices; using System.Runtime.InteropServices; using System.Threading; @@ -75,7 +74,9 @@ public sealed class DeltaLog : ScanIteratorBase, IDisposable /// Constructor /// public DeltaLog(IDevice deltaLogDevice, int logPageSizeBits, long tailAddress, ILogger logger = null) - : base(0, tailAddress >= 0 ? tailAddress : deltaLogDevice.GetFileSize(0), DiskScanBufferingMode.SinglePageBuffering, InMemoryScanBufferingMode.NoBuffering, includeClosedRecords: false, default, logPageSizeBits, false, logger: logger) + : base(readBuffers: default, // TODO need to pass in or create this iff ObjectAllocator - but this was disabled for GenericAllocator; need to dig into what's needed for support + 0, tailAddress >= 0 ? tailAddress : deltaLogDevice.GetFileSize(0), DiskScanBufferingMode.SinglePageBuffering, + InMemoryScanBufferingMode.NoBuffering, includeClosedRecords: false, default, logPageSizeBits, false, logger: logger) { LogPageSizeBits = logPageSizeBits; PageSize = 1 << LogPageSizeBits; @@ -117,7 +118,8 @@ public override void Dispose() } } - internal override void AsyncReadPagesFromDeviceToFrame(long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) + internal override void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer readBuffers, long readPageStart, int numPages, long untilAddress, + TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) { IDevice usedDevice = deltaLogDevice; completed = new CountdownEvent(numPages); @@ -133,8 +135,7 @@ internal override void AsyncReadPagesFromDeviceToFrame(long readPageSt { page = readPage, context = context, - handle = completed, - frame = frame + handle = completed }; ulong offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); @@ -167,10 +168,9 @@ private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); result.cts?.Cancel(); } - Debug.Assert(result.freeBuffer1 == null); if (errorCode == 0) - result.handle?.Signal(); + _ = result.handle?.Signal(); Interlocked.MemoryBarrier(); } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs index 1c128bd95c4..ae5ece8bf64 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs @@ -228,11 +228,11 @@ public LogFileInfo GetLogFileSize(Guid token, long version = -1) using var current = new HybridLogCheckpointInfo(); // We find the latest checkpoint metadata for the given token, including scanning the delta log for the latest metadata current.Recover(token, checkpointManager, hlogBase.LogPageSizeBits, out var _, true, version); - long snapshotDeviceOffset = hlogBase.GetStartAbsoluteLogicalAddressOfPage(hlogBase.GetPage(current.info.snapshotStartFlushedLogicalAddress)); + long snapshotDeviceOffset = hlogBase.GetLogicalAddressOfStartOfPage(hlogBase.GetPage(current.info.snapshotStartFlushedLogicalAddress)); return new LogFileInfo { snapshotFileEndAddress = current.info.snapshotFinalLogicalAddress - snapshotDeviceOffset, - hybridLogFileStartAddress = hlogBase.GetStartAbsoluteLogicalAddressOfPage(hlogBase.GetPage(current.info.beginAddress)), + hybridLogFileStartAddress = hlogBase.GetLogicalAddressOfStartOfPage(hlogBase.GetPage(current.info.beginAddress)), hybridLogFileEndAddress = current.info.flushedLogicalAddress, deltaLogTailAddress = current.info.deltaTailAddress, }; @@ -447,7 +447,7 @@ private long InternalRecover(IndexCheckpointInfo recoveredICInfo, HybridLogCheck { hlogBase.VerifyRecoveryInfo(recoveredHLCInfo, false); - if (hlogBase.GetTailAddress() > hlog.GetFirstValidLogicalAddress(0)) + if (hlogBase.GetTailAddress() > hlogBase.GetFirstValidLogicalAddressOnPage(0)) { logger?.LogInformation("Recovery called on non-empty log - resetting to empty state first. Make sure store is quiesced before calling Recover on a running store."); Reset(); @@ -493,7 +493,7 @@ private async ValueTask InternalRecoverAsync(IndexCheckpointInfo recovered { hlogBase.VerifyRecoveryInfo(recoveredHLCInfo, false); - if (hlogBase.GetTailAddress() > hlog.GetFirstValidLogicalAddress(0)) + if (hlogBase.GetTailAddress() > hlogBase.GetFirstValidLogicalAddressOnPage(0)) { logger?.LogInformation("Recovery called on non-empty log - resetting to empty state first. Make sure store is quiesced before calling Recover on a running store."); Reset(); @@ -540,12 +540,12 @@ private async ValueTask InternalRecoverAsync(IndexCheckpointInfo recovered private void DoPostRecovery(IndexCheckpointInfo recoveredICInfo, HybridLogCheckpointInfo recoveredHLCInfo, long tailAddress, ref long headAddress, ref long readOnlyAddress, long lastFreedPage) { // Adjust head and read-only address post-recovery - var _head = hlogBase.GetStartLogicalAddressOfPage(1 + hlogBase.GetPage(tailAddress) - (hlogBase.GetCapacityNumPages() - hlogBase.MinEmptyPageCount)); + var _head = hlogBase.GetLogicalAddressOfStartOfPage(1 + hlogBase.GetPage(tailAddress) - (hlogBase.GetCapacityNumPages() - hlogBase.MinEmptyPageCount)); // If additional pages have been freed to accommodate heap memory constraints, adjust head address accordingly if (lastFreedPage != NoPageFreed) { - var nextAddress = hlogBase.GetStartLogicalAddressOfPage(lastFreedPage + 1); + var nextAddress = hlogBase.GetLogicalAddressOfStartOfPage(lastFreedPage + 1); if (_head < nextAddress) _head = nextAddress; } @@ -626,17 +626,13 @@ private bool SetRecoveryPageRanges(HybridLogCheckpointInfo recoveredHLCInfo, int return false; } - // Recover segment offsets for object log - if (recoveredHLCInfo.info.objectLogSegmentOffsets != null) - Array.Copy(recoveredHLCInfo.info.objectLogSegmentOffsets, - hlog.GetSegmentOffsets(), - recoveredHLCInfo.info.objectLogSegmentOffsets.Length); + RestoreMetadata(recoveredHLCInfo); tailAddress = recoveredHLCInfo.info.finalLogicalAddress; headAddress = recoveredHLCInfo.info.headAddress; if (numPagesToPreload != -1) { - var head = hlogBase.GetStartLogicalAddressOfPage(hlogBase.GetPage(tailAddress) - numPagesToPreload); + var head = hlogBase.GetLogicalAddressOfStartOfPage(hlogBase.GetPage(tailAddress) - numPagesToPreload); if (head > headAddress) headAddress = head; } @@ -658,6 +654,17 @@ private bool SetRecoveryPageRanges(HybridLogCheckpointInfo recoveredHLCInfo, int return true; } + private void RestoreMetadata(HybridLogCheckpointInfo recoveredHLCInfo) + { +#if READ_WRITE + // Recover segment offsets for object log + if (recoveredHLCInfo.info.objectLogSegmentOffsets != null) + Array.Copy(recoveredHLCInfo.info.objectLogSegmentOffsets, + hlog.GetSegmentOffsets(), + recoveredHLCInfo.info.objectLogSegmentOffsets.Length); +#endif // READ_WRITE + } + /// /// This method ensures that before 'pagesToRead' number of pages are read into memory, any previously allocated pages /// that would cause total number of pages in memory to go beyond usableCapacity are freed. This is to ensure that @@ -676,7 +683,7 @@ private void FreePagesBeyondUsableCapacity(long startPage, int capacity, int usa for (var page = beg; page < end; page++) { var pageIndex = hlogBase.GetPageIndexForPage(page); - if (hlog.IsAllocated(pageIndex)) + if (hlogBase.IsAllocated(pageIndex)) { recoveryStatus.WaitFlush(pageIndex); hlogBase.EvictPage(page); @@ -690,12 +697,13 @@ private void ReadPagesWithMemoryConstraint(long endAddress, int capacity, Recove // constraint are freed. FreePagesBeyondUsableCapacity(startPage: page, capacity: capacity, usableCapacity: capacity - hlogBase.MinEmptyPageCount, pagesToRead: numPagesToRead, recoveryStatus); + using var readBuffers = hlogBase.CreateCircularReadBuffers(recoveryStatus.objectLogRecoveryDevice, logger); + // Issue request to read pages as much as possible for (var p = page; p < endPage; p++) recoveryStatus.readStatus[hlogBase.GetPageIndexForPage(p)] = ReadStatus.Pending; - hlogBase.AsyncReadPagesFromDevice(page, numPagesToRead, endAddress, + hlogBase.AsyncReadPagesForRecovery(readBuffers, page, numPagesToRead, endAddress, hlogBase.AsyncReadPagesCallbackForRecovery, - recoveryStatus, recoveryStatus.recoveryDevicePageOffset, - recoveryStatus.recoveryDevice, recoveryStatus.objectLogRecoveryDevice); + recoveryStatus, recoveryStatus.recoveryDevicePageOffset, recoveryStatus.recoveryDevice); } private long FreePagesToLimitHeapMemory(RecoveryStatus recoveryStatus, long page) @@ -708,7 +716,7 @@ private long FreePagesToLimitHeapMemory(RecoveryStatus recoveryStatus, long page for (var p = Math.Max(0, page - recoveryStatus.usableCapacity + 1); p < page && hlogBase.IsSizeBeyondLimit(); p++) { var pageIndex = hlogBase.GetPageIndexForPage(p); - if (hlog.IsAllocated(pageIndex)) + if (hlogBase.IsAllocated(pageIndex)) { recoveryStatus.WaitFlush(pageIndex); hlogBase.EvictPage(p); @@ -763,7 +771,7 @@ private async Task FreePagesToLimitHeapMemoryAsync(RecoveryStatus recovery for (var p = Math.Max(0, page - recoveryStatus.usableCapacity + 1); p < page && hlogBase.IsSizeBeyondLimit(); p++) { var pageIndex = hlogBase.GetPageIndexForPage(p); - if (hlog.IsAllocated(pageIndex)) + if (hlogBase.IsAllocated(pageIndex)) { await recoveryStatus.WaitFlushAsync(pageIndex, cancellationToken); hlogBase.EvictPage(p); @@ -843,7 +851,7 @@ private RecoveryStatus GetPageRangesToRead(long scanFromAddress, long untilAddre { startPage = hlogBase.GetPage(scanFromAddress); endPage = hlogBase.GetPage(untilAddress); - if (untilAddress > hlog.GetStartLogicalAddress(endPage) && untilAddress > scanFromAddress) + if (untilAddress > hlogBase.GetLogicalAddressOfStartOfPage(endPage) && untilAddress > scanFromAddress) { endPage++; } @@ -862,7 +870,7 @@ private void ProcessReadPageAndFlush(long recoverFromAddress, long untilAddress, if (ProcessReadPage(recoverFromAddress, untilAddress, nextVersion, options, recoveryStatus, page, pageIndex)) { // Page was modified due to undoFutureVersion. Flush it to disk; the callback issues the after-capacity read request if necessary. - hlogBase.AsyncFlushPages(page, 1, AsyncFlushPageCallbackForRecovery, recoveryStatus); + hlogBase.AsyncFlushPagesForRecovery(page, 1, AsyncFlushPageCallbackForRecovery, recoveryStatus); return; } @@ -872,9 +880,9 @@ private void ProcessReadPageAndFlush(long recoverFromAddress, long untilAddress, private bool ProcessReadPage(long recoverFromAddress, long untilAddress, long nextVersion, RecoveryOptions options, RecoveryStatus recoveryStatus, long page, int pageIndex) { - var startLogicalAddress = hlog.GetStartLogicalAddress(page); - var endLogicalAddress = hlog.GetStartLogicalAddress(page + 1); - var physicalAddress = hlog.GetPhysicalAddress(startLogicalAddress); + var startLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page); + var endLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page + 1); + var physicalAddress = hlogBase.GetPhysicalAddress(startLogicalAddress); if (recoverFromAddress >= endLogicalAddress) return false; @@ -937,10 +945,10 @@ private long RecoverHybridLogFromSnapshotFile(long scanFromAddress, long recover else { recoveryStatus.WaitFlush(pageIndex); - if (!hlog.IsAllocated(pageIndex)) + if (!hlogBase.IsAllocated(pageIndex)) hlog.AllocatePage(pageIndex); else - hlog.ClearPage(pageIndex); + hlogBase.ClearPage(pageIndex); } } @@ -979,10 +987,10 @@ private async ValueTask RecoverHybridLogFromSnapshotFileAsync(long scanFro else { await recoveryStatus.WaitFlushAsync(pageIndex, cancellationToken).ConfigureAwait(false); - if (!hlog.IsAllocated(pageIndex)) + if (!hlogBase.IsAllocated(pageIndex)) hlog.AllocatePage(pageIndex); else - hlog.ClearPage(pageIndex); + hlogBase.ClearPage(pageIndex); } } @@ -1002,7 +1010,7 @@ private void ApplyDelta(long scanFromAddress, long recoverFromAddress, long unti { int pageIndex = hlogBase.GetPageIndexForPage(p); - var endLogicalAddress = hlog.GetStartLogicalAddress(p + 1); + var endLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(p + 1); if (recoverFromAddress < endLogicalAddress && recoverFromAddress < untilAddress) ProcessReadSnapshotPage(recoverFromAddress, untilAddress, nextVersion, options, recoveryStatus, p, pageIndex); @@ -1011,7 +1019,7 @@ private void ApplyDelta(long scanFromAddress, long recoverFromAddress, long unti { // Flush snapshot page to main log recoveryStatus.flushStatus[pageIndex] = FlushStatus.Pending; - hlogBase.AsyncFlushPages(p, 1, AsyncFlushPageCallbackForRecovery, recoveryStatus); + hlogBase.AsyncFlushPagesForRecovery(p, 1, AsyncFlushPageCallbackForRecovery, recoveryStatus); } } } @@ -1022,11 +1030,11 @@ private void GetSnapshotPageRangesToRead(long fromAddress, long untilAddress, lo // Compute startPage and endPage startPage = hlogBase.GetPage(fromAddress); endPage = hlogBase.GetPage(untilAddress); - if (untilAddress > hlog.GetStartLogicalAddress(endPage) && untilAddress > fromAddress) + if (untilAddress > hlogBase.GetLogicalAddressOfStartOfPage(endPage) && untilAddress > fromAddress) endPage++; long snapshotStartPage = hlogBase.GetPage(snapshotStartAddress); snapshotEndPage = hlogBase.GetPage(snapshotEndAddress); - if (snapshotEndAddress > hlog.GetStartLogicalAddress(snapshotEndPage) && snapshotEndAddress > snapshotStartAddress) + if (snapshotEndAddress > hlogBase.GetLogicalAddressOfStartOfPage(snapshotEndPage) && snapshotEndAddress > snapshotStartAddress) snapshotEndPage++; // By default first page has one extra record @@ -1053,8 +1061,8 @@ private void GetSnapshotPageRangesToRead(long fromAddress, long untilAddress, lo private void ProcessReadSnapshotPage(long fromAddress, long untilAddress, long nextVersion, RecoveryOptions options, RecoveryStatus recoveryStatus, long page, int pageIndex) { // Page at hand - var startLogicalAddress = hlog.GetStartLogicalAddress(page); - var endLogicalAddress = hlog.GetStartLogicalAddress(page + 1); + var startLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page); + var endLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page + 1); // Perform recovery if page is part of the re-do portion of log if (fromAddress < endLogicalAddress && fromAddress < untilAddress) @@ -1069,7 +1077,7 @@ private void ProcessReadSnapshotPage(long fromAddress, long untilAddress, long n var pageFromAddress = 0L; var pageUntilAddress = hlogBase.GetPageSize(); - var physicalAddress = hlog.GetPhysicalAddress(startLogicalAddress); + var physicalAddress = hlogBase.GetPhysicalAddress(startLogicalAddress); if (fromAddress > startLogicalAddress && fromAddress < endLogicalAddress) @@ -1086,9 +1094,9 @@ private void ProcessReadSnapshotPage(long fromAddress, long untilAddress, long n private unsafe void ClearBitsOnPage(long page, RecoveryOptions options) { - var startLogicalAddress = hlog.GetStartLogicalAddress(page); - var endLogicalAddress = hlog.GetStartLogicalAddress(page + 1); - var physicalAddress = hlog.GetPhysicalAddress(startLogicalAddress); + var startLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page); + var endLogicalAddress = hlogBase.GetLogicalAddressOfStartOfPage(page + 1); + var physicalAddress = hlogBase.GetPhysicalAddress(startLogicalAddress); // no need to clear locks for records that will not end up in main memory if (options.headAddress >= endLogicalAddress) return; @@ -1101,14 +1109,14 @@ private unsafe void ClearBitsOnPage(long page, RecoveryOptions options) long recordStart = physicalAddress + pointer; // DiskLogRecord ctor calls ClearBitsForDiskImages(), and then we use its size to move to the next record. - var diskLogRecord = new DiskLogRecord(recordStart); + var logRecord = new LogRecord(recordStart); - if (diskLogRecord.Info.IsNull) - pointer += RecordInfo.GetLength(); + if (logRecord.Info.IsNull) + pointer += RecordInfo.Size; else { - long size = diskLogRecord.GetSerializedLength(); - Debug.Assert(size <= hlogBase.GetPageSize()); // TODO: This will likely exceed pagesize for large objects. Make sure we don't need this limitation + long size = logRecord.GetInlineRecordSizes().allocatedSize; + Debug.Assert(size <= hlogBase.GetPageSize()); pointer += size; } } @@ -1133,18 +1141,18 @@ private unsafe bool RecoverFromPage(long startRecoveryAddress, while (pointer < untilLogicalAddressInPage) { recordStart = pagePhysicalAddress + pointer; - var diskLogRecord = new DiskLogRecord(recordStart); - ref RecordInfo info = ref diskLogRecord.InfoRef; + var logRecord = new LogRecord(recordStart); + ref RecordInfo info = ref logRecord.InfoRef; if (info.IsNull) { - pointer += RecordInfo.GetLength(); + pointer += RecordInfo.Size; continue; } if (!info.Invalid) { - HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(diskLogRecord.Key)); + HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(logRecord.Key)); FindOrCreateTag(ref hei, hlogBase.BeginAddress); bool ignoreRecord = ((pageLogicalAddress + pointer) >= options.fuzzyRegionStartAddress) && info.IsInNewVersion; @@ -1170,7 +1178,7 @@ private unsafe bool RecoverFromPage(long startRecoveryAddress, } } } - pointer += diskLogRecord.GetSerializedLength(); + pointer += logRecord.GetInlineRecordSizes().allocatedSize; } return touched; @@ -1248,7 +1256,7 @@ private bool RestoreHybridLogInitializePages(long beginAddress, long headAddress { if (numPagesToPreload != -1) { - var head = GetStartLogicalAddressOfPage(GetPage(untilAddress) - numPagesToPreload); + var head = GetLogicalAddressOfStartOfPage(GetPage(untilAddress) - numPagesToPreload); if (head > headAddress) headAddress = head; } @@ -1261,7 +1269,7 @@ private bool RestoreHybridLogInitializePages(long beginAddress, long headAddress ((headAddress == untilAddress) && (GetOffsetOnPage(headAddress) == 0)) // Empty in-memory page ) { - if (!_wrapper.IsAllocated(GetPageIndexForAddress(headAddress))) + if (!IsAllocated(GetPageIndexForAddress(headAddress))) _wrapper.AllocatePage(GetPageIndexForAddress(headAddress)); } else @@ -1273,19 +1281,19 @@ private bool RestoreHybridLogInitializePages(long beginAddress, long headAddress recoveryStatus = new RecoveryStatus(GetCapacityNumPages(), MinEmptyPageCount, tailPage, untilAddress, 0); for (int i = 0; i < recoveryStatus.capacity; i++) - { recoveryStatus.readStatus[i] = ReadStatus.Done; - } var numPages = 0; for (var page = headPage; page <= tailPage; page++) - { + { var pageIndex = GetPageIndexForPage(page); recoveryStatus.readStatus[pageIndex] = ReadStatus.Pending; numPages++; } - AsyncReadPagesFromDevice(headPage, numPages, untilAddress, AsyncReadPagesCallbackForRecovery, recoveryStatus); + // Null objectLogDevice means we'll use the one in the allocator + using var readBuffers = CreateCircularReadBuffers(objectLogDevice: null, logger); + AsyncReadPagesForRecovery(readBuffers, headPage, numPages, untilAddress, AsyncReadPagesCallbackForRecovery, recoveryStatus); return true; } } @@ -1298,19 +1306,12 @@ private bool RestoreHybridLogInitializePages(long beginAddress, long headAddress internal unsafe void AsyncReadPagesCallbackForRecovery(uint errorCode, uint numBytes, object context) { if (errorCode != 0) - { logger?.LogError($"{nameof(AsyncReadPagesCallbackForRecovery)} error: {{errorCode}}", errorCode); - } // Set the page status to "read done" var result = (PageAsyncReadResult)context; - if (result.freeBuffer1 != null) - { - _wrapper.PopulatePage(result.freeBuffer1.GetValidPointer(), result.freeBuffer1.required_bytes, result.page); - result.freeBuffer1.Return(); - } - int pageIndex = GetPageIndexForPage(result.page); + var pageIndex = GetPageIndexForPage(result.page); if (errorCode != 0) result.context.SignalReadError(pageIndex); else diff --git a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs index 119be895e71..005b4b584ab 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs @@ -44,7 +44,7 @@ public enum DisposeReason CASAndRetryFailed, /// - /// A record was deserialized from the disk for a pending Read or RMW operation. + /// A record was deserialized from the disk (or network buffer) for a pending Read or RMW operation. /// DeserializedFromDisk, @@ -53,6 +53,11 @@ public enum DisposeReason /// RevivificationFreeList, + /// + /// A new record was created for Upsert or RMW but the InitialWriter or InitialUpdater operation returned false + /// + InsertAbandoned, + /// /// Deleted but remains in hash chain so Key is unchanged /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Constants.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Constants.cs index e8c1bef41f9..f6518800ec3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Constants.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Constants.cs @@ -12,6 +12,7 @@ internal static class Constants // RecordInfo has a long field, so it should be aligned to 8-bytes public const int kRecordAlignment = 8; + public const int kRecordAlignmentMask = kRecordAlignment - 1; /// Number of entries per bucket (assuming 8-byte entries to fill a cacheline) /// Number of bits per bucket (assuming 8-byte entries to fill a cacheline) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/BlockAllocate.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/BlockAllocate.cs index 2bdcc3d3652..b024eb7c18f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/BlockAllocate.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/BlockAllocate.cs @@ -45,26 +45,27 @@ internal struct AllocateOptions [MethodImpl(MethodImplOptions.AggressiveInlining)] bool TryAllocateRecord(TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext, - ref OperationStackContext stackCtx, in RecordSizeInfo sizeInfo, AllocateOptions options, - out long newLogicalAddress, out long newPhysicalAddress, out int allocatedSize, out OperationStatus status) + ref OperationStackContext stackCtx, ref RecordSizeInfo sizeInfo, AllocateOptions options, + out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status) where TSessionFunctionsWrapper : ISessionFunctionsWrapper { status = OperationStatus.SUCCESS; // MinRevivAddress is also needed for pendingContext-based record reuse. - var minMutableAddress = GetMinRevivifiableAddress(); - var minRevivAddress = minMutableAddress; + var minRevivAddress = GetMinRevivifiableAddress(); + + // If are eliding the first record in the tag chain, then there can be no others; we are removing the only record in the chain. Otherwise, the new record must be above hei.Address. + if ((minRevivAddress <= stackCtx.hei.Address) && (!options.elideSourceRecord || stackCtx.hei.Address != stackCtx.recSrc.LogicalAddress)) + minRevivAddress = stackCtx.hei.Address; if (options.recycle && pendingContext.retryNewLogicalAddress != kInvalidAddress - && GetAllocationForRetry(sessionFunctions, ref pendingContext, minRevivAddress, in sizeInfo, out newLogicalAddress, out newPhysicalAddress, out allocatedSize)) + && GetAllocationForRetry(sessionFunctions, ref pendingContext, minRevivAddress, in sizeInfo, out newLogicalAddress, out newPhysicalAddress, out var allocatedSize)) { - new LogRecord(newPhysicalAddress).PrepareForRevivification(in sizeInfo, allocatedSize); + new LogRecord(newPhysicalAddress).PrepareForRevivification(ref sizeInfo, allocatedSize); return true; } if (RevivificationManager.UseFreeRecordPool) { - if (!options.elideSourceRecord && stackCtx.hei.Address >= minMutableAddress) - minRevivAddress = stackCtx.hei.Address; if (sessionFunctions.Ctx.IsInV1) { var fuzzyStartAddress = _hybridLogCheckpoint.info.startLogicalAddress; @@ -73,7 +74,7 @@ bool TryAllocateRecord(TSes } if (TryTakeFreeRecord(sessionFunctions, in sizeInfo, minRevivAddress, out newLogicalAddress, out newPhysicalAddress, out allocatedSize)) { - new LogRecord(newPhysicalAddress).PrepareForRevivification(in sizeInfo, allocatedSize); + new LogRecord(newPhysicalAddress).PrepareForRevivification(ref sizeInfo, allocatedSize); return true; } } @@ -84,12 +85,11 @@ bool TryAllocateRecord(TSes if (!TryBlockAllocate(hlogBase, sizeInfo.AllocatedInlineRecordSize, out newLogicalAddress, ref pendingContext, out status)) break; - newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); + newPhysicalAddress = hlogBase.GetPhysicalAddress(newLogicalAddress); // If allocation had to flush and did it inline, then the epoch was refreshed and we need to check for address safety. if (VerifyInMemoryAddresses(ref stackCtx)) { - allocatedSize = sizeInfo.AllocatedInlineRecordSize; if (newLogicalAddress > stackCtx.recSrc.LatestLogicalAddress) return true; @@ -100,6 +100,8 @@ bool TryAllocateRecord(TSes // so revivification can read the record size. var logRecord = hlog.CreateLogRecord(newLogicalAddress, newPhysicalAddress); logRecord.InitializeForReuse(in sizeInfo); + + // Call RevivificationManager.TryAdd() directly, as here we've done InitializeForReuse of a new record so don't want DisposeRecord. if (RevivificationManager.TryAdd(newLogicalAddress, ref logRecord, ref sessionFunctions.Ctx.RevivificationStats)) continue; } @@ -122,28 +124,24 @@ bool TryAllocateRecord(TSes } newPhysicalAddress = 0; - allocatedSize = 0; return false; } [MethodImpl(MethodImplOptions.AggressiveInlining)] bool TryAllocateRecordReadCache(ref PendingContext pendingContext, ref OperationStackContext stackCtx, - in RecordSizeInfo recordSizeInfo, out long newLogicalAddress, out long newPhysicalAddress, out int allocatedSize, out OperationStatus status) + in RecordSizeInfo recordSizeInfo, out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status) { // Spin to make sure the start of the tag chain is not readcache, or that newLogicalAddress is > the first address in the tag chain. while (true) { if (!TryBlockAllocate(readCacheBase, recordSizeInfo.AllocatedInlineRecordSize, out newLogicalAddress, ref pendingContext, out status)) break; - newPhysicalAddress = readcache.GetPhysicalAddress(newLogicalAddress); + newPhysicalAddress = readCacheBase.GetPhysicalAddress(newLogicalAddress); if (VerifyInMemoryAddresses(ref stackCtx)) { if (!stackCtx.hei.IsReadCache || newLogicalAddress > stackCtx.hei.Address) - { - allocatedSize = recordSizeInfo.AllocatedInlineRecordSize; return true; - } // This allocation is below the necessary address so abandon it and repeat the loop. ReadCacheAbandonRecord(newPhysicalAddress); @@ -158,7 +156,6 @@ bool TryAllocateRecordReadCache(ref PendingContext( goto Fail; } - newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); + newPhysicalAddress = hlogBase.GetPhysicalAddress(newLogicalAddress); var newLogRecord = new LogRecord(newPhysicalAddress); allocatedSize = newLogRecord.GetInlineRecordSizes().allocatedSize; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ConditionalCopyToTail.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ConditionalCopyToTail.cs index 778d90c374f..78e6aef38bc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ConditionalCopyToTail.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ConditionalCopyToTail.cs @@ -124,8 +124,10 @@ internal OperationStatus PrepareIOForConditionalOperation { } : obj => storeFunctions.DisposeValueObject(obj, DisposeReason.DeserializedFromDisk)); return OperationStatus.RECORD_ON_DISK; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs index 54f64efd1a1..762b8e12933 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs @@ -35,12 +35,11 @@ internal OperationStatus ContinuePendingRead stackCtx = new(storeFunctions.GetKeyHashCode64(diskRecord.Key)); + OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(pendingContext.Key)); while (true) { @@ -60,7 +59,7 @@ internal OperationStatus ContinuePendingRead stackCtx = new(pendingContext.keyHash); // See if the record was added above the highest address we checked before issuing the IO. @@ -282,15 +279,15 @@ internal OperationStatus ContinuePendingConditionalCopyToTail(sessionFunctions, diskRecord.Key, ref stackCtx, + if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, pendingContext.request_key.Get(), ref stackCtx, currentAddress: request.logicalAddress, minAddress, pendingContext.maxAddress, out internalStatus, out var needIO)) return OperationStatus.SUCCESS; if (!OperationStatusUtils.IsRetry(internalStatus)) { // HeadAddress may have risen above minAddress; if so, we need IO. internalStatus = needIO - ? PrepareIOForConditionalOperation(ref pendingContext, in diskRecord, ref stackCtx, minAddress, pendingContext.maxAddress) - : ConditionalCopyToTail(sessionFunctions, ref pendingContext, in diskRecord, ref stackCtx); + ? PrepareIOForConditionalOperation(ref pendingContext, in pendingContext.diskLogRecord, ref stackCtx, minAddress, pendingContext.maxAddress) + : ConditionalCopyToTail(sessionFunctions, ref pendingContext, in pendingContext.diskLogRecord, ref stackCtx); } } while (sessionFunctions.Store.HandleImmediateNonPendingRetryStatus(internalStatus, sessionFunctions)); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/FindRecord.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/FindRecord.cs index 036a6ac3da6..9f4f0384d79 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/FindRecord.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/FindRecord.cs @@ -64,8 +64,8 @@ internal bool TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref stackCtx, out internalStatus)) return needIO = false; } - - stackCtx.SetRecordSourceToHashEntry(hlogBase); + else + stackCtx.SetRecordSourceToHashEntry(hlogBase); try { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/HandleOperationStatus.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/HandleOperationStatus.cs index cc8e70834d4..cb0de060e84 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/HandleOperationStatus.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/HandleOperationStatus.cs @@ -138,13 +138,13 @@ internal Status HandleOperationStatus( // Issue asynchronous I/O request request.id = pendingContext.id; - request.request_key = PinnedSpanByte.FromPinnedSpan(pendingContext.Key); + request.request_key = pendingContext.request_key.Get(); request.logicalAddress = pendingContext.logicalAddress; request.minAddress = pendingContext.minAddress; request.record = default; request.callbackQueue = sessionCtx.readyResponses; - hlogBase.AsyncGetFromDisk(pendingContext.logicalAddress, DiskLogRecord.InitialIOSize, request); + hlogBase.AsyncGetFromDisk(pendingContext.logicalAddress, IStreamBuffer.InitialIOSize, request); return new(StatusCode.Pending); } else diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs index 80fde7eb233..0fc908f62f4 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs @@ -21,12 +21,12 @@ private enum LatchDestination } [MethodImpl(MethodImplOptions.AggressiveInlining)] - static LogRecord WriteNewRecordInfo(ReadOnlySpan key, AllocatorBase log, long logicalAddress, long physicalAddress, bool inNewVersion, long previousAddress) + static LogRecord WriteNewRecordInfo(ReadOnlySpan key, AllocatorBase log, long logicalAddress, long physicalAddress, + in RecordSizeInfo sizeInfo, bool inNewVersion, long previousAddress) { - ref var recordInfo = ref LogRecord.GetInfoRef(physicalAddress); - recordInfo.WriteInfo(inNewVersion, previousAddress); var logRecord = log._wrapper.CreateLogRecord(logicalAddress, physicalAddress); - log._wrapper.SerializeKey(key, logicalAddress, ref logRecord); + logRecord.InfoRef.WriteInfo(inNewVersion, previousAddress); + log._wrapper.InitializeRecord(key, logicalAddress, in sizeInfo, ref logRecord); return logRecord; } @@ -79,7 +79,7 @@ private bool IsEntryVersionNew(ref HashBucketEntry entry) // If the record is in memory, check if it has the new version bit set if (entry.Address < hlogBase.HeadAddress) return false; - return LogRecord.GetInfo(hlog.GetPhysicalAddress(entry.Address)).IsInNewVersion; + return LogRecord.GetInfo(hlogBase.GetPhysicalAddress(entry.Address)).IsInNewVersion; } // Can only elide the record if it is the tail of the tag chain (i.e. is the record in the hash bucket entry) and its @@ -123,25 +123,24 @@ internal long GetMinRevivifiableAddress() if (!stackCtx.hei.TryElide()) return (false, false); - return (true, TryTransferToFreeList(sessionFunctions, ref stackCtx, ref logRecord)); + return (true, TryTransferToFreeList(sessionFunctions, stackCtx.recSrc.LogicalAddress, ref logRecord)); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryTransferToFreeList(TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref LogRecord logRecord) + private bool TryTransferToFreeList(TSessionFunctionsWrapper sessionFunctions, long logicalAddress, ref LogRecord logRecord) where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // The record has been CAS'd out of the hashtable or elided from the chain, so add it to the free list. - Debug.Assert(logRecord.Info.IsSealed, "Expected a Sealed record in TryTransferToFreeList"); + Debug.Assert(logRecord.Info.IsClosed, "Expected a Closed record in TryTransferToFreeList"); + + // If its address is not revivifiable, just leave it orphaned and invalid; the caller will Dispose with its own DisposeReason. + if (logicalAddress < GetMinRevivifiableAddress()) + return false; // Dispose any existing key and value. We do this as soon as we have elided so objects are released for GC as early as possible. DisposeRecord(ref logRecord, DisposeReason.RevivificationFreeList); - // Now that we've Disposed the record, see if its address is revivifiable. If not, just leave it orphaned and invalid. - if (stackCtx.recSrc.LogicalAddress < GetMinRevivifiableAddress()) - return false; - - return RevivificationManager.TryAdd(stackCtx.recSrc.LogicalAddress, ref logRecord, ref sessionFunctions.Ctx.RevivificationStats); + return RevivificationManager.TryAdd(logicalAddress, ref logRecord, ref sessionFunctions.Ctx.RevivificationStats); } [MethodImpl(MethodImplOptions.NoInlining)] // Do not try to inline this, to keep TryAllocateRecord lean @@ -177,7 +176,7 @@ internal enum LatchOperation : byte internal void SetRecordInvalid(long logicalAddress) { // This is called on exception recovery for a newly-inserted record. - var localLog = IsReadCache(logicalAddress) ? readcache : hlog; + var localLog = IsReadCache(logicalAddress) ? readCacheBase : hlogBase; LogRecord.GetInfoRef(localLog.GetPhysicalAddress(logicalAddress)).SetInvalid(); } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs index 7dc8d5e6b36..f0083bf7e0d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs @@ -204,10 +204,10 @@ private OperationStatus CreateNewRecordDelete(sessionFunctions, newLogicalAddress, ref newLogRecord)) + DisposeRecord(ref newLogRecord, DisposeReason.InsertAbandoned); if (deleteInfo.Action == DeleteAction.CancelOperation) return OperationStatus.CANCELED; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs index f683207c490..f3652b39117 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs @@ -163,6 +163,12 @@ internal OperationStatus InternalRMW(sessionFunctions, ref stackCtx, ref inMemoryLogRecord); // no new record created and hash entry is empty now - return OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.Found | StatusCode.Expired); + return OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.Expired); } // otherwise we shall continue down the tombstoning path addTombstone = true; } + else if (rmwInfo.Action == RMWAction.WrongType) + return OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, StatusCode.WrongType); else return OperationStatus.SUCCESS; } @@ -416,19 +424,16 @@ private OperationStatus CreateNewRecordRMW storeFunctions.DisposeValueObject(obj, DisposeReason.CopyUpdated)); + srcLogRecord.ClearValueIfHeap(obj => storeFunctions.DisposeValueObject(obj, DisposeReason.CopyUpdated)); goto DoCAS; } if (rmwInfo.Action == RMWAction.CancelOperation) @@ -492,15 +497,19 @@ private OperationStatus CreateNewRecordRMW(sessionFunctions, newLogicalAddress, ref newLogRecord)) + DisposeRecord(ref newLogRecord, DisposeReason.InsertAbandoned); goto RetryNow; } addTombstone = newLogRecord.Info.Tombstone; goto DoCAS; } + else if (rmwInfo.Action == RMWAction.WrongType) + { + status = OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, StatusCode.CreatedRecord | StatusCode.Expired); + return OperationStatus.NOTFOUND; + } else return OperationStatus.SUCCESS | (forExpiration ? OperationStatus.EXPIRED : OperationStatus.SUCCESS); } @@ -532,16 +541,32 @@ private OperationStatus CreateNewRecordRMW storeFunctions.DisposeValueObject(obj, DisposeReason.CopyUpdated)); + } + else if (rmwInfo.Action == RMWAction.ExpireAndStop) { newLogRecord.InfoRef.SetTombstone(); status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.CopyUpdatedRecord | StatusCode.Expired); } - else + else if (rmwInfo.Action == RMWAction.WrongType) { - Debug.Fail("Can only handle RMWAction.ExpireAndStop on a false return from PostCopyUpdater"); + status = OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, StatusCode.CopyUpdatedRecord | StatusCode.Expired); + goto Done; } } @@ -553,9 +578,9 @@ private OperationStatus CreateNewRecordRMW= GetMinRevivifiableAddress()) - _ = TryTransferToFreeList(sessionFunctions, ref stackCtx, ref inMemoryLogRecord); + _ = TryTransferToFreeList(sessionFunctions, stackCtx.recSrc.LogicalAddress, ref inMemoryLogRecord); else DisposeRecord(ref inMemoryLogRecord, DisposeReason.Elided); } @@ -563,6 +588,7 @@ private OperationStatus CreateNewRecordRMW(sessionFunctions, ref stackCtx, srcLogRecord.Info) }; - if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, in sizeInfo, allocOptions, out var newLogicalAddress, out var newPhysicalAddress, out var allocatedSize, out var status)) + if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, ref sizeInfo, allocOptions, out var newLogicalAddress, out var newPhysicalAddress, out var status)) return status; - var newLogRecord = WriteNewRecordInfo(key, hlogBase, newLogicalAddress, newPhysicalAddress, sessionFunctions.Ctx.InNewVersion, previousAddress: stackCtx.recSrc.LatestLogicalAddress); + var newLogRecord = WriteNewRecordInfo(key, hlogBase, newLogicalAddress, newPhysicalAddress, in sizeInfo, sessionFunctions.Ctx.InNewVersion, previousAddress: stackCtx.recSrc.LatestLogicalAddress); if (allocOptions.elideSourceRecord) newLogRecord.InfoRef.PreviousAddress = srcLogRecord.Info.PreviousAddress; stackCtx.SetNewRecord(newLogicalAddress); @@ -305,19 +305,15 @@ private OperationStatus CreateNewRecordUpsert( ref newLogRecord, in sizeInfo, ref input, srcStringValue, srcObjectValue, in inputLogRecord, ref output, ref upsertInfo, sessionFunctions); if (!success) { // Save allocation for revivification (not retry, because these aren't retry status codes), or abandon it if that fails. - if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, ref newLogRecord, ref sessionFunctions.Ctx.RevivificationStats)) - stackCtx.ClearNewRecord(); - else - stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); + if (!RevivificationManager.UseFreeRecordPool || !TryTransferToFreeList(sessionFunctions, newLogicalAddress, ref newLogRecord)) + DisposeRecord(ref newLogRecord, DisposeReason.InsertAbandoned); if (upsertInfo.Action == UpsertAction.CancelOperation) return OperationStatus.CANCELED; @@ -343,7 +339,7 @@ private OperationStatus CreateNewRecordUpsert= GetMinRevivifiableAddress()) - _ = TryTransferToFreeList(sessionFunctions, ref stackCtx, ref srcLogRecord); + _ = TryTransferToFreeList(sessionFunctions, stackCtx.recSrc.LogicalAddress, ref srcLogRecord); else DisposeRecord(ref srcLogRecord, DisposeReason.Elided); } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ModifiedBitOperation.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ModifiedBitOperation.cs index 04301bb8126..1ba869a499b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ModifiedBitOperation.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ModifiedBitOperation.cs @@ -43,7 +43,7 @@ internal OperationStatus InternalModifiedBitOperation(ReadOnlySpan key, ou modifiedInfo = default; if (logicalAddress >= hlogBase.HeadAddress) { - ref var recordInfo = ref LogRecord.GetInfoRef(hlog.GetPhysicalAddress(logicalAddress)); + ref var recordInfo = ref LogRecord.GetInfoRef(hlogBase.GetPhysicalAddress(logicalAddress)); if (reset) { if (!recordInfo.TryResetModifiedAtomic()) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ReadCache.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ReadCache.cs index 3b2ec952fbe..02145351721 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ReadCache.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ReadCache.cs @@ -7,6 +7,7 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; // Partial file for readcache functions @@ -20,7 +21,7 @@ internal bool FindInReadCache(ReadOnlySpan key, ref OperationStackContext< Debug.Assert(UseReadCache, "Should not call FindInReadCache if !UseReadCache"); // minAddress, if present, comes from the pre-pendingIO entry.Address; there may have been no readcache entries then. - minAddress = IsReadCache(minAddress) ? minAddress : readCacheBase.HeadAddress; + minAddress = IsReadCache(minAddress) ? AbsoluteAddress(minAddress) : readCacheBase.HeadAddress; RestartChain: @@ -40,7 +41,7 @@ internal bool FindInReadCache(ReadOnlySpan key, ref OperationStackContext< // LatestLogicalAddress is the "leading" pointer and will end up as the highest logical address in the main log for this tag chain. // Increment the trailing "lowest read cache" address (for the splice point). We'll look ahead from this to examine the next record. stackCtx.recSrc.LowestReadCacheLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; - stackCtx.recSrc.LowestReadCachePhysicalAddress = readcache.GetPhysicalAddress(stackCtx.recSrc.LowestReadCacheLogicalAddress); + stackCtx.recSrc.LowestReadCachePhysicalAddress = readCacheBase.GetPhysicalAddress(stackCtx.recSrc.LowestReadCacheLogicalAddress); // Use a non-ref local, because we don't need to update. var recordInfo = LogRecord.GetInfo(stackCtx.recSrc.LowestReadCachePhysicalAddress); @@ -50,7 +51,7 @@ internal bool FindInReadCache(ReadOnlySpan key, ref OperationStackContext< if (!recordInfo.Invalid && stackCtx.recSrc.LatestLogicalAddress >= minAddress && !stackCtx.recSrc.HasReadCacheSrc) { ReadOnlySpan keySpan = recordInfo.KeyIsInline - ? LogRecord.GetInlineKey(stackCtx.recSrc.LowestReadCachePhysicalAddress) + ? LogRecord.GetInlineKey(stackCtx.recSrc.LowestReadCachePhysicalAddress) // Most keys are inline and this is faster : readcache.CreateLogRecord(stackCtx.recSrc.LowestReadCacheLogicalAddress).Key; if (storeFunctions.KeysEqual(key, keySpan)) { @@ -137,7 +138,7 @@ internal void SkipReadCache(ref OperationStackContextIsReadCache) continue; var logicalAddress = entry->Address; - var physicalAddress = readcache.GetPhysicalAddress(logicalAddress); + var physicalAddress = readCacheBase.GetPhysicalAddress(logicalAddress); while (true) { @@ -171,7 +172,7 @@ private void SkipReadCacheBucket(HashBucket* bucket) entry->Address = logicalAddress; if (!entry->IsReadCache) break; - physicalAddress = readcache.GetPhysicalAddress(logicalAddress); + physicalAddress = readCacheBase.GetPhysicalAddress(logicalAddress); } } } @@ -246,12 +247,10 @@ void ReadCacheAbandonRecord(long physicalAddress) internal void ReadCacheEvict(long rcLogicalAddress, long rcToLogicalAddress) { - Debug.Assert(IsReadCache(rcLogicalAddress) && IsReadCache(rcToLogicalAddress), "rcLogicalAddress and rcToLogicalAddress must be readcache addresses"); - // Iterate readcache entries in the range rcFrom/ToLogicalAddress, and remove them from the hash chain. while (rcLogicalAddress < rcToLogicalAddress) { - var logRecord = new LogRecord(readcache.GetPhysicalAddress(rcLogicalAddress)); + var logRecord = new LogRecord(readCacheBase.GetPhysicalAddress(rcLogicalAddress)); var (_, rcAllocatedSize) = logRecord.GetInlineRecordSizes(); var rcRecordInfo = logRecord.Info; @@ -282,7 +281,7 @@ internal void ReadCacheEvict(long rcLogicalAddress, long rcToLogicalAddress) NextRecord: if (readCacheBase.GetOffsetOnPage(rcLogicalAddress) + rcAllocatedSize > readCacheBase.PageSize) { - rcLogicalAddress = readCacheBase.GetStartLogicalAddressOfPage(1 + readCacheBase.GetPage(rcLogicalAddress)); + rcLogicalAddress = readCacheBase.GetLogicalAddressOfStartOfPage(1 + readCacheBase.GetPage(rcLogicalAddress)); continue; } rcLogicalAddress += rcAllocatedSize; @@ -298,7 +297,7 @@ private void ReadCacheEvictChain(long rcToLogicalAddress, ref HashEntryInfo hei) HashBucketEntry entry = new() { word = hei.entry.word }; while (entry.IsReadCache) { - var logRecord = new LogRecord(readcache.GetPhysicalAddress(entry.Address)); + var logRecord = new LogRecord(readCacheBase.GetPhysicalAddress(entry.Address)); ref var recordInfo = ref logRecord.InfoRef; #if DEBUG diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/RecordSource.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/RecordSource.cs index 99344180995..73b6c50c72b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/RecordSource.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/RecordSource.cs @@ -127,7 +127,7 @@ void append(int value, string name) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void SetHasMainLogSrc() { - Debug.Assert(IsInLogMemory(LogicalAddress), "LogicalAddress must be a non-readcache in-mmeory address to set HasMainLogSrc"); + Debug.Assert(!IsReadCache(LogicalAddress), "LogicalAddress must be a main log address to set HasMainLogSrc"); internalState |= InternalStates.MainLogSrc; } @@ -149,7 +149,7 @@ internal void SetHasReadCacheSrc() internal void ClearHasReadCacheSrc() => internalState &= ~InternalStates.ReadCacheSrc; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal long SetPhysicalAddress() => PhysicalAddress = Allocator.GetPhysicalAddress(LogicalAddress); + internal long SetPhysicalAddress() => PhysicalAddress = AllocatorBase.GetPhysicalAddress(LogicalAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] internal readonly ref RecordInfo GetInfoRef() => ref LogRecord.GetInfoRef(PhysicalAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/FreeRecordPool.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/FreeRecordPool.cs index 20ad2737b38..b35e4fe0762 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/FreeRecordPool.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/FreeRecordPool.cs @@ -10,17 +10,20 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; using static Utility; [StructLayout(LayoutKind.Explicit, Size = sizeof(long))] internal struct FreeRecord { - internal const int kSizeBits = 64 - kAddressBits; // 14 + internal const int kSizeBits = 64 - kAddressBits; // 16 currently +#pragma warning disable IDE1006 // Naming Styles const int kSizeShiftInWord = kAddressBits; const long kSizeMask = RevivificationBin.MaxInlineRecordSize - 1; const long kSizeMaskInWord = kSizeMask << kSizeShiftInWord; +#pragma warning restore IDE1006 // Naming Styles // This is the empty word we replace the current word with on Reads. private const long emptyWord = 0; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs index 98628ff4b91..2394b6bd4b9 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs @@ -249,13 +249,13 @@ private long TraceBackForOtherChainStart(long logicalAddress, int bit) { if (logicalAddress < readCacheBase.HeadAddress) break; - logRecord = new LogRecord(readcache.GetPhysicalAddress(logicalAddress)); + logRecord = new LogRecord(readCacheBase.GetPhysicalAddress(logicalAddress)); } else { if (logicalAddress < hlogBase.HeadAddress) break; - logRecord = new LogRecord(hlog.GetPhysicalAddress(logicalAddress)); + logRecord = new LogRecord(hlogBase.GetPhysicalAddress(logicalAddress)); } var hash = storeFunctions.GetKeyHashCode64(logRecord.Key); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToReadCache.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToReadCache.cs index 84d97d6b359..3a501b11efe 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToReadCache.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToReadCache.cs @@ -3,7 +3,7 @@ namespace Tsavorite.core { - using System; +#pragma warning disable IDE0065 // Misplaced using directive using static LogAddress; public unsafe partial class TsavoriteKV : TsavoriteBase @@ -28,19 +28,16 @@ internal bool TryCopyToReadCache srcValue = default; - UpsertInfo upsertInfo = new() - { - Version = sessionFunctions.Ctx.version, - SessionID = sessionFunctions.Ctx.sessionID, - Address = kInvalidAddress, // stackCtx.recSrc.LogicalAddress, - KeyHash = stackCtx.hei.hash - }; - - // TODO: This is called by readcache directly, but is the only ISessionFunctions call for that; the rest is internal. Clean this up, maybe as a new PostReadCacheInsert method. - sessionFunctions.PostInitialWriter(ref newLogRecord, in sizeInfo, ref input, srcValue, ref output, ref upsertInfo); newLogRecord.InfoRef.UnsealAndValidate(); pendingContext.logicalAddress = kInvalidAddress; // We aren't doing anything with this; and we never expose readcache addresses stackCtx.ClearNewRecord(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToTail.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToTail.cs index 53fb9e21d22..d569aea610b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToTail.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/TryCopyToTail.cs @@ -31,14 +31,12 @@ internal OperationStatus TryCopyToTail + /// for serializing/deserializing . + /// + public ObjectIdMap TransientObjectIdMap => allocatorBase.transientObjectIdMap; + /// /// Set empty page count in allocator /// @@ -88,7 +93,7 @@ public void SetEmptyPageCount(int pageCount, bool wait = false) allocatorBase.EmptyPageCount = pageCount; if (wait) { - long newHeadAddress = allocatorBase.GetAddressOfStartOfPage(allocatorBase.GetTailAddress()) - allocatorBase.HeadAddressLagOffset; + long newHeadAddress = allocatorBase.GetAddressOfStartOfPageOfAddress(allocatorBase.GetTailAddress()) - allocatorBase.HeadAddressLagOffset; ShiftHeadAddress(newHeadAddress, wait); } } @@ -101,7 +106,7 @@ public void SetEmptyPageCount(int pageCount, bool wait = false) /// /// Actual memory used by log (not including heap objects) and overflow pages /// - public long MemorySizeBytes => allocatorBase.GetStartLogicalAddressOfPage((long)(allocatorBase.AllocatedPageCount + allocator.OverflowPageCount)); + public long MemorySizeBytes => allocatorBase.GetLogicalAddressOfStartOfPage((long)(allocatorBase.AllocatedPageCount + allocator.OverflowPageCount)); /// /// Maximum memory size in bytes @@ -124,7 +129,7 @@ public void SetEmptyPageCount(int pageCount, bool wait = false) public void ShiftBeginAddress(long untilAddress, bool snapToPageStart = false, bool truncateLog = false) { if (snapToPageStart) - untilAddress = allocatorBase.GetAddressOfStartOfPage(untilAddress); + untilAddress = allocatorBase.GetAddressOfStartOfPageOfAddress(untilAddress); var epochProtected = store.epoch.ThisInstanceProtected(); try diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 26bad6c8437..dde3cb76d1f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -148,7 +148,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFunctions, Func hlogBase.FlushedUntilAddress - ; // TODO remove && !hlog.HasObjectLog; - if (incremental) - { - stateMachine = Checkpoint.IncrementalHybridLogOnly(this, token); - } - else - { - stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, out token); - } + stateMachine = tryIncremental && InternalCanTakeIncrementalCheckpoint(checkpointType, ref token) + ? Checkpoint.IncrementalHybridLogOnly(this, token) + : stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, out token); } return stateMachineDriver.Register(stateMachine); } @@ -312,16 +302,21 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp /// /// Whether we can take an incremental snapshot checkpoint given current state of the store /// - /// - /// - public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid guid) + public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid token) { - guid = _lastSnapshotCheckpoint.info.guid; - return - checkpointType == CheckpointType.Snapshot - && guid != default - && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress - ; // TODO remove: && !hlog.HasObjectLog; + token = _lastSnapshotCheckpoint.info.guid; + return InternalCanTakeIncrementalCheckpoint(checkpointType, ref token); + } + + /// + /// Whether we can take an incremental snapshot checkpoint given current state of the store + /// + private bool InternalCanTakeIncrementalCheckpoint(CheckpointType checkpointType, ref Guid token) + { + return checkpointType == CheckpointType.Snapshot + && token != default + && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress + && !hlog.HasObjectLog; } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs index a9ae470b3d1..3f4c67228b0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs @@ -264,6 +264,15 @@ bool IsTailmostMainKvRecord(ReadOnlySpan key, RecordInfo mainKvRecordInfo, /// public RecordInfo Info => CurrentIter.Info; + /// + public byte RecordType => CurrentIter.RecordType; + + /// + public byte Namespace => CurrentIter.Namespace; + + /// + public ObjectIdMap ObjectIdMap => CurrentIter.ObjectIdMap; + /// public bool IsSet => !CurrentIter.IsSet; @@ -277,13 +286,17 @@ bool IsTailmostMainKvRecord(ReadOnlySpan key, RecordInfo mainKvRecordInfo, public unsafe byte* PinnedKeyPointer => CurrentIter.PinnedKeyPointer; /// - public unsafe Span ValueSpan => CurrentIter.ValueSpan; + public OverflowByteArray KeyOverflow + { + get => CurrentIter.KeyOverflow; + set => CurrentIter.KeyOverflow = value; + } /// - public IHeapObject ValueObject => CurrentIter.ValueObject; + public unsafe Span ValueSpan => CurrentIter.ValueSpan; /// - public ReadOnlySpan RecordSpan => CurrentIter.RecordSpan; + public IHeapObject ValueObject => CurrentIter.ValueObject; /// public bool IsPinnedValue => CurrentIter.IsPinnedValue; @@ -291,6 +304,13 @@ bool IsTailmostMainKvRecord(ReadOnlySpan key, RecordInfo mainKvRecordInfo, /// public unsafe byte* PinnedValuePointer => CurrentIter.PinnedValuePointer; + /// + public OverflowByteArray ValueOverflow + { + get => CurrentIter.ValueOverflow; + set => CurrentIter.ValueOverflow = value; + } + /// public long ETag => CurrentIter.ETag; @@ -298,13 +318,19 @@ bool IsTailmostMainKvRecord(ReadOnlySpan key, RecordInfo mainKvRecordInfo, public long Expiration => CurrentIter.Expiration; /// - public void ClearValueObject(Action disposer) { } // Not relevant for iterators + public void ClearValueIfHeap(Action disposer) { } // Not relevant for "iterator as logrecord" + + /// + public bool IsMemoryLogRecord => CurrentIter.IsMemoryLogRecord; + + /// + public unsafe ref LogRecord AsMemoryLogRecordRef() => throw new InvalidOperationException("Cannot cast a TsavoriteKVIterator to a memory LogRecord."); /// - public bool AsLogRecord(out LogRecord logRecord) => CurrentIter.AsLogRecord(out logRecord); + public bool IsDiskLogRecord => CurrentIter.IsDiskLogRecord; /// - public bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) => CurrentIter.AsDiskLogRecord(out diskLogRecord); + public unsafe ref DiskLogRecord AsDiskLogRecordRef() => ref CurrentIter.AsDiskLogRecordRef(); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 33691d8fcbd..ce1820c4627 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -101,7 +101,10 @@ internal void InternalCompletePendingRequest ContinuePendingRead(request, ref pendingContext, sessionFunctions), OperationType.RMW => ContinuePendingRMW(request, ref pendingContext, sessionFunctions), @@ -136,7 +137,7 @@ internal unsafe Status InternalCompletePendingRequestFromContext public long MaxMemorySizeBytes => allocator.MaxMemorySizeBytes; - internal long SetAddressType(long address) => allocator.SetAddressType(address); - /// /// Actual memory used by log /// - public long MemorySizeBytes => allocator.GetStartLogicalAddressOfPage(allocator.AllocatedPageCount + allocator.OverflowPageCount); + public long MemorySizeBytes => allocator.GetLogicalAddressOfStartOfPage(allocator.AllocatedPageCount + allocator.OverflowPageCount); /// /// Create new log instance @@ -2056,7 +2054,7 @@ public void UnsafeShiftBeginAddress(long untilAddress, bool snapToPageStart = fa if (Utility.MonotonicUpdate(ref beginAddress, untilAddress, out _)) { if (snapToPageStart) - untilAddress = allocator.GetAddressOfStartOfPage(untilAddress); + untilAddress = allocator.GetAddressOfStartOfPageOfAddress(untilAddress); bool epochProtected = epoch.ThisInstanceProtected(); try @@ -2081,7 +2079,7 @@ public void UnsafeShiftBeginAddress(long untilAddress, bool snapToPageStart = fa /// Until address public void TruncateUntilPageStart(long untilAddress) { - Utility.MonotonicUpdate(ref beginAddress, allocator.GetAddressOfStartOfPage(untilAddress), out _); + Utility.MonotonicUpdate(ref beginAddress, allocator.GetAddressOfStartOfPageOfAddress(untilAddress), out _); } /// @@ -2185,7 +2183,7 @@ public TsavoriteLogScanSingleIterator ScanSingle(long beginAddress, long endAddr }; unsafe { - allocator.AsyncReadRecordToMemory(address, headerSize + estimatedLength, AsyncGetFromDiskCallback, ref ctx); + allocator.AsyncReadBlittableRecordToMemory(address, headerSize + estimatedLength, AsyncGetFromDiskCallback, ref ctx); } epoch.Suspend(); await ctx.completedRead.WaitAsync(token).ConfigureAwait(false); @@ -2216,7 +2214,7 @@ public TsavoriteLogScanSingleIterator ScanSingle(long beginAddress, long endAddr }; unsafe { - allocator.AsyncReadRecordToMemory(address, headerSize + estimatedLength, AsyncGetFromDiskCallback, ref ctx); + allocator.AsyncReadBlittableRecordToMemory(address, headerSize + estimatedLength, AsyncGetFromDiskCallback, ref ctx); } epoch.Suspend(); await ctx.completedRead.WaitAsync(token).ConfigureAwait(false); @@ -2245,7 +2243,7 @@ public async ValueTask ReadRecordLengthAsync(long address, CancellationToke }; unsafe { - allocator.AsyncReadRecordToMemory(address, headerSize, AsyncGetHeaderOnlyFromDiskCallback, ref ctx); + allocator.AsyncReadBlittableRecordToMemory(address, headerSize, AsyncGetHeaderOnlyFromDiskCallback, ref ctx); } epoch.Suspend(); await ctx.completedRead.WaitAsync(token).ConfigureAwait(false); @@ -2811,7 +2809,7 @@ private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, obje else { ctx.record.Return(); - allocator.AsyncReadRecordToMemory(ctx.logicalAddress, requiredBytes, AsyncGetFromDiskCallback, ref ctx); + allocator.AsyncReadBlittableRecordToMemory(ctx.logicalAddress, requiredBytes, AsyncGetFromDiskCallback, ref ctx); } } } diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs index f1d943862bb..288aaab7f8c 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs @@ -34,7 +34,8 @@ public class TsavoriteLogIterator : ScanIteratorBase, IDisposable /// Constructor internal unsafe TsavoriteLogIterator(TsavoriteLog tsavoriteLog, TsavoriteLogAllocatorImpl hlog, long beginAddress, long endAddress, GetMemory getMemory, DiskScanBufferingMode diskScanBufferingMode, LightEpoch epoch, int headerSize, bool scanUncommitted = false, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddressOnPage(0) : beginAddress, endAddress, diskScanBufferingMode, InMemoryScanBufferingMode.NoBuffering, includeClosedRecords: false, epoch, hlog.LogPageSizeBits, logger: logger) + : base(readBuffers: default, beginAddress == 0 ? hlog.GetFirstValidLogicalAddressOnPage(0) : beginAddress, endAddress, + diskScanBufferingMode, InMemoryScanBufferingMode.NoBuffering, includeClosedRecords: false, epoch, hlog.LogPageSizeBits, logger: logger) { this.tsavoriteLog = tsavoriteLog; allocator = hlog; @@ -632,32 +633,23 @@ public override void Dispose() } } - internal override void AsyncReadPagesFromDeviceToFrame(long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) - => allocator.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice, cts); + internal override void AsyncReadPagesFromDeviceToFrame(CircularDiskReadBuffer _ /*readBuffers*/, long readPageStart, int numPages, long untilAddress, TContext context, out CountdownEvent completed, + long devicePageOffset = 0, IDevice device = null, IDevice objectLogDevice = null, CancellationTokenSource cts = null) + => allocator.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesToFrameCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice, cts); - private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) + private unsafe void AsyncReadPagesToFrameCallback(uint errorCode, uint numBytes, object context) { try { var result = (PageAsyncReadResult)context; - if (errorCode != 0) + if (errorCode == 0) + _ = result.handle?.Signal(); + else { - logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); + logger?.LogError($"{nameof(AsyncReadPagesToFrameCallback)} error: {{errorCode}}", errorCode); result.cts?.Cancel(); } - - if (result.freeBuffer1 != null) - { - if (errorCode == 0) - allocator._wrapper.PopulatePage(result.freeBuffer1.GetValidPointer(), result.freeBuffer1.required_bytes, result.page); - result.freeBuffer1.Return(); - result.freeBuffer1 = null; - } - - if (errorCode == 0) - result.handle?.Signal(); - Interlocked.MemoryBarrier(); } catch when (disposed) { } @@ -702,7 +694,7 @@ internal unsafe bool ScanForwardForCommit(ref TsavoriteLogRecoveryInfo info, lon if (info.CommitNum == commitNum) return true; - // User wants any commie + // User wants any commit if (commitNum == -1) return foundCommit; // requested commit not found @@ -710,16 +702,8 @@ internal unsafe bool ScanForwardForCommit(ref TsavoriteLogRecoveryInfo info, lon } /// - /// Retrieve physical address of next iterator value - /// (under epoch protection if it is from main page buffer) + /// Retrieve physical address of next iterator value (under epoch protection if it is from main page buffer) /// - /// - /// - /// - /// - /// - /// - /// private unsafe bool GetNextInternal(out long physicalAddress, out int entryLength, out long currentAddress, out long outNextAddress, out bool commitRecord, out bool onFrame) { while (true) @@ -787,7 +771,7 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt if (entryLength == 0) { // Zero-ed out bytes could be padding at the end of page, first jump to the start of next page. - var nextStart = allocator.GetStartLogicalAddressOfPage(1 + allocator.GetPage(currentAddress)); + var nextStart = allocator.GetLogicalAddressOfStartOfPage(1 + allocator.GetPage(currentAddress)); if (Utility.MonotonicUpdate(ref nextAddress, nextStart, out _)) { var pageOffset = allocator.GetOffsetOnPage(currentAddress); @@ -828,7 +812,7 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt } if ((allocator.GetOffsetOnPage(currentAddress) + recordSize) == allocator.PageSize) - currentAddress = allocator.GetStartLogicalAddressOfPage(1 + allocator.GetPage(currentAddress)); + currentAddress = allocator.GetLogicalAddressOfStartOfPage(1 + allocator.GetPage(currentAddress)); else currentAddress += recordSize; @@ -933,7 +917,7 @@ private unsafe bool ExpandGetNextInternal(long startPhysicalAddress, ref int tot } if ((allocator.GetOffsetOnPage(currentAddress) + recordSize) == allocator.PageSize) - currentAddress = allocator.GetStartLogicalAddressOfPage(1 + allocator.GetPage(currentAddress)); + currentAddress = allocator.GetLogicalAddressOfStartOfPage(1 + allocator.GetPage(currentAddress)); else currentAddress += recordSize; diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs index 7900fec9625..f65bf5923f1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs @@ -16,6 +16,9 @@ namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + /// /// Sector aligned memory allocator /// @@ -35,27 +38,33 @@ public sealed unsafe class SectorAlignedMemory internal GCHandle handle; /// - /// Offset + /// Offset for initial allocation alignment of the block; this is the offset from the first element of to form . + /// This alignment is internal to , and ensures that callers see an aligned starting address. /// - public int offset; + public int aligned_offset; /// - /// Aligned pointer + /// Aligned pointer; initial allocation (the first element of ) plus + /// This alignment is internal to , and ensures that callers see an aligned starting address. /// public byte* aligned_pointer; /// - /// Valid offset + /// Valid offset for operations above , to get their own desired alignment relative to our aligned starting address. + /// This is set by the caller for operations such as file reading, which rounds down to the nearest sector size; this is the amount of that rounding down. + /// Used by , which is + . /// public int valid_offset; /// - /// Required bytes + /// Required (requested) bytes for the current operation: the unaligned number of bytes to read. There will always be at least this much usable space in the allocation. + /// Use this when the original request size is needed. /// public int required_bytes; /// - /// Available bytes + /// Available bytes after the operation is complete: the number of bytes actually read, e.g. aligned number of bytes requested. See . + /// Use this to see if there are additional bytes over the original request (see . /// public int available_bytes; @@ -106,13 +115,14 @@ public SectorAlignedMemory(int level = default) /// public SectorAlignedMemory(int numRecords, int sectorSize) { - int recordSize = 1; - int requiredSize = sectorSize + (((numRecords) * recordSize + (sectorSize - 1)) & ~(sectorSize - 1)); + const int recordSize = 1; + required_bytes = numRecords * recordSize; + int requiredSize = sectorSize + RoundUp(required_bytes, sectorSize); // An additional sector size for the aligned_offset buffer = GC.AllocateArray(requiredSize, true); long bufferAddr = (long)Unsafe.AsPointer(ref buffer[0]); aligned_pointer = (byte*)((bufferAddr + (sectorSize - 1)) & ~((long)sectorSize - 1)); - offset = (int)((long)aligned_pointer - bufferAddr); + aligned_offset = (int)((long)aligned_pointer - bufferAddr); // Assume ctor is called for allocation and leave Free unset } @@ -145,17 +155,44 @@ public void Return() /// /// Get the total aligned memory capacity of the buffer /// - public int AlignedTotalCapacity => buffer.Length - offset; + public int AlignedTotalCapacity => buffer.Length - aligned_offset; + + /// + /// Get the total valid memory capacity of the buffer + /// + public int ValidTotalCapacity => AlignedTotalCapacity - valid_offset; + + /// + /// Get the total valid required (requested) capacity of the buffer + /// + public int RequiredCapacity => required_bytes - valid_offset; /// - /// Get valid pointer + /// Get valid pointer (accounts for aligned padding plus any offset specified for the valid start of data) /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public byte* GetValidPointer() - { - return aligned_pointer + valid_offset; - } + public byte* GetValidPointer() => aligned_pointer + valid_offset; + + /// + /// Get Span of entire allocated space after the valid pointer + /// + public Span TotalValidSpan => new(GetValidPointer(), ValidTotalCapacity); + + /// + /// Get Span of entire allocated space after the aligned pointer (see ). + /// + public Span AvailableSpan => new(aligned_pointer, available_bytes); + + /// + /// Get Span of entire allocated space after the valid pointer (see ). + /// + public Span AvailableValidSpan => new(GetValidPointer(), available_bytes - valid_offset); + + /// + /// Returns the Span of requested space (see ). + /// + public Span RequiredValidSpan => new(GetValidPointer(), RequiredCapacity); /// /// ToString @@ -163,9 +200,11 @@ public void Return() /// public override string ToString() { - return string.Format($"{(long)aligned_pointer} {offset} {valid_offset} {required_bytes} {available_bytes}" + return string.Format($"aligned: [offset {aligned_offset}, ptr {(long)aligned_pointer} = 0x{(long)aligned_pointer:X}];" + + $" valid: [offset {valid_offset} ptr {(long)GetValidPointer()} = 0x{(long)GetValidPointer():X}];" + + $" reqBytes {required_bytes}; availBytes {available_bytes}; cap {AlignedTotalCapacity}" #if CHECK_FREE - + $" {this.Free}" + + $"; free {Free}" #endif ); } @@ -223,7 +262,11 @@ public void EnsureSize(ref SectorAlignedMemory page, int size) { page.Return(); page = Get(size); + return; } + + // Reusing the page, so ensure this is set correctly. + page.required_bytes = size; } /// @@ -282,7 +325,8 @@ public unsafe SectorAlignedMemory Get(int numRecords) Interlocked.Increment(ref totalGets); #endif - int requiredSize = sectorSize + ((numRecords * recordSize + (sectorSize - 1)) & ~(sectorSize - 1)); + int required_bytes = numRecords * recordSize; + int requiredSize = RoundUp(required_bytes, sectorSize); int index = Position(requiredSize / sectorSize); if (queue[index] == null) { @@ -298,21 +342,24 @@ public unsafe SectorAlignedMemory Get(int numRecords) if (UnpinOnReturn) { page.handle = GCHandle.Alloc(page.buffer, GCHandleType.Pinned); - page.aligned_pointer = (byte*)(((long)page.handle.AddrOfPinnedObject() + (sectorSize - 1)) & ~((long)sectorSize - 1)); - page.offset = (int)((long)page.aligned_pointer - (long)page.handle.AddrOfPinnedObject()); + page.aligned_pointer = (byte*)RoundUp(page.handle.AddrOfPinnedObject(), sectorSize); + page.aligned_offset = (int)((long)page.aligned_pointer - page.handle.AddrOfPinnedObject()); } + page.required_bytes = required_bytes; return page; } page = new SectorAlignedMemory(level: index) { - buffer = GC.AllocateArray(sectorSize * (1 << index), !UnpinOnReturn) + // Add an additional sector for the leading RoundUp of pageAddr to sectorSize. + buffer = GC.AllocateArray(sectorSize * ((1 << index) + 1), !UnpinOnReturn) }; if (UnpinOnReturn) page.handle = GCHandle.Alloc(page.buffer, GCHandleType.Pinned); long pageAddr = (long)Unsafe.AsPointer(ref page.buffer[0]); - page.aligned_pointer = (byte*)((pageAddr + (sectorSize - 1)) & ~((long)sectorSize - 1)); - page.offset = (int)((long)page.aligned_pointer - pageAddr); + page.aligned_pointer = (byte*)RoundUp(pageAddr, sectorSize); + page.aligned_offset = (int)((long)page.aligned_pointer - pageAddr); + page.required_bytes = required_bytes; page.pool = this; return page; } diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/PageAsyncResultTypes.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/PageAsyncResultTypes.cs index 75098b5458c..f18abf0bb96 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/PageAsyncResultTypes.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/PageAsyncResultTypes.cs @@ -3,6 +3,9 @@ #define CALLOC +using System; +using System.Diagnostics; +using System.Runtime.InteropServices; using System.Threading; namespace Tsavorite.core @@ -13,38 +16,46 @@ namespace Tsavorite.core /// public sealed class PageAsyncReadResult { + /// Index of the main-log page being read internal long page; - internal long offset; + + /// Recovery device page offset + internal long devicePageOffset; + + /// Context state to be passed through the read operation internal TContext context; + + /// Event to be signaled when the main-log page read is complete internal CountdownEvent handle; - internal SectorAlignedMemory freeBuffer1; - internal SectorAlignedMemory freeBuffer2; + + /// Callback to be called when the main-log page has completed processing; for + /// this means after all Overflow or Objects on the page have been read as well. internal DeviceIOCompletionCallback callback; - internal IDevice objlogDevice; - internal object frame; + + /// The destination pointer being read into. + internal IntPtr destinationPtr; + + /// The cancellation token source, if any, for the Read operation internal CancellationTokenSource cts; - /* Used for iteration */ - internal long resumePtr; - internal long untilPtr; + /// Read buffers if Reading ObjectAllocator. + public CircularDiskReadBuffer readBuffers; + + /// The max offset on the main log page to iterate records when determining how many bytes in the ObjectLog to read. internal long maxPtr; - /// - /// Free - /// + public override string ToString() + => $"page {page}, devPgOffset {devicePageOffset}, ctx {context}, countdown {handle?.CurrentCount}, destPtr {destinationPtr} ({destinationPtr:X}), maxPtr {maxPtr}"; + + /// Currently nothing to free. public void Free() { - if (freeBuffer1 != null) - { - freeBuffer1.Return(); - freeBuffer1 = null; - } + } - if (freeBuffer2 != null) - { - freeBuffer2.Return(); - freeBuffer2 = null; - } + /// + public void DisposeHandle() + { + handle?.Dispose(); } } @@ -68,6 +79,13 @@ internal sealed class FlushCompletionTracker /// int count; + public override string ToString() + { + var compSemCount = completedSemaphore?.CurrentCount.ToString() ?? "null"; + var flushSemCount = completedSemaphore?.CurrentCount.ToString() ?? "null"; + return $"count {count}, compSemCount {compSemCount}, flushSemCount {flushSemCount}"; + } + /// /// Create a flush completion tracker /// @@ -91,8 +109,7 @@ public void CompleteFlush() completedSemaphore.Release(); } - public void WaitOneFlush() - => flushSemaphore?.Wait(); + public void WaitOneFlush() => flushSemaphore?.Wait(); } /// @@ -102,26 +119,51 @@ public void WaitOneFlush() public sealed class PageAsyncFlushResult { /// - /// Page + /// The index of the log Page being written /// public long page; + /// - /// Context + /// Context object for the callback /// public TContext context; + + /// + /// Flush buffers if flushing ObjectAllocator. + /// + public CircularDiskWriteBuffer flushBuffers; + /// - /// Count + /// Count of active pending flush operations; the callback decrements this and when it hits 0, the overall flush operation is complete. /// public int count; + /// + /// If true, this is a flush of a partial page. + /// internal bool partial; + internal long fromAddress; internal long untilAddress; + + /// + /// This is the record buffer, passed through the IO process to retain a reference to it so it will not be GC'd before the Flush write completes. + /// internal SectorAlignedMemory freeBuffer1; - internal SectorAlignedMemory freeBuffer2; + + /// + /// The event that is signaled by the callback so any waiting thread knows the IO has completed. + /// internal AutoResetEvent done; + internal FlushCompletionTracker flushCompletionTracker; + public override string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + return $"page {page}, ctx {context}, count {count}, partial {bstr(partial)}, fromAddr {fromAddress} ({untilAddress:X}), fromAddr {untilAddress} ({untilAddress:X}), flushCompTrack [{flushCompletionTracker}], circFlushBufs [{flushBuffers}]"; + } + /// /// Free /// @@ -132,13 +174,227 @@ public void Free() freeBuffer1.Return(); freeBuffer1 = null; } - if (freeBuffer2 != null) + + flushCompletionTracker?.CompleteFlush(); + } + } + + /// + /// A class to carry callback and context through operations that may chain callbacks. + /// + internal sealed class DiskWriteCallbackContext + { + /// If we had separate Writes for multiple spans of a single array, this is a refcounted wrapper for the ; + /// it is released after the write and if it is the final release, all spans have been written and the GCHandle is freed (and the object unpinned). + public RefCountedPinnedGCHandle refCountedGCHandle { get; private set; } + + /// Separate public Set() call so we ensure it is AddRef'd + /// + public void SetRefCountedHandle(RefCountedPinnedGCHandle refGcHandle) + { + Debug.Assert(!gcHandle.IsAllocated, "Cannot have both GCHandle and RefCountedPinnedGCHandle"); + refCountedGCHandle = refGcHandle; + refCountedGCHandle.AddRef(); + } + + /// If this Write is from a , this keeps its byte[] pinned during the Write. + /// It is freed (and the array unpinned) after the Write. Used instead of for only a single span of the array to avoid a heap allocation. + private readonly GCHandle gcHandle; + + /// The countdown callback for the entire partial flush, including s, external writes, and final sector-aligning write. + private readonly CountdownCallbackAndContext countdownCallbackAndContext; + + /// The countdown event if this write is associated with a . + private CountdownEvent bufferCountdownEvent; + + public override string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + var countdownString = bufferCountdownEvent is null ? "null" : bufferCountdownEvent.CurrentCount.ToString(); + var cbcString = countdownCallbackAndContext is null ? "null" : countdownCallbackAndContext.ToString(); + return $"refCntGcH [{refCountedGCHandle}], gcH {bstr(gcHandle.IsAllocated)}, countdown {countdownString}, cb&c {cbcString}"; + } + + public DiskWriteCallbackContext(CountdownCallbackAndContext callbackAndContext) + { + countdownCallbackAndContext = callbackAndContext; + callbackAndContext.Increment(); + } + + public DiskWriteCallbackContext(CountdownCallbackAndContext callbackAndContext, RefCountedPinnedGCHandle refGcHandle) : this(callbackAndContext) + => SetRefCountedHandle(refGcHandle); + + public DiskWriteCallbackContext(CountdownCallbackAndContext callbackAndContext, GCHandle gcHandle) : this(callbackAndContext) + => this.gcHandle = gcHandle; + + /// This write is associated with a so we need to signal the countdown event for that buffer when we are done. + public void SetBufferCountdownEvent(CountdownEvent countdownEvent) => bufferCountdownEvent = countdownEvent; + + public long Release() + { + refCountedGCHandle?.Release(); + if (gcHandle.IsAllocated) + gcHandle.Free(); + _ = bufferCountdownEvent?.Signal(); + return countdownCallbackAndContext?.Decrement() ?? 0; + } + } + + /// + /// Hold the callback and context for a refcounted callback and context. Used to ensure global completion of multi-buffer writes (which use a "local" + /// callback) before invoking the external callback. + /// + /// + /// The sequence is illustrated for flushes: + /// + /// Initialize the field to a new instance of this at the start of a partial flush + /// AddRef and Release for each operation (for flushes, there will be two levels of refcount: + /// + /// Per-buffer + /// Globally (within the ), to await the completion of all partial flushes before invoking the external callback. + /// + /// + /// + /// When the count hits zero, if the callback is not null, call it; it will only be set to non-null when we have completed a partial flush. This allows the count to drop to 0 and + /// be increased again throughout the partial flush, as various data spans are written. + /// + internal sealed class CountdownCallbackAndContext + { + /// Original caller's callback + public DeviceIOCompletionCallback callback; + /// Original caller's callback context + public object context; + /// Number of bytes written + private uint numBytes; + /// Number of in-flight operations + internal long count; + + public override string ToString() + { + var callbackString = callback is null ? "null" : callback.ToString(); + var contextString = callback is null ? "null" : context.ToString(); + return $"numBytes {numBytes}, count {count}, callback {callbackString}, context {context}"; + } + + public void Set(DeviceIOCompletionCallback callback, object context, uint numBytes) + { + this.callback = callback; + this.context = context; + this.numBytes = numBytes; + } + + internal void Increment() => _ = Interlocked.Increment(ref count); + + internal long Decrement() + { + var remaining = Interlocked.Decrement(ref count); + if (remaining == 0) + callback?.Invoke(errorCode: 0, numBytes, context); + return remaining; + } + } + + /// + /// Hold a and a refcount; free the handle when the refcount reaches 0. Used when multiple sections of the + /// same byte[] are being written, such as when it is split across segments. + /// + internal sealed class RefCountedPinnedGCHandle + { + /// The being held. + internal GCHandle gcHandle; + /// Number of in-flight operations + private long count; + + public override string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + return $"gcH {bstr(gcHandle.IsAllocated)}, count {count}"; + } + + internal RefCountedPinnedGCHandle(object targetObject, long initialCount) + { + gcHandle = GCHandle.Alloc(targetObject, GCHandleType.Pinned); + count = initialCount; + } + + internal RefCountedPinnedGCHandle(GCHandle gcHandle, long initialCount) + { + this.gcHandle = gcHandle; + count = initialCount; + } + + internal void AddRef() + { + ObjectDisposedException.ThrowIf(count <= 0, $"Uninitialized or final-released {nameof(RefCountedPinnedGCHandle)}"); + _ = Interlocked.Increment(ref count); + } + + internal void Release() + { + ObjectDisposedException.ThrowIf(count <= 0, $"Uninitialized or final-released {nameof(RefCountedPinnedGCHandle)}"); + if (Interlocked.Decrement(ref count) == 0 && gcHandle.IsAllocated) + gcHandle.Free(); + } + + internal object Target + { + get { - freeBuffer2.Return(); - freeBuffer2 = null; + ObjectDisposedException.ThrowIf(count <= 0 || !gcHandle.IsAllocated, $"Uninitialized or final-released {nameof(RefCountedPinnedGCHandle)}"); + return gcHandle.Target; } + } - flushCompletionTracker?.CompleteFlush(); + internal bool IsAllocated => gcHandle.IsAllocated; + } + + /// + /// A class to carry callback and context through operations that may chain callbacks. + /// + internal sealed class DiskReadCallbackContext + { + /// If we had separate Reads directly into multiple spans of a single byte[], such as across segments, this is a refcounted wrapper for the ; + /// it is released after the write and if it is the final release, all spans have been written and the GCHandle is freed (and the object unpinned). + public RefCountedPinnedGCHandle refCountedGCHandle { get; private set; } + + /// Separate public Set() call so we ensure it is AddRef'd + /// + public void SetRefCountedHandle(RefCountedPinnedGCHandle refGcHandle) + { + Debug.Assert(!gcHandle.IsAllocated, "Cannot have both GCHandle and RefCountedPinnedGCHandle"); + refCountedGCHandle = refGcHandle; + refCountedGCHandle.AddRef(); + } + + /// An event that can be waited for; the caller's callback will signal it if non-null. + internal CountdownEvent countdownEvent; + + /// If we had a Read directly into the byte[] of an , this is the that keps it pinned during the Read. + /// After the Read it is freed (and the object unpinned). + public GCHandle gcHandle; + + public override string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + return $"refCntGcH {refCountedGCHandle}, gcH {bstr(gcHandle.IsAllocated)}, countdown {countdownEvent?.CurrentCount}"; + } + + /// If non-null, this is the target buffer to copy data to (the copy is done by the caller's callback). + public byte[] CopyTarget => (byte[])(gcHandle.IsAllocated ? gcHandle.Target : refCountedGCHandle.Target); + + internal DiskReadCallbackContext(CountdownEvent countdownEvent) => this.countdownEvent = countdownEvent; + + internal DiskReadCallbackContext(CountdownEvent countdownEvent, RefCountedPinnedGCHandle refGcHandle) : this(countdownEvent) + => SetRefCountedHandle(refGcHandle); + + internal DiskReadCallbackContext(CountdownEvent countdownEvent, GCHandle gcHandle) : this(countdownEvent) + => this.gcHandle = gcHandle; + + public void Dispose() + { + if (gcHandle.IsAllocated) + gcHandle.Free(); + _ = (countdownEvent?.Signal()); } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/Status.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/Status.cs index c5acc553fcd..4019129695b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/Status.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/Status.cs @@ -84,37 +84,42 @@ internal Status(OperationStatus operationStatus) : this() /// /// Whether a Read or RMW found the key /// - public bool Found => (Record.statusCode & StatusCode.BasicMask) == StatusCode.Found; + public readonly bool Found => (Record.statusCode & StatusCode.BasicMask) == StatusCode.Found; /// /// Whether a Read or RMW did not find the key /// - public bool NotFound => (statusCode & StatusCode.BasicMask) == StatusCode.NotFound; + public readonly bool NotFound => (statusCode & StatusCode.BasicMask) == StatusCode.NotFound; /// /// Whether the operation went pending /// - public bool IsPending => statusCode == StatusCode.Pending; + public readonly bool IsPending => statusCode == StatusCode.Pending; /// /// Whether the operation went pending /// - public bool IsCompleted => !IsPending; + public readonly bool IsCompleted => !IsPending; /// /// Whether the operation is in an error state /// - public bool IsFaulted => statusCode == StatusCode.Error; + public readonly bool IsFaulted => statusCode == StatusCode.Error; /// /// Whether the operation was canceled /// - public bool IsCanceled => statusCode == StatusCode.Canceled; + public readonly bool IsCanceled => statusCode == StatusCode.Canceled; /// /// Whether the operation found an expired record /// - public bool Expired => (statusCode & StatusCode.Expired) == StatusCode.Expired; + public readonly bool IsExpired => (statusCode & StatusCode.Expired) == StatusCode.Expired; + + /// + /// Whether the operation found an expired record + /// + public readonly bool IsWrongType => (statusCode & StatusCode.WrongType) == StatusCode.WrongType; /// /// Whether the operation completed successfully, i.e., it is not pending and did not error out diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs index 996181e85c2..3bdab194296 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs @@ -119,7 +119,14 @@ internal enum StatusCode : byte /// CopiedRecordToReadCache = 0x50, - // unused 0x60, + /// + /// Indicates that an existing record was found but was of the wrong type for the requested operation. + /// + /// + /// See basic codes for details of usage. + /// + WrongType = 0x60, + // unused 0x70, /// diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs index 862ad32ac02..293475ec4f8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs @@ -127,20 +127,86 @@ internal static string PrettySize(long value) return v.ToString() + "B"; } - /// Rounds up value to alignment - /// Value to be aligned - /// Align to this - /// Aligned value + /// Rounds up to (which must be a power of two) [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int RoundUp(int value, int alignment) => (value + (alignment - 1)) & ~(alignment - 1); + public static int RoundUp(int value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundUp(int) alignment must be a power of two"); + return (value + (alignment - 1)) & ~(alignment - 1); + } + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint RoundUp(uint value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundUp(uint) alignment must be a power of two"); + return (value + ((uint)alignment - 1)) & ~((uint)alignment - 1); + } + + /// Rounds up to (which must be a power of two) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal static long RoundUp(long value, int alignment) { - Debug.Assert(IsPowerOfTwo(alignment), "RoundUp alignment must be a power of two"); + Debug.Assert(IsPowerOfTwo(alignment), "RoundUp(long) alignment must be a power of two"); return (value + (alignment - 1)) & ~(alignment - 1); } + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static ulong RoundUp(ulong value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundUp(ulong) alignment must be a power of two"); + return (value + ((uint)alignment - 1)) & ~((uint)alignment - 1); + } + + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int RoundDown(int value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundDown(int) alignment must be a power of two"); + return value & ~(alignment - 1); + } + + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static uint RoundDown(uint value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundDown(uint) alignment must be a power of two"); + return value & ~((uint)alignment - 1); + } + + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static long RoundDown(long value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundDown(long) alignment must be a power of two"); + return value & ~(alignment - 1); + } + + /// Rounds up to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static ulong RoundDown(ulong value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "RoundDown(ulong) alignment must be a power of two"); + return value & ~((uint)alignment - 1); + } + + /// Verifies that is aligned to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static bool IsAligned(long value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "IsAligned(long) alignment must be a power of two"); + return (value & (alignment - 1)) == 0; + } + + /// Verifies that is aligned to (which must be a power of two) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static bool IsAligned(ulong value, int alignment) + { + Debug.Assert(IsPowerOfTwo(alignment), "IsAligned(ulong) alignment must be a power of two"); + return (value & ((uint)alignment - 1)) == 0; + } + /// /// Is type blittable /// @@ -283,7 +349,8 @@ public static bool MonotonicUpdate(ref long variable, long newValue, out long ol do { oldValue = variable; - if (oldValue >= newValue) return false; + if (oldValue >= newValue) + return false; } while (Interlocked.CompareExchange(ref variable, newValue, oldValue) != oldValue); return true; } @@ -301,7 +368,8 @@ public static bool MonotonicUpdate(ref int variable, int newValue, out int oldVa do { oldValue = variable; - if (oldValue >= newValue) return false; + if (oldValue >= newValue) + return false; } while (Interlocked.CompareExchange(ref variable, newValue, oldValue) != oldValue); return true; } diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs index a6068064508..7af6bbfc45d 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs @@ -9,20 +9,22 @@ namespace Tsavorite.core public struct RecordFieldInfo { /// - /// The data length of the key for the new record (not including the length prefix) and may become overflow; see + /// The data length of the key for the record. Its behavior varies between the String and Object stores: + /// + /// String store: It is the data length of the Span + /// Object store: It is the data length of the Span (which may or may not Overflow) + /// /// - public int KeyDataSize; + public int KeySize; /// - /// The data length of the value for the new record. Its behavior varies between the String and Object stores: + /// The data length of the value for the record. Its behavior varies between the String and Object stores: /// - /// String store: It is the data length of the Span without any length prefix and may become overflow; see - /// Object store: If is specified it should be set to . - /// Otherwise it is handled the same as : the data length of the span (not including the length prefix) and may become overflow; - /// see + /// String store: It is the data length of the Span + /// Object store: It is either the data length of the Span (which may or may not Overflow) or if the Value is an Object /// /// - public int ValueDataSize; + public int ValueSize; /// Whether the value was specified to be an object. public bool ValueIsObject; @@ -35,6 +37,6 @@ public struct RecordFieldInfo /// public override string ToString() - => $"KeySize {KeyDataSize}, ValSize {ValueDataSize}, ValIsObj {ValueIsObject}, HasETag {HasETag}, HasExpir {HasExpiration}"; + => $"KeySize {KeySize}, ValSize {ValueSize}, ValIsObj {ValueIsObject}, HasETag {HasETag}, HasExpir {HasExpiration}"; } -} \ No newline at end of file +} diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs index edfec4b16fe..0a8be261ccc 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs @@ -2,9 +2,14 @@ // Licensed under the MIT license. using System.Diagnostics; +using System.Runtime.CompilerServices; namespace Tsavorite.core { +#pragma warning disable IDE0065 // Misplaced using directive + using static Utility; + using static VarbyteLengthUtility; + /// /// Struct for information about the key and the fields and their sizes in a record. /// @@ -13,12 +18,21 @@ public struct RecordSizeInfo /// The value length and whether optional fields are present. public RecordFieldInfo FieldInfo; - /// Whether the key was within the inline max key length. Set automatically by Tsavorite based on key size. + /// Whether the key was within the inline max key length. Set automatically by Tsavorite based on key size. public bool KeyIsInline; /// Whether the value was within the inline max value length. public bool ValueIsInline; + /// Varbyte indicator word, containing the Indicator Byte as well as the key and value lengths; see . + public long IndicatorWord; + + /// Number of bytes in key length; see . + public int KeyLengthBytes; + + /// Number of bytes in value length; see . + public int ValueLengthBytes; + /// Whether the value was specified to be an object. public readonly bool ValueIsObject => FieldInfo.ValueIsObject; @@ -29,13 +43,13 @@ public struct RecordSizeInfo public readonly bool ValueIsOverflow => !ValueIsInline && !ValueIsObject; /// Returns the inline length of the key (the amount it will take in the record). - public readonly int InlineTotalKeySize => KeyIsInline ? FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + public readonly int InlineKeySize => KeyIsInline ? FieldInfo.KeySize : ObjectIdMap.ObjectIdSize; /// Returns the inline length of the value (the amount it will take in the record). - public readonly int InlineTotalValueSize => ValueIsInline ? FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + public readonly int InlineValueSize => ValueIsInline ? FieldInfo.ValueSize : ObjectIdMap.ObjectIdSize; /// The max inline value size if this is a record in the string log. - public int MaxInlineValueSpanSize { readonly get; internal set; } + public int MaxInlineValueSize { readonly get; internal set; } /// The inline size of the record (in the main log). If Key and/or Value are overflow (or value is Object), /// then their contribution to inline length is just . @@ -50,12 +64,45 @@ public struct RecordSizeInfo /// Size to allocate for Expiration if it will be included, else 0. public readonly int ExpirationSize => FieldInfo.HasExpiration ? LogRecord.ExpirationSize : 0; + /// If true, this record specification has non-inline Key or Value or both, so will require the object log. + public readonly bool RecordHasObjects => !KeyIsInline || !ValueIsInline; + + /// Size to allocate for Expiration if it will be included, else 0. + public readonly int ObjectLogPositionSize => RecordHasObjects ? LogRecord.ObjectLogPositionSize : 0; + /// Size to allocate for all optional fields that will be included; possibly 0. - public readonly int OptionalSize => ETagSize + ExpirationSize; + public readonly int OptionalSize => ETagSize + ExpirationSize + ObjectLogPositionSize; /// Whether these values are set (default instances are used for Delete internally, for example). public readonly bool IsSet => AllocatedInlineRecordSize != 0; + internal void CalculateSizes(int keySize, int valueSize) + { + // Varbyte lengths. Add optionalSize to the effective value size when calculating valueLengthBytes so the value can grow if optionals are removed + // (otherwise the filler-related calculations would require additional logic to constrain value size to the # of bytes we calculate here). + IndicatorWord = ConstructInlineVarbyteLengthWord(keySize, valueSize, flagBits: 0, out KeyLengthBytes, out ValueLengthBytes); + + // Record + var numVarbytes = NumIndicatorBytes + KeyLengthBytes + ValueLengthBytes; + ActualInlineRecordSize = RecordInfo.Size + numVarbytes + keySize + valueSize + OptionalSize; + AllocatedInlineRecordSize = RoundUp(ActualInlineRecordSize, Constants.kRecordAlignment); + } + + /// Gets the value length currently in the record (e.g. before being updated with FieldInfo.ValueSize). + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly unsafe int GetValueInlineLength(long recordPhysicalAddress) + => (int)ReadVarbyteLength(ValueLengthBytes, (byte*)(recordPhysicalAddress + RecordInfo.Size + NumIndicatorBytes + KeyLengthBytes)); + + /// Gets the Key address in the record. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly unsafe long GetKeyAddress(long recordPhysicalAddress) + => recordPhysicalAddress + RecordInfo.Size + NumIndicatorBytes + KeyLengthBytes + ValueLengthBytes; + + /// Gets the Value address in the record. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly unsafe long GetValueAddress(long recordPhysicalAddress) + => recordPhysicalAddress + RecordInfo.Size + NumIndicatorBytes + KeyLengthBytes + ValueLengthBytes + InlineKeySize; + /// /// Called from Upsert or RMW methods for Span Values with the actual data size of the update value; ensures consistency between the Get*FieldInfo methods and the actual update methods. /// Usually called directly to save the cost of calculating actualDataSize twice (in Get*FieldInfo and the actual update methods). @@ -63,7 +110,7 @@ public struct RecordSizeInfo [Conditional("DEBUG")] public static void AssertValueDataLength(int dataSize, in RecordSizeInfo sizeInfo) { - Debug.Assert(sizeInfo.FieldInfo.ValueDataSize == dataSize, $"Mismatch between expected value size {sizeInfo.FieldInfo.ValueDataSize} and actual value size {dataSize}"); + Debug.Assert(sizeInfo.FieldInfo.ValueSize == dataSize, $"Mismatch between expected value size {sizeInfo.FieldInfo.ValueSize} and actual value size {dataSize}"); } /// Called from Upsert or RMW methods with the final record info; ensures consistency between the Get*FieldInfo methods and the actual update methods./// diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs index 02a90f5a1ca..6ed724067e1 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs @@ -31,16 +31,16 @@ public override bool Reader(in TSourceLogRecord srcLogRecord, /// public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref PinnedSpanByte input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = input.Length }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = input.Length }; /// public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref PinnedSpanByte input) - => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + => new() { KeySize = key.Length, ValueSize = input.Length }; /// public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref PinnedSpanByte input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; /// public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref PinnedSpanByte input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; /// public override void ConvertOutputToHeap(ref PinnedSpanByte input, ref SpanByteAndMemory output) diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs index 9713e8504a8..ae94b21bd36 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs @@ -22,7 +22,7 @@ public unsafe SpanByteHeapContainer(ReadOnlySpan item, SectorAlignedBuffer } mem = pool.Get(item.TotalSize()); item.SerializeTo(mem.GetValidPointer()); - this.pinnedSpanByte = PinnedSpanByte.FromLengthPrefixedPinnedPointer(mem.GetValidPointer()); + pinnedSpanByte = PinnedSpanByte.FromLengthPrefixedPinnedPointer(mem.GetValidPointer()); } public unsafe ref PinnedSpanByte Get() => ref pinnedSpanByte; diff --git a/libs/storage/Tsavorite/cs/test/BasicLockTests.cs b/libs/storage/Tsavorite/cs/test/BasicLockTests.cs index 9b264f84024..41efd8c5048 100644 --- a/libs/storage/Tsavorite/cs/test/BasicLockTests.cs +++ b/libs/storage/Tsavorite/cs/test/BasicLockTests.cs @@ -176,12 +176,12 @@ public unsafe void CollidingDeletedRecordTest([Values(UpdateOp.RMW, UpdateOp.Ups HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(deleteKey)); ClassicAssert.IsTrue(store.FindTag(ref hei), "Cannot find deleteKey entry"); ClassicAssert.Greater(hei.Address, LogAddress.kInvalidAddress, "Couldn't find deleteKey Address"); - var physicalAddress = store.hlog.GetPhysicalAddress(hei.Address); + var physicalAddress = store.hlogBase.GetPhysicalAddress(hei.Address); var lookupKey = LogRecord.GetInlineKey(physicalAddress); ClassicAssert.AreEqual(collidingKeyNum, lookupKey.AsRef(), "Expected collidingKey"); // Backtrace to deleteKey - physicalAddress = store.hlog.GetPhysicalAddress(LogRecord.GetInfo(physicalAddress).PreviousAddress); + physicalAddress = store.hlogBase.GetPhysicalAddress(LogRecord.GetInfo(physicalAddress).PreviousAddress); lookupKey = LogRecord.GetInlineKey(physicalAddress); ClassicAssert.AreEqual(deleteKey.AsRef(), lookupKey.AsRef(), "Expected deleteKey"); ClassicAssert.IsFalse(LogRecord.GetInfo(physicalAddress).Tombstone, "Tombstone should be false"); @@ -252,7 +252,7 @@ public unsafe void SetInvalidOnException([Values] UpdateOp updateOp) ClassicAssert.IsTrue(threw, "Test should have thrown"); ClassicAssert.AreEqual(expectedThrowAddress, session.functions.initialUpdaterThrowAddress, "Unexpected throw address"); - var physicalAddress = store.hlog.GetPhysicalAddress(expectedThrowAddress); + var physicalAddress = store.hlogBase.GetPhysicalAddress(expectedThrowAddress); var recordInfo = LogRecord.GetInfo(physicalAddress); ClassicAssert.IsTrue(recordInfo.Invalid, "Expected Invalid record"); } diff --git a/libs/storage/Tsavorite/cs/test/CancellationTests.cs b/libs/storage/Tsavorite/cs/test/CancellationTests.cs index 0e230073ae1..54668cbf94a 100644 --- a/libs/storage/Tsavorite/cs/test/CancellationTests.cs +++ b/libs/storage/Tsavorite/cs/test/CancellationTests.cs @@ -118,16 +118,16 @@ public override bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo si /// public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref int input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(int) }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = sizeof(int) }; /// public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref int input) - => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(int) }; + => new() { KeySize = key.Length, ValueSize = sizeof(int) }; /// public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref int input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; /// public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref int input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } IDevice log; diff --git a/libs/storage/Tsavorite/cs/test/LogRecordTests.cs b/libs/storage/Tsavorite/cs/test/LogRecordTests.cs index db645c1a5b9..1593cc95e55 100644 --- a/libs/storage/Tsavorite/cs/test/LogRecordTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogRecordTests.cs @@ -5,11 +5,13 @@ using System.Runtime.InteropServices; using NUnit.Framework; using Tsavorite.core; -using static Tsavorite.core.Utility; using static Tsavorite.test.TestUtils; namespace Tsavorite.test.LogRecordTests { + using static Utility; + using static VarbyteLengthUtility; + /// /// This also tests and , /// which in turn tests . @@ -19,12 +21,30 @@ unsafe class LogRecordTests { long nativePointer; ObjectIdMap objectIdMap; - SectorAlignedBufferPool bufferPool; - SectorAlignedMemory recordBuffer; + SpanByteAndMemory sbamOutput; + +#pragma warning disable IDE1006 // Naming Styles + const int initialKeyLen = 10; + const int initialValueLen = 40; + const int initialVarbyteSize = 3; // indicator byte, and 1 byte each for key and value len + const int initialOptionalSize = sizeof(long) * 2; + + const int maxInlineKeySize = 64; + const int maxInlineValueSize = 128; + + const long initialETag = 1000; + const long initialExpiration = 2000; +#pragma warning restore IDE1006 // Naming Styles + + int expectedInitialActualInlineRecordSize; + int expectedInitialAllocatedInlineRecordSize; [SetUp] public void Setup() { + expectedInitialActualInlineRecordSize = RecordInfo.Size + initialVarbyteSize + initialKeyLen + initialValueLen + initialOptionalSize; + expectedInitialAllocatedInlineRecordSize = RoundUp(expectedInitialActualInlineRecordSize, Constants.kRecordAlignment); + DeleteDirectory(MethodTestDir); objectIdMap = new(); } @@ -38,43 +58,124 @@ public void TearDown() NativeMemory.AlignedFree((void*)nativePointer); nativePointer = IntPtr.Zero; } - recordBuffer?.Return(); - recordBuffer = null; - bufferPool = null; + sbamOutput.Dispose(); DeleteDirectory(MethodTestDir); } - const int initialKeyLen = 10; - const int initialValueLen = 40; - - const int expectedInitialActualInlineRecordSize = 82; // based on the initial values - const int expectedInitialAllocatedInlineRecordSize = 88; - - const int maxInlineKeySize = 64; - const int maxInlineValueSize = 128; - - const long initialETag = 1000; - const long InitialExpiration = 2000; - - void UpdateRecordSizeInfo(ref RecordSizeInfo sizeInfo, int keySize = -1, int valueSize = -1) + static void UpdateRecordSizeInfo(ref RecordSizeInfo sizeInfo, int keySize = -1, int valueSize = -1) { if (keySize > 0) - sizeInfo.FieldInfo.KeyDataSize = keySize; + sizeInfo.FieldInfo.KeySize = keySize; if (valueSize > 0) - sizeInfo.FieldInfo.ValueDataSize = valueSize; + sizeInfo.FieldInfo.ValueSize = valueSize; // Key - sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeyDataSize <= maxInlineKeySize; - keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeySize <= maxInlineKeySize; + keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeySize : ObjectIdMap.ObjectIdSize; // Value - sizeInfo.MaxInlineValueSpanSize = maxInlineValueSize; - sizeInfo.ValueIsInline = !sizeInfo.ValueIsObject && sizeInfo.FieldInfo.ValueDataSize <= maxInlineValueSize; - valueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + sizeInfo.MaxInlineValueSize = maxInlineValueSize; + sizeInfo.ValueIsInline = !sizeInfo.ValueIsObject && sizeInfo.FieldInfo.ValueSize <= maxInlineValueSize; + valueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueSize : ObjectIdMap.ObjectIdSize; // Record - sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; - sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); + sizeInfo.CalculateSizes(keySize, valueSize); + } + + [Test] + [Category(LogRecordCategory), Category(SmokeTestCategory)] + //[Repeat(900)] + public unsafe void VarbyteWordTests() + { + long value; + byte* ptr = (byte*)&value; + long indicatorAddress = (long)ptr; + + Assert.That(GetByteCount(0), Is.EqualTo(1)); + + int inputKeyLength = 16; + var inputValueLength = 1 << 8 - 1; + byte* keyPtr, valuePtr; + + // Test 1- and 2-byte valueLengthByte boundary with 1-keyLengthByte key + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(1)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out int keyLengthBytes, out int valueLengthBytes); + Assert.That(keyLengthBytes, Is.EqualTo(1)); + Assert.That(valueLengthBytes, Is.EqualTo(1)); + VerifyKeyAndValue(); + + inputValueLength = 1 << 8; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(2)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out _ /*keyLengthBytes*/, out valueLengthBytes); + Assert.That(valueLengthBytes, Is.EqualTo(2)); + VerifyKeyAndValue(); + + // Test 2- and 3-byte valueLengthByte boundary with 2-keyLengthByte key + inputKeyLength = inputValueLength = (1 << 16) - 1; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(2)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out keyLengthBytes, out valueLengthBytes); + Assert.That(keyLengthBytes, Is.EqualTo(2)); + Assert.That(valueLengthBytes, Is.EqualTo(2)); + VerifyKeyAndValue(); + + inputValueLength = 1 << 16; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(3)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out _ /*keyLengthBytes*/, out valueLengthBytes); + Assert.That(valueLengthBytes, Is.EqualTo(3)); + VerifyKeyAndValue(); + + // Test 3- and 4-byte valueLengthByte boundary with 3-keyLengthByte key + inputKeyLength = inputValueLength = (1 << 24) - 1; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(3)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out keyLengthBytes, out valueLengthBytes); + Assert.That(keyLengthBytes, Is.EqualTo(3)); + Assert.That(valueLengthBytes, Is.EqualTo(3)); + VerifyKeyAndValue(); + + inputValueLength = 1 << 24; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(4)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out _ /*keyLengthBytes*/, out valueLengthBytes); + Assert.That(valueLengthBytes, Is.EqualTo(4)); + VerifyKeyAndValue(); + + // Test max ValueLength + inputValueLength = int.MaxValue; + Assert.That(GetByteCount(inputValueLength), Is.EqualTo(4)); + value = ConstructInlineVarbyteLengthWord(inputKeyLength, inputValueLength, flagBits: 0, out _ /*keyLengthBytes*/, out valueLengthBytes); + Assert.That(valueLengthBytes, Is.EqualTo(4)); + VerifyKeyAndValue(); + + void VerifyKeyAndValue() + { + keyPtr = GetFieldPtr(indicatorAddress, isKey: true, out var keyLengthPtr, out var outputKeyLengthBytes, out var outputKeyLength); + valuePtr = GetFieldPtr(indicatorAddress, isKey: false, out var valueLengthPtr, out var outputValueLengthBytes, out var outputValueLength); + Assert.That(outputKeyLengthBytes, Is.EqualTo(keyLengthBytes)); + Assert.That(outputKeyLength, Is.EqualTo(inputKeyLength)); + Assert.That(outputValueLength, Is.EqualTo(inputValueLength)); + Assert.That((long)keyPtr, Is.EqualTo((long)(ptr + NumIndicatorBytes + outputKeyLengthBytes + outputValueLengthBytes))); + Assert.That((long)keyLengthPtr, Is.EqualTo((long)(ptr + 1))); + Assert.That((long)valuePtr, Is.EqualTo((long)(keyPtr + outputKeyLength))); + Assert.That((long)valueLengthPtr, Is.EqualTo((long)(keyLengthPtr + keyLengthBytes))); + + // Now test the word-based forms for in-memory use only + if (inputValueLength <= int.MaxValue) + { + // First verify reading from the pointer-update version. + long word = *(long*)indicatorAddress; + var keyLength2 = ReadVarbyteLengthInWord(word, precedingNumBytes: 0, keyLengthBytes); + var valueLength2 = ReadVarbyteLengthInWord(word, keyLengthBytes, valueLengthBytes); + Assert.That(keyLength2, Is.EqualTo(inputKeyLength)); + Assert.That(valueLength2, Is.EqualTo(inputValueLength)); + + word = 0x7171717171717171; // Initialize with a bit pattern to mask off + WriteVarbyteLengthInWord(ref word, inputKeyLength, precedingNumBytes: 0, keyLengthBytes); // Write key + WriteVarbyteLengthInWord(ref word, inputValueLength, precedingNumBytes: keyLengthBytes, valueLengthBytes); // Write value + keyLength2 = ReadVarbyteLengthInWord(word, precedingNumBytes: 0, keyLengthBytes); + valueLength2 = ReadVarbyteLengthInWord(word, precedingNumBytes: keyLengthBytes, valueLengthBytes); + Assert.That(keyLength2, Is.EqualTo(inputKeyLength)); + Assert.That(valueLength2, Is.EqualTo(inputValueLength)); + } + } } [Test] @@ -93,7 +194,7 @@ public unsafe void InlineBasicTest() // Shrink var offset = 12; - sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + sizeInfo.FieldInfo.ValueSize = initialValueLen - offset; Assert.That(logRecord.TrySetValueLength(in sizeInfo), Is.True); Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); @@ -104,7 +205,7 @@ public unsafe void InlineBasicTest() // Grow within range offset = 6; - sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + sizeInfo.FieldInfo.ValueSize = initialValueLen - offset; Assert.That(logRecord.TrySetValueLength(in sizeInfo), Is.True); Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); @@ -115,11 +216,11 @@ public unsafe void InlineBasicTest() // Grow beyond range offset = -10; - sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + sizeInfo.FieldInfo.ValueSize = initialValueLen - offset; Assert.That(logRecord.TrySetValueLength(in sizeInfo), Is.False); // Restore to original - sizeInfo.FieldInfo.ValueDataSize = initialValueLen; + sizeInfo.FieldInfo.ValueSize = initialValueLen; Assert.That(logRecord.TrySetValueLength(in sizeInfo), Is.True); Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); @@ -195,146 +296,24 @@ public unsafe void ConversionTest() [Test] [Category(LogRecordCategory), Category(SmokeTestCategory)] - //[Repeat(900)] - public unsafe void SerializationTest() + public void CopyDiskLogRecordToLogRecord() { - Span key = stackalloc byte[initialKeyLen]; - Span value = stackalloc byte[initialValueLen]; - Span overflowValue = stackalloc byte[maxInlineValueSize + 12]; - - key.Fill(0x42); - value.Fill(0x43); - overflowValue.Fill(0x53); - - // Local diskLogRecord. We keep recordBuffer outside DiskLogRecord for the reuse scenario. - DiskLogRecord diskLogRecord; - var valueSerializer = new TestObjectValue.Serializer(); - bufferPool = new(recordSize: 1, sectorSize: 512); - - var sizeInfo = new RecordSizeInfo(); - InitializeRecord(key, value, ref sizeInfo, out var logRecord, out var expectedFillerLengthAddress, out var expectedFillerLength, out long eTag, out long expiration); - diskLogRecord = new(); - diskLogRecord.Serialize(in logRecord, bufferPool, valueSerializer, ref recordBuffer); - Assert.That(diskLogRecord.Info.RecordIsInline); - // verify inline copy by checking SerializedSize - Assert.That(diskLogRecord.GetSerializedLength(), Is.EqualTo(RoundUp(logRecord.ActualRecordSize, Constants.kRecordAlignment))); - // verify getting the key and value - length and data; eTag; expiration - Assert.That(diskLogRecord.Key.SequenceEqual(logRecord.Key)); - Assert.That(diskLogRecord.ValueSpan.SequenceEqual(logRecord.ValueSpan)); - Assert.That(!diskLogRecord.Info.ValueIsObject); - Assert.That(diskLogRecord.ETag, Is.EqualTo(eTag)); - Assert.That(diskLogRecord.Expiration, Is.EqualTo(expiration)); - - // From here down in this test we want diskLogRecord to be serialized in IndicatorByte format (varbyte). - var optionalLength = 2 * sizeof(long); - var expectedValueLengthBytes = 1; - - // Convert to overflow. Because objectIdSize is the same as InlineLengthPrefixSize, our value space will shrink by the original value data size. - var offset = value.Length; - ConvertToOverflow(overflowValue, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration, offset); - diskLogRecord = new(); - diskLogRecord.Serialize(in logRecord, bufferPool, valueSerializer, ref recordBuffer); - Assert.That(!diskLogRecord.Info.RecordIsInline); - // verify out-of-line copy by checking SerializedSize - Assert.That(diskLogRecord.GetSerializedLength(), Is.GreaterThan(RoundUp(logRecord.ActualRecordSize, Constants.kRecordAlignment))); - // verify indicator byte - Assert.That(diskLogRecord.Version, Is.EqualTo(0)); - // verify getting the key and value - length and data; eTag; expiration - Assert.That(diskLogRecord.Key.Length, Is.EqualTo(key.Length)); - Assert.That(diskLogRecord.Key.SequenceEqual(logRecord.Key)); - Assert.That(diskLogRecord.ValueSpan.Length, Is.EqualTo(overflowValue.Length)); - Assert.That(diskLogRecord.ValueSpan.SequenceEqual(logRecord.ValueSpan)); - Assert.That(!diskLogRecord.Info.ValueIsObject); - Assert.That(!diskLogRecord.Info.ValueIsInline, "To avoid issues with Info.RecordIsInline, varbyte-format DiskLogRecords do not set Info.ValueIsInline; see discussion in SerializeCommonVarByteFields"); - Assert.That(diskLogRecord.ETag, Is.EqualTo(eTag)); - Assert.That(diskLogRecord.Expiration, Is.EqualTo(expiration)); - - // Convert to Object. Because objectIdSize is the same as InlineLengthPrefixSize, we can reuse the same offset as above. - ConvertToObject(ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration, offset); + Assert.Ignore("TODO CopyDiskLogRecordToLogRecord"); + } - // Now test different value sizes, using TestLargeObjectValue to test large objects. - var largeValueSerializer = new TestLargeObjectValue.Serializer(); - for (var ii = 0; ii < sizeof(int); ++ii) - { - ///////////////////////////// - // Set up the LogRecord with the object. - ///////////////////////////// - var valueDataSize = (1 << (ii * 8)) + 42; // TODO: test long values - var valueObject = new TestLargeObjectValue(valueDataSize); - Array.Fill(valueObject.value, (byte)ii); - sizeInfo.FieldInfo.ValueDataSize = ObjectIdMap.ObjectIdSize; - sizeInfo.FieldInfo.ValueIsObject = true; - UpdateRecordSizeInfo(ref sizeInfo); - Assert.That(logRecord.TrySetValueObject(valueObject, in sizeInfo), Is.True); - - expectedValueLengthBytes = 1; // Non-serialized object so only a 1-byte "0" length - var expectedKeyDataOffset = RecordInfo.GetLength() + 1 + 1 + expectedValueLengthBytes; // IndicatorByte + key length byte - var expectedKeyDataAddress = diskLogRecord.physicalAddress + expectedKeyDataOffset; - - ///////////////////////////// - // Serialize with a null object serializer to copy the object instance rather than serializing it into space in the record buffer. - ///////////////////////////// - diskLogRecord = new(); - diskLogRecord.Serialize(in logRecord, bufferPool, valueSerializer: null, ref recordBuffer); - Assert.That(diskLogRecord.Version, Is.EqualTo(0)); - expectedKeyDataAddress = diskLogRecord.physicalAddress + expectedKeyDataOffset; - - var keyInfo = diskLogRecord.KeyInfo; - Assert.That(keyInfo.length, Is.EqualTo(initialKeyLen)); - Assert.That(keyInfo.dataAddress, Is.EqualTo(expectedKeyDataAddress)); - - var expectedSerializedValueLength = 0; // The object instance was copied; no serialization was done so the length is zero. - var valueInfo = diskLogRecord.ValueInfo; - Assert.That(valueInfo.length, Is.EqualTo(expectedSerializedValueLength)); - Assert.That(valueInfo.dataAddress, Is.EqualTo(expectedKeyDataAddress + keyInfo.length)); - - Assert.That(diskLogRecord.ETag, Is.EqualTo(eTag)); - Assert.That(diskLogRecord.Expiration, Is.EqualTo(expiration)); - - Assert.That(diskLogRecord.ValueObject, Is.Not.Null); - Assert.That(new Span(((TestLargeObjectValue)diskLogRecord.ValueObject).value).SequenceEqual(new Span(((TestLargeObjectValue)logRecord.ValueObject).value))); - var expectedRecordSize = RoundUp(expectedKeyDataOffset + key.Length + expectedSerializedValueLength + optionalLength, Constants.kRecordAlignment); - Assert.That(diskLogRecord.GetSerializedLength(), Is.EqualTo(expectedRecordSize)); - - ///////////////////////////// - // Serialize with an object serializer to allocate space in the record buffer and serialize the object into it. - ///////////////////////////// - diskLogRecord = new(); - diskLogRecord.Serialize(in logRecord, bufferPool, largeValueSerializer, ref recordBuffer); - Assert.That(diskLogRecord.Version, Is.EqualTo(0)); - - expectedValueLengthBytes = ii + 1; // Serialized object so the value length is used - expectedKeyDataOffset = RecordInfo.GetLength() + 1 + 1 + expectedValueLengthBytes; // IndicatorByte + key length byte - expectedKeyDataAddress = diskLogRecord.physicalAddress + expectedKeyDataOffset; - - keyInfo = diskLogRecord.KeyInfo; - Assert.That(keyInfo.length, Is.EqualTo(initialKeyLen)); - Assert.That(keyInfo.dataAddress, Is.EqualTo(expectedKeyDataAddress)); - - valueInfo = diskLogRecord.ValueInfo; - Assert.That(valueInfo.length, Is.EqualTo(valueObject.DiskSize)); - Assert.That(valueInfo.dataAddress, Is.EqualTo(expectedKeyDataAddress + keyInfo.length)); - - Assert.That(diskLogRecord.ETag, Is.EqualTo(eTag)); - Assert.That(diskLogRecord.Expiration, Is.EqualTo(expiration)); - - Assert.That(diskLogRecord.Info.ValueIsObject); - expectedSerializedValueLength = (int)valueObject.DiskSize; - expectedRecordSize = RoundUp(expectedKeyDataOffset + key.Length + expectedSerializedValueLength + optionalLength, Constants.kRecordAlignment); - Assert.That(diskLogRecord.GetSerializedLength(), Is.EqualTo(expectedRecordSize)); - - Assert.That(diskLogRecord.DeserializeValueObject(largeValueSerializer), Is.Not.Null); - Assert.That(diskLogRecord.ValueObject, Is.Not.Null); - Assert.That(new Span(((TestLargeObjectValue)diskLogRecord.ValueObject).value).SequenceEqual(new Span(((TestLargeObjectValue)logRecord.ValueObject).value))); - } + [Test] + [Category(LogRecordCategory), Category(SmokeTestCategory)] + public void SerializeToMemoryPool() + { + Assert.Ignore("TODO SerializeToMemoryPool"); } private void InitializeRecord(Span key, Span value, ref RecordSizeInfo sizeInfo, out LogRecord logRecord, out long expectedFillerLengthAddress, out long expectedFillerLength, out long eTag, out long expiration) { sizeInfo.FieldInfo = new() { - KeyDataSize = initialKeyLen, - ValueDataSize = initialValueLen, + KeySize = initialKeyLen, + ValueSize = initialValueLen, HasETag = true, HasExpiration = true }; @@ -349,22 +328,12 @@ private void InitializeRecord(Span key, Span value, ref RecordSizeIn long recordEndAddress = nativePointer + sizeInfo.AllocatedInlineRecordSize; logRecord = new LogRecord(nativePointer, objectIdMap) { InfoRef = default }; - logRecord.InfoRef.SetKeyIsInline(); - logRecord.InfoRef.SetValueIsInline(); - - // SerializeKey - var keySpan = LogField.SetInlineDataLength(logRecord.KeyAddress, key.Length); - key.CopyTo(keySpan); - _ = LogField.SetInlineDataLength(logRecord.ValueAddress, value.Length); + logRecord.InitializeRecord(key, in sizeInfo); // InitializeValue Assert.That(logRecord.ValueSpan.Length, Is.EqualTo(initialValueLen)); - // FillerLength is set after initialization of Value field, and must be done before actually setting the ValueSpan - // (it ignores optionals as it's called before they're set up). - logRecord.SetFillerLength(sizeInfo.AllocatedInlineRecordSize); - - expectedFillerLengthAddress = logRecord.ValueAddress + value.Length + LogField.InlineLengthPrefixSize; + expectedFillerLengthAddress = logRecord.physicalAddress + RecordInfo.Size + initialVarbyteSize + key.Length + value.Length; // no OptionalsLength expectedFillerLength = recordEndAddress - expectedFillerLengthAddress; Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); @@ -389,7 +358,7 @@ private void InitializeRecord(Span key, Span value, ref RecordSizeIn Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); - expiration = InitialExpiration; + expiration = initialExpiration; Assert.That(logRecord.TrySetExpiration(expiration), Is.True); Assert.That(logRecord.ETag, Is.EqualTo(eTag)); @@ -401,7 +370,7 @@ private void InitializeRecord(Span key, Span value, ref RecordSizeIn private void ConvertToOverflow(Span overflowValue, ref RecordSizeInfo sizeInfo, ref LogRecord logRecord, long expectedFillerLengthAddress, long expectedFillerLength, long eTag, long expiration, int offset) { - sizeInfo.FieldInfo.ValueDataSize = overflowValue.Length; + sizeInfo.FieldInfo.ValueSize = overflowValue.Length; sizeInfo.FieldInfo.ValueIsObject = false; UpdateRecordSizeInfo(ref sizeInfo); @@ -421,7 +390,7 @@ private void ConvertToOverflow(Span overflowValue, ref RecordSizeInfo size private void ConvertToObject(ref RecordSizeInfo sizeInfo, ref LogRecord logRecord, long expectedFillerLengthAddress, long expectedFillerLength, long eTag, long expiration, int offset) { - sizeInfo.FieldInfo.ValueDataSize = ObjectIdMap.ObjectIdSize; + sizeInfo.FieldInfo.ValueSize = ObjectIdMap.ObjectIdSize; sizeInfo.FieldInfo.ValueIsObject = true; UpdateRecordSizeInfo(ref sizeInfo); @@ -442,7 +411,7 @@ private void ConvertToObject(ref RecordSizeInfo sizeInfo, ref LogRecord logRecor private void RestoreToOriginal(Span value, ref RecordSizeInfo sizeInfo, ref LogRecord logRecord, long expectedFillerLengthAddress, long expectedFillerLength, long eTag, long expiration) { - sizeInfo.FieldInfo.ValueDataSize = initialValueLen; + sizeInfo.FieldInfo.ValueSize = initialValueLen; sizeInfo.FieldInfo.ValueIsObject = false; UpdateRecordSizeInfo(ref sizeInfo); diff --git a/libs/storage/Tsavorite/cs/test/LogScanTests.cs b/libs/storage/Tsavorite/cs/test/LogScanTests.cs index 521d1cc5efb..55007613e1b 100644 --- a/libs/storage/Tsavorite/cs/test/LogScanTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogScanTests.cs @@ -302,7 +302,7 @@ public void ScanWithoutRecoverTest([Values] TestUtils.DeviceType deviceType) // Read the log int currentEntry = 9; // since starting at specified address of 1000, need to set current entry as 9 so verification starts at proper spot - using (var iter = log.Scan(log.SetAddressType(1000), LogAddress.MaxValidAddress, recover: false)) + using (var iter = log.Scan(1000, LogAddress.MaxValidAddress, recover: false)) { while (iter.GetNext(out byte[] result, out _, out _)) { diff --git a/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs b/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs index 7e33f1533bf..644cadc2015 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs @@ -7,7 +7,7 @@ using Tsavorite.core; using static Tsavorite.test.TestUtils; -namespace Tsavorite.test +namespace Tsavorite.test.Objects { /// /// This also tests and , @@ -82,8 +82,7 @@ void runLoadFreeListThread(int tid) // After being freed, the slot in the objectVector should be cleared (so objects are freed as early as possible). var objectId = tid * MultiLevelPageArray.ChapterSize + page; map.Free(objectId); - if (map.Get(objectId) is not null) - Assert.Fail("map.Get(objectId) should be null after Free() pt 1"); + Assert.That(map.GetHeapObject(objectId), Is.Null, "map.GetHeapObject(objectId) should be null after Free() pt 1"); } } @@ -115,8 +114,7 @@ void runAllocateFromFreeListThread(int tid) Assert.Fail("objectId should be less than allocatedCount"); // Make sure the slot in the objectVector is still cleared. - if (map.Get(objectId) is not null) - Assert.Fail("map.Get(objectId) should be null after Free() pt 2"); + Assert.That(map.GetHeapObject(objectId), Is.Null, "map.GetHeapObject(objectId) should be null after Free() pt 2"); } } diff --git a/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs b/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs index 0ccec9a6f30..7571bfd8f44 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs @@ -7,7 +7,7 @@ using Tsavorite.core; using static Tsavorite.test.TestUtils; -namespace Tsavorite.test +namespace Tsavorite.test.Objects { using ClassAllocator = ObjectAllocator>; using ClassStoreFunctions = StoreFunctions; @@ -330,23 +330,30 @@ private bool DoUpdater(in TSourceLogRecord srcLogRecord, ref L srcValue = ((TestObjectValue)srcLogRecord.ValueObject).value; output.value = srcLogRecord.Info.ValueIsObject ? (TestObjectValue)srcLogRecord.ValueObject : new TestObjectValue { value = (int)srcValue }; - output.value.value += input.value; + var result = false; switch (output.destValueStyle) { case TestValueStyle.Inline: ValueStruct valueStruct = new() { vfield1 = srcValue + input.value, vfield2 = (srcValue + input.value) * 100 }; - return logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref valueStruct), in sizeInfo); + result = logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref valueStruct), in sizeInfo); + break; case TestValueStyle.Overflow: Span overflowValue = GetOverflowValueSpanByte(); overflowValue.AsRef() = srcValue + input.value; - return logRecord.TrySetValueSpan(overflowValue, in sizeInfo); + result = logRecord.TrySetValueSpan(overflowValue, in sizeInfo); + break; case TestValueStyle.Object: - return logRecord.TrySetValueObject(output.value, in sizeInfo); + result = logRecord.TrySetValueObject(output.value, in sizeInfo); + break; default: Assert.Fail("Unknown value style"); return false; } + + if (result) + output.value.value += input.value; + return result; } public override bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) @@ -422,8 +429,8 @@ static void SetAndVerify(ref TestObjectInput input, ref TestValueStyle style, bo static RecordFieldInfo GetFieldInfo(ReadOnlySpan key, ref TestObjectInput input) => new() { - KeyDataSize = key.Length, - ValueDataSize = input.wantValueStyle switch + KeySize = key.Length, + ValueSize = input.wantValueStyle switch { TestValueStyle.Inline => ValueStruct.AsSpanByteDataSize, TestValueStyle.Overflow => OverflowValueSize, diff --git a/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs b/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs index f411071c3f0..5144a2c8607 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs @@ -2,6 +2,8 @@ // Licensed under the MIT license. using System; +using System.IO; +using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; @@ -9,7 +11,7 @@ namespace Tsavorite.test { public enum TestValueStyle : byte { None, Inline, Overflow, Object }; - public struct TestObjectKey + public struct TestObjectKey { public int key; @@ -23,16 +25,24 @@ public struct Comparer : IKeyComparer } } - public class TestObjectValue : IHeapObject + public class TestObjectValue : HeapObjectBase { public int value; - public long MemorySize { get => sizeof(int); set => throw new NotImplementedException("TestValueObject.MemorySize.set"); } - public long DiskSize { get => MemorySize; set => throw new NotImplementedException("TestValueObject.MemorySize.set"); } + public override string ToString() => value.ToString(); - public void Dispose() { } + public override void Dispose() { } - public override string ToString() => value.ToString(); + public override HeapObjectBase Clone() => throw new NotImplementedException(); + public override void DoSerialize(BinaryWriter writer) => throw new NotImplementedException(); + public override void WriteType(BinaryWriter writer, bool isNull) => throw new NotImplementedException(); + + public TestObjectValue() + { + HeapMemorySize = sizeof(int); + SerializedSize = HeapMemorySize; + SerializedSizeIsExact = true; + } public class Serializer : BinaryObjectSerializer { @@ -107,11 +117,11 @@ public override bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo si => logRecord.TrySetValueObject(srcValue, in sizeInfo); public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref TestObjectInput input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } public class TestObjectFunctionsDelete : SessionFunctionsBase @@ -167,31 +177,38 @@ public override bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo si => logRecord.TrySetValueObject(srcValue, in sizeInfo); public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref TestObjectInput input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } - public class TestLargeObjectValue : IHeapObject + public class TestLargeObjectValue : HeapObjectBase { public byte[] value; - public long MemorySize { get => DiskSize + 24 /* TODO: ByteArrayOverhead */; set => throw new NotImplementedException("TestValueObject.MemorySize.set"); } - public long DiskSize { get => sizeof(int) + value.Length; set => throw new NotImplementedException("TestValueObject.DiskSize.set"); } + public override HeapObjectBase Clone() => throw new NotImplementedException(); + public override void DoSerialize(BinaryWriter writer) => throw new NotImplementedException(); + public override void WriteType(BinaryWriter writer, bool isNull) => throw new NotImplementedException(); - public void Dispose() { } + public override void Dispose() { } - public TestLargeObjectValue() - { - } + public TestLargeObjectValue() { } - public TestLargeObjectValue(int size) + public TestLargeObjectValue(int size, bool serializedSizeIsExact) { value = new byte[size]; for (int i = 0; i < size; i++) value[i] = (byte)(size + i); + SetSizes(serializedSizeIsExact); + } + + private void SetSizes(bool serializedSizeIsExact = true) + { + SerializedSize = sizeof(int) + value.Length; + HeapMemorySize = SerializedSize + 24; // TODO: ByteArrayOverhead + SerializedSizeIsExact = serializedSizeIsExact; } public class Serializer : BinaryObjectSerializer @@ -201,7 +218,12 @@ public override void Deserialize(out IHeapObject obj) var value = new TestLargeObjectValue(); obj = value; int size = reader.ReadInt32(); + Assert.That(size, Is.Not.EqualTo(0)); + value.value = reader.ReadBytes(size); + Assert.That(value.value.Length, Is.EqualTo(size)); + + value.SetSizes(); } public override void Serialize(IHeapObject obj) @@ -213,52 +235,79 @@ public override void Serialize(IHeapObject obj) } } + public struct TestLargeObjectInput + { + public int value; + + public TestValueStyle wantValueStyle; + + public int expectedSpanLength; + + public override readonly string ToString() => $"value {value}, wantValStyle {wantValueStyle}"; + } + public class TestLargeObjectOutput { - public TestLargeObjectValue value; + public TestLargeObjectValue valueObject; + public byte[] valueArray; } - public class TestLargeObjectFunctions : SessionFunctionsBase + public class TestLargeObjectFunctions : SessionFunctionsBase { - public override void ReadCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestLargeObjectOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestLargeObjectInput input, ref TestLargeObjectOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { - ClassicAssert.IsTrue(status.Found); - for (int i = 0; i < output.value.value.Length; i++) - { - ClassicAssert.AreEqual((byte)(output.value.value.Length + i), output.value.value[i]); - } + Assert.That(status.Found, Is.True); } - public override bool Reader(in TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestLargeObjectOutput output, ref ReadInfo readInfo) + public override bool Reader(in TSourceLogRecord srcLogRecord, ref TestLargeObjectInput input, ref TestLargeObjectOutput output, ref ReadInfo readInfo) { - output.value = (TestLargeObjectValue)srcLogRecord.ValueObject; + switch (input.wantValueStyle) + { + case TestValueStyle.None: + Assert.Fail("wantValueStyle should not be None"); + break; + case TestValueStyle.Inline: + Assert.That(srcLogRecord.Info.ValueIsInline, Is.True); + Assert.That(srcLogRecord.ValueSpan.Length, Is.EqualTo(input.expectedSpanLength)); + output.valueArray = srcLogRecord.ValueSpan.ToArray(); + break; + case TestValueStyle.Overflow: + Assert.That(srcLogRecord.Info.ValueIsOverflow, Is.True); + Assert.That(srcLogRecord.ValueSpan.Length, Is.EqualTo(input.expectedSpanLength)); + output.valueArray = srcLogRecord.ValueSpan.ToArray(); + break; + case TestValueStyle.Object: + Assert.That(srcLogRecord.Info.ValueIsObject, Is.True); + break; + } + output.valueObject = srcLogRecord.Info.ValueIsObject ? (TestLargeObjectValue)srcLogRecord.ValueObject : default; return true; } - public override bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref TestLargeObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) { if (!logRecord.TrySetValueObject(srcValue)) // We should always be non-inline return false; - output.value = (TestLargeObjectValue)logRecord.ValueObject; + output.valueObject = logRecord.Info.ValueIsObject ? (TestLargeObjectValue)logRecord.ValueObject : default; return true; } - public override bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) + public override bool InitialWriter(ref LogRecord logRecord, in RecordSizeInfo sizeInfo, ref TestLargeObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) { if (!logRecord.TrySetValueObject(srcValue)) // We should always be non-inline return false; - output.value = (TestLargeObjectValue)logRecord.ValueObject; + output.valueObject = logRecord.Info.ValueIsObject ? (TestLargeObjectValue)logRecord.ValueObject : default; return true; } - public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref TestObjectInput input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; - public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; - public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; - public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref TestObjectInput input) - => new() { KeyDataSize = key.Length, ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject, + public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref TestLargeObjectInput input) + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestLargeObjectInput input) + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TestLargeObjectInput input) + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, in TSourceLogRecord inputLogRecord, ref TestLargeObjectInput input) + => new() { KeySize = key.Length, ValueSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject, HasETag = inputLogRecord.Info.HasETag, HasExpiration = inputLogRecord.Info.HasExpiration}; } } diff --git a/libs/storage/Tsavorite/cs/test/ObjectTests.cs b/libs/storage/Tsavorite/cs/test/ObjectTests.cs index 0bd778b7ce5..17114ed796c 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectTests.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectTests.cs @@ -1,14 +1,17 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; +using System.Diagnostics; using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; -using static Tsavorite.test.TestUtils; -namespace Tsavorite.test +namespace Tsavorite.test.Objects { + using static TestUtils; + using ClassAllocator = ObjectAllocator>; using ClassStoreFunctions = StoreFunctions; @@ -24,6 +27,9 @@ public void Setup() DeleteDirectory(MethodTestDir, wait: true); log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "ObjectTests.log"), deleteOnClose: true); objlog = Devices.CreateLogDevice(Path.Join(MethodTestDir, "ObjectTests.obj.log"), deleteOnClose: true); + var storeFunctions = TestContext.CurrentContext.Test.MethodName.StartsWith("ObjectDiskWriteReadLarge") + ? StoreFunctions.Create(new TestObjectKey.Comparer(), () => new TestLargeObjectValue.Serializer(), DefaultRecordDisposer.Instance) + : StoreFunctions.Create(new TestObjectKey.Comparer(), () => new TestObjectValue.Serializer(), DefaultRecordDisposer.Instance); store = new(new() { @@ -33,7 +39,7 @@ public void Setup() MutableFraction = 0.1, MemorySize = 1L << 15, PageSize = 1L << 10 - }, StoreFunctions.Create(new TestObjectKey.Comparer(), () => new TestObjectValue.Serializer(), DefaultRecordDisposer.Instance) + }, storeFunctions , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -51,7 +57,7 @@ public void TearDown() } [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] - public void ObjectInMemWriteRead() + public void ObjectInMemWriteReadUpsert() { using var session = store.NewSession(new TestObjectFunctions()); var bContext = session.BasicContext; @@ -69,7 +75,7 @@ public void ObjectInMemWriteRead() } [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] - public void ObjectInMemWriteRead2() + public void ObjectInMemWriteReadRMW() { using var session = store.NewSession(new TestObjectFunctions()); var bContext = session.BasicContext; @@ -94,8 +100,36 @@ public void ObjectInMemWriteRead2() ClassicAssert.AreEqual(input2.value, output.value.value); } -#if LOGRECORD_TODO - [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + [Test, Category(TsavoriteKVTestCategory), Category(LogRecordCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + public void ObjectDiskWriteReadSingle() + { + using var session = store.NewSession(new TestObjectFunctions()); + var bContext = session.BasicContext; + const int keyInt = 42; + + var keyStruct = new TestObjectKey { key = keyInt }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + var value = new TestObjectValue { value = keyInt }; + _ = bContext.Upsert(key, value, Empty.Default); + + TestObjectInput input = new(); + TestObjectOutput output = new(); + var status = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(status.IsPending, Is.False); + Assert.That(status.Found, Is.True); + Assert.That(output.value.value, Is.EqualTo(keyInt)); + + store.Log.FlushAndEvict(wait: true); + + status = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(status.IsPending, Is.True); + Assert.That(bContext.CompletePendingWithOutputs(out var outputs, wait: true), Is.True); + (status, output) = GetSinglePendingResult(outputs); + Assert.That(status.Found, Is.True); + Assert.That(output.value.value, Is.EqualTo(keyInt)); + } + + [Test, Category(TsavoriteKVTestCategory), Category(LogRecordCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] public void ObjectDiskWriteRead() { using var session = store.NewSession(new TestObjectFunctions()); @@ -107,7 +141,6 @@ public void ObjectDiskWriteRead() var key = SpanByte.FromPinnedVariable(ref key1Struct); var value = new TestObjectValue { value = i }; _ = bContext.Upsert(key, value, Empty.Default); - // store.ShiftReadOnlyAddress(store.LogTailAddress); } TestObjectKey key2Struct = new() { key = 23 }; @@ -143,6 +176,7 @@ public void ObjectDiskWriteRead() } // Update first 100 using RMW from storage + var numPendingUpdates = 0; for (int i = 0; i < 100; i++) { var keyStruct = new TestObjectKey { key = i }; @@ -150,9 +184,14 @@ public void ObjectDiskWriteRead() input = new TestObjectInput { value = 1 }; status = bContext.RMW(key, ref input, Empty.Default); if (status.IsPending) + { + numPendingUpdates++; _ = bContext.CompletePending(true); + } } + Assert.That(numPendingUpdates, Is.EqualTo(100)); + var numPendingReads = 0; for (int i = 0; i < 2000; i++) { var output = new TestObjectOutput(); @@ -162,16 +201,173 @@ public void ObjectDiskWriteRead() status = bContext.Read(key, ref input, ref output, Empty.Default); if (status.IsPending) + { + numPendingReads++; (status, output) = bContext.GetSinglePendingResult(); + } + + if (i is < 100 or >= 1900) + ClassicAssert.AreEqual(value.value + 1, output.value.value); else + ClassicAssert.AreEqual(value.value, output.value.value); + } + Assert.That(numPendingReads, Is.GreaterThanOrEqualTo(numPendingUpdates)); + } + + /// Whether value object sizes are known at the time of serialization (Exact, as most Garnet objects) or not (Inexact, e.g. JsonObject). + public enum SerializeValueSizeMode { SerSizeExact, SerSizeInexact }; + + /// Various sizes to test + public enum SerializeKeyValueSize + { + Thirty = 30, + OneK = 1024, + HalfBuffer = IStreamBuffer.BufferSize / 2, + OneBuffer = IStreamBuffer.BufferSize, + ThreeHalfBuffer = (IStreamBuffer.BufferSize / 2) * 3, + TwoBuffer = IStreamBuffer.BufferSize * 2 + } + + [Test, Category(TsavoriteKVTestCategory), Category(LogRecordCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + //[Repeat(300)] + public void ObjectDiskWriteReadLargeValueSmallKey([Values] SerializeValueSizeMode serSizeMode, [Values] SerializeKeyValueSize serializeValueSize) + { + if (TestContext.CurrentContext.CurrentRepeatCount > 0) + Debug.WriteLine($"*** Current test iteration: {TestContext.CurrentContext.CurrentRepeatCount + 1} ***"); + + using var session = store.NewSession(new TestLargeObjectFunctions()); + var bContext = session.BasicContext; + + var input = new TestLargeObjectInput(); + var output = new TestLargeObjectOutput(); + var valueSize = (int)serializeValueSize; + const int numRec = 3; + for (int ii = 0; ii < numRec; ii++) + { + var key1Struct = new TestObjectKey { key = ii }; + var key = SpanByte.FromPinnedVariable(ref key1Struct); + var value = new TestLargeObjectValue(valueSize + (ii * 4096), serializedSizeIsExact: serSizeMode == SerializeValueSizeMode.SerSizeExact); + new Span(value.value).Fill(0x42); + _ = bContext.Upsert(key, ref input, value, ref output); + } + + // Test before and after the flush + DoRead(onDisk: false); + store.Log.FlushAndEvict(wait: true); + DoRead(onDisk: true); + + void DoRead(bool onDisk) + { + TestLargeObjectInput input = new() { wantValueStyle = TestValueStyle.Object }; + for (int ii = 0; ii < numRec; ii++) { - if (i < 100 || i >= 1900) - ClassicAssert.AreEqual(value.value + 1, output.value.value); - else - ClassicAssert.AreEqual(value.value, output.value.value); + var output = new TestLargeObjectOutput(); + var keyStruct = new TestObjectKey { key = ii }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + + var status = bContext.Read(key, ref input, ref output, Empty.Default); + if (status.IsPending) + (status, output) = bContext.GetSinglePendingResult(); + + Assert.That(output.valueObject.value.Length, Is.EqualTo(valueSize + (ii * 4096))); + var numLongs = output.valueObject.value.Length % 8; + var badIndex = new ReadOnlySpan(output.valueObject.value).IndexOfAnyExcept((byte)0x42); + if (badIndex != -1) + Assert.Fail($"Unexpected byte value at index {badIndex}, onDisk {onDisk}, record# {ii}: {output.valueObject.value[badIndex]}"); + } + } + } + + [Test, Category(TsavoriteKVTestCategory), Category(LogRecordCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + public void ObjectDiskWriteReadOverflowValue() + { + using var session = store.NewSession(new TestLargeObjectFunctions()); + var bContext = session.BasicContext; + + var valueSize = IStreamBuffer.BufferSize / 2; + const int numRec = 5; + var valueBuffer = new byte[valueSize * numRec]; + new Span(valueBuffer).Fill(0x42); + + for (int ii = 0; ii < numRec; ii++) + { + var key1Struct = new TestObjectKey { key = ii }; + var key = SpanByte.FromPinnedVariable(ref key1Struct); + var value = new ReadOnlySpan(valueBuffer).Slice(0, valueSize * (ii + 1)); + _ = bContext.Upsert(key, value, Empty.Default); + } + + store.Log.FlushAndEvict(wait: true); + + TestLargeObjectInput input = new() { wantValueStyle = TestValueStyle.Overflow }; + + for (int ii = 0; ii < numRec; ii++) + { + var output = new TestLargeObjectOutput(); + var keyStruct = new TestObjectKey { key = ii }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + + input.expectedSpanLength = valueSize * (ii + 1); + var status = bContext.Read(key, ref input, ref output, Empty.Default); + if (status.IsPending) + (status, output) = bContext.GetSinglePendingResult(); + + Assert.That(output.valueArray.Length, Is.EqualTo(valueSize * (ii + 1))); + Assert.That(new ReadOnlySpan(output.valueArray).SequenceEqual(new ReadOnlySpan(valueBuffer).Slice(0, output.valueArray.Length))); + } + } + + [Test, Category(TsavoriteKVTestCategory), Category(LogRecordCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + //[Repeat(300)] + public void ObjectDiskWriteReadLargeKeyAndValue([Values] SerializeKeyValueSize serializeKeySize, [Values] SerializeValueSizeMode serSizeMode, [Values] SerializeKeyValueSize serializeValueSize) + { + if (TestContext.CurrentContext.CurrentRepeatCount > 0) + Debug.WriteLine($"*** Current test iteration: {TestContext.CurrentContext.CurrentRepeatCount + 1} ***"); + + using var session = store.NewSession(new TestLargeObjectFunctions()); + var bContext = session.BasicContext; + + var input = new TestLargeObjectInput(); + var output = new TestLargeObjectOutput(); + var keySize = (int)serializeKeySize; + var keyBuf = new byte[keySize]; + var valueSize = (int)serializeValueSize; + const int numRec = 3; + for (int ii = 0; ii < numRec; ii++) + { + var value = new TestLargeObjectValue(valueSize + (ii * 4096), serializedSizeIsExact: serSizeMode == SerializeValueSizeMode.SerSizeExact); + var key = new Span(keyBuf); + key.Fill((byte)(ii + 100)); + new Span(value.value).Fill(0x42); + _ = bContext.Upsert(key, ref input, value, ref output); + } + + // Test before and after the flush + DoRead(onDisk: false); + store.Log.FlushAndEvict(wait: true); + DoRead(onDisk: true); + + void DoRead(bool onDisk) + { + TestLargeObjectInput input = new() { wantValueStyle = TestValueStyle.Object }; + for (int ii = 0; ii < numRec; ii++) + { + var output = new TestLargeObjectOutput(); + var key = new Span(keyBuf); + key.Fill((byte)(ii + 100)); + + var status = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(status.IsPending, Is.EqualTo(onDisk)); + if (status.IsPending) + (status, output) = bContext.GetSinglePendingResult(); + Assert.That(status.Found, Is.True); + + Assert.That(output.valueObject.value.Length, Is.EqualTo(valueSize + (ii * 4096)), $"record# ii {ii}"); + var badIndex = new ReadOnlySpan(output.valueObject.value).IndexOfAnyExcept((byte)0x42); + if (badIndex != -1) + Assert.Fail($"Unexpected byte value at index {badIndex}, onDisk {onDisk}, record# {ii}: {output.valueObject.value[badIndex]}"); } } } -#endif // LOGRECORD_TODO } -} +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs b/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs index ccbab20207b..62f438f9a90 100644 --- a/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs +++ b/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs @@ -206,7 +206,7 @@ internal static (long logicalAddress, long physicalAddress) GetHashChain { - var log = isReadCache ? store.readcache : store.hlog; + var log = isReadCache ? store.readCacheBase : store.hlogBase; var info = LogRecord.GetInfo(physicalAddress); var la = info.PreviousAddress; isReadCache = IsReadCache(la); - log = isReadCache ? store.readcache : store.hlog; + log = isReadCache ? store.readCacheBase : store.hlogBase; var pa = log.GetPhysicalAddress(la); recordKey = PinnedSpanByte.FromPinnedSpan(LogRecord.GetInlineKey(pa)); // Must return PinnedSpanByte to avoid scope issues with ReadOnlySpan invalid = LogRecord.GetInfo(pa).Invalid; diff --git a/libs/storage/Tsavorite/cs/test/RevivificationTests.cs b/libs/storage/Tsavorite/cs/test/RevivificationTests.cs index d87f830693c..d5c6bb7b063 100644 --- a/libs/storage/Tsavorite/cs/test/RevivificationTests.cs +++ b/libs/storage/Tsavorite/cs/test/RevivificationTests.cs @@ -37,6 +37,8 @@ internal RevivificationSpanByteComparer(CollisionRange range) namespace Tsavorite.test.Revivification { + using static VarbyteLengthUtility; + #if LOGRECORD_TODO using ClassAllocator = GenericAllocator>>; using ClassStoreFunctions = StoreFunctions>; @@ -65,7 +67,7 @@ struct RevivificationTestUtils [ new RevivificationBin() { - RecordSize = RoundUp(RecordInfo.GetLength() + 2 * (sizeof(int) + sizeof(long)), Constants.kRecordAlignment), // We have "fixed length" for these integer bins, with long Key and Value + RecordSize = RoundUp(RecordInfo.Size + 2 * (sizeof(int) + sizeof(long)), Constants.kRecordAlignment), // We have "fixed length" for these integer bins, with long Key and Value BestFitScanLimit = RevivificationBin.UseFirstFit } ] @@ -222,14 +224,14 @@ internal static void AssertElidable(TsavoriteKV stackCtx = new(store.storeFunctions.GetKeyHashCode64(key)); ClassicAssert.IsTrue(store.FindTag(ref stackCtx.hei), $"AssertElidable: Cannot find key {key.ToShortString()}"); - var recordInfo = LogRecord.GetInfo(store.hlog.GetPhysicalAddress(stackCtx.hei.Address)); + var recordInfo = LogRecord.GetInfo(store.hlogBase.GetPhysicalAddress(stackCtx.hei.Address)); ClassicAssert.Less(recordInfo.PreviousAddress, store.hlogBase.BeginAddress, "AssertElidable: expected elidable key"); } internal static int GetRevivifiableRecordCount(TsavoriteKV store, int numRecords) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator - => (int)(numRecords * store.RevivificationManager.revivifiableFraction); + => (int)(numRecords * store.RevivificationManager.revivifiableFraction); // Add extra for rounding issues internal static int GetMinRevivifiableKey(TsavoriteKV store, int numRecords) where TStoreFunctions : IStoreFunctions @@ -254,12 +256,17 @@ internal class RevivificationFixedLenFunctions : SimpleLongSimpleFunctions private BasicContext bContext; private IDevice log; + private int recordSize; + [SetUp] public void Setup() { DeleteDirectory(MethodTestDir, wait: true); log = Devices.CreateLogDevice(Path.Combine(MethodTestDir, "test.log"), deleteOnClose: true); + // Records all have a Span corresponding to a 'long' key and value, which means one length byte. + recordSize = RoundUp(RecordInfo.Size + NumIndicatorBytes + 2 + sizeof(long) * 2, Constants.kRecordAlignment); + double? revivifiableFraction = default; RecordElision? recordElision = default; foreach (var arg in TestContext.CurrentContext.Test.Arguments) @@ -329,7 +336,7 @@ public void SimpleFixedLenTest([Values] DeleteDest deleteDest, [Values(UpdateOp. if (stayInChain) _ = RevivificationTestUtils.SwapFreeRecordPool(store, default); - long deleteKeyNum = RevivificationTestUtils.GetMinRevivifiableKey(store, NumRecords); + long deleteKeyNum = RevivificationTestUtils.GetMinRevivifiableKey(store, NumRecords) + 2; // +2 to allow for page headers and rounding var deleteKey = SpanByte.FromPinnedVariable(ref deleteKeyNum); if (!stayInChain) RevivificationTestUtils.AssertElidable(store, deleteKey); @@ -386,12 +393,11 @@ public void UnelideTest([Values] RecordElision elision, [Values(UpdateOp.Upsert, // Now re-add the keys. For the elision case, we should see tailAddress grow sharply as only the records in the bin are available // for revivification. For In-Chain, we will revivify records that were unelided after the bin overflowed. But we have some records // ineligible for revivification due to revivifiableFraction. - var recordSize = RoundUp(RecordInfo.GetLength() + (sizeof(int) + sizeof(long)) * 2, Constants.kRecordAlignment); var numIneligibleRecords = NumRecords - RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords); var noElisionExpectedTailAddress = tailAddress + numIneligibleRecords * recordSize; - if (elision == RecordElision.NoElide) - ClassicAssert.AreEqual(noElisionExpectedTailAddress, store.Log.TailAddress, "Expected tail address not to grow (records were revivified)"); + if (elision == RecordElision.NoElide) // Add 4 to account for page headers and rounding + ClassicAssert.GreaterOrEqual(noElisionExpectedTailAddress + 4 * recordSize, store.Log.TailAddress, "Expected tail address not to grow (records were revivified)"); else ClassicAssert.Less(noElisionExpectedTailAddress, store.Log.TailAddress, "Expected tail address to grow (records were not revivified)"); } @@ -524,7 +530,10 @@ void CheckExpectedLengthsBefore(ref LogRecord logRecord, in RecordSizeInfo sizeI // If an overflow logRecord is from new record creation it has not had its overflow set yet; it has just been initialized to inline length of ObjectIdMap.ObjectIdSize, // and we'll call LogField.ConvertToOverflow later in this ISessionFunctions call to do the actual overflow allocation. if (!logRecord.Info.ValueIsInline || (sizeInfo.IsSet && !sizeInfo.ValueIsInline)) - ClassicAssert.AreEqual(ObjectIdMap.ObjectIdSize, LogField.GetTotalSizeOfInlineField(logRecord.ValueAddress)); + { + var (valueLength, valueAddress) = GetValueFieldInfo(logRecord.IndicatorAddress); + ClassicAssert.AreEqual(ObjectIdMap.ObjectIdSize, (int)valueLength); + } if (sizeInfo.ValueIsInline) ClassicAssert.AreEqual(expectedValueLength, logRecord.ValueSpan.Length); else @@ -658,13 +667,13 @@ public override void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref // Override the default SpanByteFunctions impelementation; for these tests, we always want the input length. /// public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref PinnedSpanByte input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = input.Length }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = input.Length }; /// public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref PinnedSpanByte input) - => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + => new() { KeySize = key.Length, ValueSize = input.Length }; /// public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref PinnedSpanByte input) - => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + => new() { KeySize = key.Length, ValueSize = input.Length }; } const int NumRecords = 200; @@ -874,7 +883,10 @@ public void SpanByteSimpleTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp functions.expectedValueLengths.Enqueue(InitialLength); Status status = PerformDeletion(deletionRoute, key, fillByte); - ClassicAssert.IsTrue(status.Found, status.ToString()); + //if (deletionRoute == DeletionRoutes.DELETE) + ClassicAssert.IsTrue(status.Found, status.ToString()); + //else + // ClassicAssert.IsTrue(status.NotFound && status.IsExpired, status.ToString()); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); @@ -915,7 +927,8 @@ public void SpanByteDeletionViaRMWRCURevivifiesOriginalRecordAfterTombstoning( ClassicAssert.AreEqual(1, RevivificationTestUtils.GetFreeRecordCount(store)); - ClassicAssert.IsTrue(status.Found, status.ToString()); + //ClassicAssert.IsTrue(status.NotFound && status.IsExpired, status.ToString()); + ClassicAssert.IsTrue(status.Found && status.IsExpired, status.ToString()); var tailAddress = store.Log.TailAddress; @@ -1270,9 +1283,6 @@ public void DeleteAllRecordsAndRevivifyTest([Values(CollisionRange.None)] Collis Span key = stackalloc byte[KeyLength]; - // "sizeof(int) +" because SpanByte has an int length prefix - var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + key.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); - // Delete for (var ii = 0; ii < NumRecords; ++ii) { @@ -1419,7 +1429,7 @@ public unsafe void LiveBinWrappingTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); // "sizeof(int) +" because SpanByte has an int length prefix. - var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + key.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); + var recordSize = RecordInfo.Size + RoundUp(sizeof(int) + key.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); ClassicAssert.IsTrue(pool.GetBinIndex(recordSize, out int binIndex)); ClassicAssert.AreEqual(3, binIndex); diff --git a/libs/storage/Tsavorite/cs/test/TestTypes.cs b/libs/storage/Tsavorite/cs/test/TestTypes.cs index cec8cc5b09e..aa30a3457fc 100644 --- a/libs/storage/Tsavorite/cs/test/TestTypes.cs +++ b/libs/storage/Tsavorite/cs/test/TestTypes.cs @@ -134,13 +134,13 @@ public override bool CopyUpdater(in TSourceLogRecord srcLogRec /// public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref InputStruct input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; } public class FunctionsCompaction : SessionFunctionsBase @@ -204,13 +204,13 @@ public override bool CopyUpdater(in TSourceLogRecord srcLogRec /// public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref InputStruct input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; } public class FunctionsCopyOnWrite : SessionFunctionsBase @@ -288,13 +288,13 @@ public override bool CopyUpdater(in TSourceLogRecord srcLogRec /// public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref InputStruct input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + => new() { KeySize = key.Length, ValueSize = sizeof(ValueStruct) }; /// public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; } public class SimpleLongSimpleFunctions : SessionFunctionsBase @@ -354,12 +354,12 @@ public override bool InPlaceUpdater(ref LogRecord logRecord, in RecordSizeInfo s /// public override RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref long input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(long) }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = sizeof(long) }; /// public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref long input) - => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(long) }; + => new() { KeySize = key.Length, ValueSize = sizeof(long) }; /// public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref long input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + => new() { KeySize = key.Length, ValueSize = value.Length }; } } \ No newline at end of file diff --git a/main/GarnetServer/Extensions/MyDictObject.cs b/main/GarnetServer/Extensions/MyDictObject.cs index b82aa95f2ff..7e792b1bb4f 100644 --- a/main/GarnetServer/Extensions/MyDictObject.cs +++ b/main/GarnetServer/Extensions/MyDictObject.cs @@ -143,19 +143,19 @@ private void UpdateSize(byte[] key, byte[] value, bool add = true) { var memorySize = Utility.RoundUp(key.Length, IntPtr.Size) + Utility.RoundUp(value.Length, IntPtr.Size) + (2 * MemoryUtils.ByteArrayOverhead) + MemoryUtils.DictionaryEntryOverhead; - var kvSize = sizeof(int) * 2 + key.Length + value.Length; + var kvSize = (sizeof(int) * 2) + key.Length + value.Length; if (add) { - this.MemorySize += memorySize; - this.DiskSize += kvSize; + HeapMemorySize += memorySize; + SerializedSize += kvSize; } else { - this.MemorySize -= memorySize; - this.DiskSize -= kvSize; - Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); - Debug.Assert(this.DiskSize >= sizeof(int)); + HeapMemorySize -= memorySize; + SerializedSize -= kvSize; + Debug.Assert(HeapMemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(SerializedSize >= sizeof(int)); } } diff --git a/modules/GarnetJSON/GarnetJsonObject.cs b/modules/GarnetJSON/GarnetJsonObject.cs index 69e3d011198..0078a5da47b 100644 --- a/modules/GarnetJSON/GarnetJsonObject.cs +++ b/modules/GarnetJSON/GarnetJsonObject.cs @@ -224,7 +224,8 @@ indent is null && newLine is null && space is null /// The error message if the operation fails. /// The result of the set operation. /// Thrown when there is an error in JSON processing. - /// TODO: This currently does not update or + /// TODO: This currently does not update or + /// and does not support . public SetResult Set(ReadOnlySpan path, ReadOnlySpan value, ExistOptions existOptions, out ReadOnlySpan errorMessage) { try diff --git a/test/Garnet.test.cluster/ClusterMigrateTests.cs b/test/Garnet.test.cluster/ClusterMigrateTests.cs index 70a412b7ad9..65c7bbd02f0 100644 --- a/test/Garnet.test.cluster/ClusterMigrateTests.cs +++ b/test/Garnet.test.cluster/ClusterMigrateTests.cs @@ -1096,7 +1096,7 @@ public void ClusterSimpleMigrateKeysWithObjects() { var resp = DoZCOUNT(targetNodeIndex, key, out var count, out _Address, out _Port, out _Slot, logger: context.logger); ClassicAssert.AreEqual(resp, "OK"); - ClassicAssert.AreEqual(data[_key].Count, count); + ClassicAssert.AreEqual(data[_key].Count, count, $"key: {Encoding.UTF8.GetString(_key)}"); List members; (resp, members) = DoZRANGE(targetNodeIndex, _key, out _Address, out _Port, out _Slot, context.logger); diff --git a/test/Garnet.test/GarnetBitmapTests.cs b/test/Garnet.test/GarnetBitmapTests.cs index fce41e69723..bc59848ae14 100644 --- a/test/Garnet.test/GarnetBitmapTests.cs +++ b/test/Garnet.test/GarnetBitmapTests.cs @@ -264,7 +264,7 @@ public void BitmapSetGetBitTest_LTM(bool preSet) bool expectedVal = false; if (state.ContainsKey(key) && state[key].ContainsKey(offset)) expectedVal = state[key][offset]; - ClassicAssert.AreEqual(expectedVal, returnedVal, $"{offset}"); + ClassicAssert.AreEqual(expectedVal, returnedVal, $"offset {offset}"); } } } @@ -402,7 +402,7 @@ public void BitmapBitCountBetweenOffsetsTest() long expectedCount = Count(bitmap, 0, -1); count = db.StringBitCount(key, 0, -1); - ClassicAssert.AreEqual(expectedCount, count, $"{0} {-1} {bitmap.Length}"); + ClassicAssert.AreEqual(expectedCount, count, $"startOffset {0}, endOffset {-1}, bitmapLength {bitmap.Length}"); //Test with startOffset for (int i = 0; i < iter; i++) @@ -411,7 +411,7 @@ public void BitmapBitCountBetweenOffsetsTest() expectedCount = Count(bitmap, startOffset, -1); count = db.StringBitCount(key, startOffset); - ClassicAssert.AreEqual(expectedCount, count, $"{startOffset} {-1} {maxSizeInBytes}"); + ClassicAssert.AreEqual(expectedCount, count, $"startOffset {startOffset}, endOffset {-1}, maxSizeInBytes {maxSizeInBytes}"); } //Test with startOffset and endOffset @@ -422,7 +422,7 @@ public void BitmapBitCountBetweenOffsetsTest() expectedCount = Count(bitmap, startOffset, endOffset); count = db.StringBitCount(key, startOffset, endOffset); - ClassicAssert.AreEqual(expectedCount, count, $"{startOffset} {endOffset} {maxSizeInBytes}"); + ClassicAssert.AreEqual(expectedCount, count, $"startOffset {startOffset}, endOffset {endOffset}, maxSizeInBytes {maxSizeInBytes}"); } } diff --git a/test/Garnet.test/GarnetObjectTests.cs b/test/Garnet.test/GarnetObjectTests.cs index 8e365f1b0a0..3b69df8e6e2 100644 --- a/test/Garnet.test/GarnetObjectTests.cs +++ b/test/Garnet.test/GarnetObjectTests.cs @@ -94,7 +94,7 @@ void LocalRead() } [Test] - public async Task CopyUpdate() + public async Task WriteCheckpointCopyUpdate() { IGarnetObject obj = new SortedSetObject(); @@ -145,18 +145,18 @@ public override bool Reader(in TSourceLogRecord srcLogRecord, public override bool CopyUpdater(in TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, in RecordSizeInfo sizeInfo, ref IGarnetObject input, ref IGarnetObject output, ref RMWInfo rmwInfo) { - _ = ((IGarnetObject)srcLogRecord.ValueObject).CopyUpdate(srcLogRecord.Info.IsInNewVersion, ref rmwInfo); + Assert.That(dstLogRecord.Info.ValueIsObject, Is.True); return true; } public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(in TSourceLogRecord srcLogRecord, ref IGarnetObject input) - => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = srcLogRecord.Key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref IGarnetObject input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref IGarnetObject input) - => new() { KeyDataSize = key.Length, ValueDataSize = value.Length, ValueIsObject = false }; + => new() { KeySize = key.Length, ValueSize = value.Length, ValueIsObject = false }; public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref IGarnetObject input) - => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + => new() { KeySize = key.Length, ValueSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } private void CreateStore() diff --git a/test/Garnet.test/RespEtagTests.cs b/test/Garnet.test/RespEtagTests.cs index 61b193e714f..8c6e93c627e 100644 --- a/test/Garnet.test/RespEtagTests.cs +++ b/test/Garnet.test/RespEtagTests.cs @@ -288,7 +288,7 @@ public void GetIfNotMatchReturnsDataWhenEtagDoesNotMatch() } [Test] - public void SetWithEtagWorksWithMetadata() + public void SetWithEtagWorksWithExpiration() { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); IDatabase db = redis.GetDatabase(0); diff --git a/test/Garnet.test/RespListTests.cs b/test/Garnet.test/RespListTests.cs index d2ff055c7a3..d7c06ae42a0 100644 --- a/test/Garnet.test/RespListTests.cs +++ b/test/Garnet.test/RespListTests.cs @@ -171,11 +171,9 @@ public void MultiLPUSHAndLLENWithPendingStatus() var db = redis.GetDatabase(0); var nVals = 100; - RedisValue[] values = new RedisValue[nVals]; + var values = new RedisValue[nVals]; for (int i = 0; i < 100; i++) - { - values[i] = ($"val-{i + 1}"); - } + values[i] = $"val-{i + 1}"; for (int j = 0; j < 25; j++) { @@ -183,7 +181,7 @@ public void MultiLPUSHAndLLENWithPendingStatus() ClassicAssert.AreEqual(nVals, nAdded); } - long nLen = db.ListLength("List_Test-10"); + var nLen = db.ListLength("List_Test-10"); ClassicAssert.AreEqual(100, nLen); } diff --git a/website/docs/dev/tsavorite/logrecord.md b/website/docs/dev/tsavorite/logrecord.md index 4aa9750389c..cab550ada66 100644 --- a/website/docs/dev/tsavorite/logrecord.md +++ b/website/docs/dev/tsavorite/logrecord.md @@ -6,10 +6,12 @@ title: LogRecord # `LogRecord` -The `LogRecord` struct is a major revision in the Tsavorite `ISessionFunctions` design. It replaces individual `ref key` and `ref value` parameters in the `ISessionFunctions` methods (as well as endoding optional `ETag` and `Expirattion` into the Value) with a single `LogRecord`, which may be either `LogRecord` for in-memory log records, or `DiskLogRecord` for on-disk records. These `LogRecord` have properties for `Key` and `Value` as well as making `Etag` and `Expiration` first-class properties. There are a number of additional changes in this design as well, as shown in the following sections. +The `LogRecord` struct is a major revision in the Tsavorite `ISessionFunctions` design. It replaces individual `ref key` and `ref value` parameters in the `ISessionFunctions` methods (as well as endoding optional `ETag` and `Expiration` into the Value) with a single `LogRecord`, which may be either `LogRecord` for in-memory log records, or `DiskLogRecord` for on-disk records. These `LogRecord` have properties for `Key` and `Value` as well as making `Etag` and `Expiration` first-class properties. There are a number of additional changes in this design as well, as shown in the following sections. Much of the record-related logic of the allocators (e.g. `SpanByteAllocator`) has been moved into the `LogRecord` structs. +See LogRecord.cs for details of the layout, including `RecordType`, `Namespace`, and the ObjectLogPosition ulong if the record is not inline (has an Overflow Key and/or an Overflow or Object value). + ## `SpanByte` and `ArgSlice` are now `PinnedSpanByte` or `ReadOnlySpan` To clarify that the element must be a pointer to a pinned span of bytes, the `SpanByte` and `ArgSlice` types have been replaced with `PinnedSpanByte` and `ReadOnlySpan`. The `PinnedSpanByte` is similar to the earlier `SpanByte`; a struct that wraps a pointer to a pinned span of bytes. Its construction has been changed from direct constructor calls to static `FromPinned*` calls, e.g. `FromLengthPrefixedPinnedPointer`. This is mostly used for cases where `(ReadOnly)Span` are not possible due to restrictions on their use; further work could reduce these areas. @@ -36,11 +38,11 @@ With the move to `SpanByte`-only keys we also created a new `ObjectAllocator` fo An object field's object must inherit from `IHeapObject`. The Garnet processing layer uses `IGarnetObject`, which inherits from `IHeapObject`). The Tsavorite Unit Tests use object types that implement `IHeapObject`. -`IHeapObject` provides two basic properties: +`IHeapObject` provides two basic properties for size tracking: - MemorySize: The size the object takes in memory. This includes .NET object overhead as well as the size of the actual data. It is used in object size tracking. -- DiskSize: The size the object will take when serialized to disk format. This usually includes a length prefix followed by the actual data. +- DiskSize: The size the object will take when serialized to disk. This usually includes a length prefix followed by the actual data. JsonObject cannot efficiently track this on a per-modification basis, so there is also a "SerializedSizeIsExact" field that lets us know we can serialize the object with a known length. For other objects, Garnet ensures that each operation on an object (such as a `SortedSet`) keeps these fields up to date. -Garnet ensures that each operation on an object (such as a `SortedSet`) keeps these fields up to date. +There are a number of other methods on IHeapObject, mostly to handle serialization. ### `ObjectIdMap` In `ObjectAllocator` we have an `ObjectIdMap` that provides a GC root for objects (and overflows, as discussed next). In the log record itself, there is a 4-byte `ObjectId` that is an index into the `ObjectIdMap`. @@ -111,6 +113,8 @@ In the `ObjectAllocator`, `TrySetValueLength` also manages conversion between th Although `TrySetValueLength` allocates the `ObjectId` slot, it does not know the actual object, so the `ISessionFunctions` implementation must create the object and call `TrySetValueObject`. + Performance note: `TrySetValueLength` handles all conversions. It should be beneficial to provide some leaner versions, for example string-only when lengths are unlikely to change. + #### RecordSizeInfo This structure is populated prior to record allocation (it is necessary to know what size to allocate from the log), and then is passed through to `ISessionFunctions` implementation and subsequently to the `LogRecord`. The flow is: @@ -135,19 +139,13 @@ As a terminology note, `LogField` (and `RecordSizeInfo` and `LogRecord`) use the ### DiskLogRecord struct -The DiskLogRecord is an `ISourceLogRecord` that is backed by a record in on-disk format. See `DiskLogRecord.cs` for more details, inluding the record layout and comments. It is a read-only record. - -In on-disk format the data has two forms: -- Keys and Values are stored inline, as byte streams. This is a direct copy of the inline `LogRecord` layout, and is intended to support the `SpanByteAllocator` inline case being written directly to disk. -- An optimized layout where the `RecordInfo` is followed immediately by an indicator byte that contains version of the layout design and the number of bits for key and value lengths. The lengths immediately follow (i.e. the Value length is directly after the key length, and before the key data). This ensures that if we do not get the full record on the first IO, we know the length needed to read the entire record on the next IO. - -In both of these layouts the optionals are stored after the Value, as in `LogRecord`. The `RecordInfo` knows whether they are present, and thus their length is known when calculating IO size. +The DiskLogRecord is an `ISourceLogRecord` that is backed by a `LogRecord`. See `DiskLogRecord.cs` for more details. Its main purpose is to act as a container for a `LogRecord` and the `SectorAlignedMemory` buffer and value-object disposer associated with that `LogRecord`. ### PendingContext `PendingContext` implements `ISourceLogRecord` because it carries a information through the IO process and provides the source record for RMW copy updates. -Previously `PendingContext` had separate `HeapContainers` for keys and values. However, for operations such as conditional insert for Copy-To-Tail or Compaction, we need to carry through the entire log record (including optionals). In the case of records read from disk (e.g. Compaction), it is easiest to pass the `DiskLogRecord` in its entirety, including its `SectorAlignedMemory` buffer. So now PendingContext will also serialize the Key passed to Upsert or RMW, and the value passed to Upsert, as a `DiskLogRecord`. `PendingContext` still carries the `HeapContainer` for Input, and `CompletedOutputs` must still retain the Key's `HeapContainer`. +Previously `PendingContext` had separate `HeapContainers` for keys and values. However, for operations such as conditional insert for Copy-To-Tail or Compaction, we need to carry through the entire log record (including optionals). In the case of records read from disk (e.g. Compaction), it is easiest to pass the `LogRecord` in its entirety, including its `SectorAlignedMemory` buffer, in the `DiskLogRecord`. So now PendingContext will also serialize the Key passed to Upsert or RMW, and the value passed to Upsert, as a `DiskLogRecord`. `PendingContext` still carries the `HeapContainer` for Input, and `CompletedOutputs` must still retain the Key's `HeapContainer`. For Compaction or other operations that must carry an in-memory record's data through the pending process, `PendingContext` serializes that in-memory `LogRecord` to its `DiskLogRecord`. @@ -161,4 +159,4 @@ For Compaction or other operations that must carry an in-memory record's data th ## Migration and Replication -Key migration and diskless Replication have been converted to serialize the record to a `DiskLogRecord` on the sending side, and on the receiving side call one of the new `Upsert` overloads that take a `TSourceLogRecord` as the Value. \ No newline at end of file +Key migration and diskless Replication have been converted to serialize the record to a `DiskLogRecord` on the sending side, and on the receiving side call one of the new `Upsert` overloads that take a `TSourceLogRecord` as the Value. This serialization mimics the writing to disk, but instead of a separate file or memory allocation, it allocates one chunk large enough for the entire inline portion followed by the out-of-line portions appended after the inline portion. Note that this limits the capacity of out-of-line allocations to a single network buffer; there is a pending work item to provide "chunked" output to (and read from) the network buffer. \ No newline at end of file diff --git a/website/docs/dev/tsavorite/object-allocator.md b/website/docs/dev/tsavorite/object-allocator.md index 702203a35b8..30639923678 100644 --- a/website/docs/dev/tsavorite/object-allocator.md +++ b/website/docs/dev/tsavorite/object-allocator.md @@ -13,12 +13,26 @@ The `ObjectAllocator` replaces the `GenericAllocator` to provide two important i - If the key or value is overflow, it is written inline into the main log record on flush, and addresses are modified. - If the value is an object, it is serialized inline. -Garnet uses a two-allocator scheme: +Tsavorite provides two allocators: - Strings are stored in a version of `TsavoriteKV` that uses a `SpanByteAllocator`. -- Objects are stored in a version of `TsavoriteKV` that uses an `ObjectAllocator`. +- Objects are stored in a version of `TsavoriteKV` that uses an `ObjectAllocator`. This is also the "unified allocator" used by Garnet. One big difference between the two is that `SpanByteAllocator` allows larger pages for strings, while `ObjectAllocator` allows using a smaller page size because objects use only 4 byte identifiers in the inline log record. Thus, the page size can be much smaller, allowing finer control over Object size tracking and memory-budget enforcement. Either allocator can also set the Key and Value max inline sizes to cause the field to be stored in an overflow allocation, although this is less performant. -## Address Expansion +## Two Log Files -To be filled in when the implementation is underway. +ObjectAllocator supports two log files simlarly to GenericAllocator. However, GenericAllocator was limited to a fixed 100MB buffer size for object serialization. ObjectAllocator uses a circular buffer system, currently 4 buffers of 4MB each, and writes one to disk while populating the next (or reads the following buffers from disk while processing the current one, e.g. deserializing objects. + +The key points here are AsyncWrite, AsyncRead, AsyncGetFromDisk, and iterators. The basic flow is: +- Flush: + - This is AsyncFlushPagesForReadOnly and Checkpointing. AsyncFlushPagesForReadOnly is a loop on partial or complete pages; it creates a CircularDiskWriteBuffer that is reused throughout the entire call to AsyncFlushPagesForReadOnly, for all the partial ranges. The main driver is an ObjectLogWriter, which handles writing for Overflow (large, out-of-line allocations that are byte[]; this writing is optimized to minimize copies) strings and serialization of objects. The basic operation of the circular buffer is to track the current DiskWriteBuffer's current position, last flushed position, and end position (e.g. if it ends in the middle of a segment). +- Read: + - This may be either AsyncGetFromDisk for RUMD operations or Scan, either for iteration or recovery. Similar to Flush, it creates a CircularDiskReadBuffer composed of multiple DiskReadBuffers, and issues disk reads ahead of the operation on the buffer (such as object deserialization). + +### Adaptive Fields in LogRecord +When we serialize a LogRecord to disk, as we write the object log file we also modify the disk-image of the LogRecord (but, importantly, do not modify the length of the logRecord, to keep the LogicalAddress space consistent). +- We record the position in the Log file in the ObjectLogPosition ulong that is allocated for the record when we specify non-inline. +- We reuse the ObjectId space for the length of that field. For Value objects, we also reuse one byte of the varbyte indicator word's valueLenght, giving a total of 40 bits or 1 TB of address for a single object. + +### ObjectIdMap Remapping +The ObjectAllocator carries a second ObjectIdMap, a transient one intended for IO and iterator and pending operations. (These are the same scenarios that embed a LogRecord into a DiskLogRecord.) In these cases we remap the ObjectIds in the temporary image to use the TransientObjectIdMap, so pages can be evicted and their ObjectIdMaps reused. \ No newline at end of file